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:25 UTC

[trafficcontrol] branch master updated (83ef179 -> 71972ef)

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

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


    from 83ef179  Fix license for moved lib/pq
     new cf101fc  convert all implementations of CRUD interfaces to use Keys methods
     new 43f00c3  all packages through origins (alphabetically) refactored
     new 084310b  converted all packages to use db-less interfaces
     new 4d980d9  fix cachegroup where clause info to match select statement
     new 70fe7c0  remove nested tenancy check from deliveryservice request read method
     new c679b49  fix bug with insert query introduced by conflict
     new 6056ee0  fix tenants test by resetting id_seq and fixing error message
     new a323f5a  fix profiles by removing nested query with rows.Next loop and fix routes for deliveryservice_requests
     new ec1e7f8  run goimports on all files
     new 840729a  convert tenancy package to new interfaces
     new 0ad2e84  define CRUDFactory function type to cleanup signatures
     new 8fe0a12  convert Validate functions to return a single error with all validation issues
     new 2c495a9  documentation cleanup and code simplifications
     new 6dc69de  fix tests broken by validation error change
     new 71972ef  fix up places where IsTenancyEnabledTx now returns an error

The 15 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 lib/go-tc/deliveryservices.go                      |   8 +-
 lib/go-tc/parameters.go                            |  46 +--
 traffic_ops/testing/api/v13/tenants_test.go        |   3 +-
 traffic_ops/testing/api/v13/todb.go                |   5 +-
 traffic_ops/traffic_ops_golang/api/api.go          |   8 +-
 traffic_ops/traffic_ops_golang/api/change_log.go   |  24 +-
 .../traffic_ops_golang/api/change_log_test.go      |   3 +-
 .../traffic_ops_golang/api/shared_handlers.go      | 176 +++++++----
 .../traffic_ops_golang/api/shared_handlers_test.go | 113 ++++---
 .../traffic_ops_golang/api/shared_interfaces.go    |  26 +-
 traffic_ops/traffic_ops_golang/asn/asns.go         | 199 ++++---------
 traffic_ops/traffic_ops_golang/asn/asns_test.go    |  30 +-
 .../traffic_ops_golang/cachegroup/cachegroups.go   | 205 +++++--------
 .../cachegroup/cachegroups_test.go                 |  30 +-
 .../traffic_ops_golang/cachegroup/queueupdate.go   |   2 +-
 traffic_ops/traffic_ops_golang/cdn/cdns.go         | 112 ++-----
 traffic_ops/traffic_ops_golang/cdn/cdns_test.go    |  29 +-
 traffic_ops/traffic_ops_golang/cdn/queue.go        |   2 +-
 .../traffic_ops_golang/coordinate/coordinates.go   | 103 ++-----
 .../coordinate/coordinates_test.go                 |  30 +-
 .../traffic_ops_golang/crconfig/deliveryservice.go |   2 +-
 .../traffic_ops_golang/crconfig/edgelocations.go   |   1 -
 traffic_ops/traffic_ops_golang/crconfig/handler.go |   4 +-
 .../deliveryservice/deliveryservicesv12.go         | 234 +++++++++------
 .../deliveryservice/deliveryservicesv13.go         | 330 ++++++++++-----------
 .../deliveryservice/request/comment/comments.go    | 115 ++-----
 .../request/comment/comments_test.go               |  18 +-
 .../deliveryservice/request/requests.go            | 240 +++++----------
 .../deliveryservice/request/requests_test.go       |  26 +-
 .../deliveryservice/request/validate.go            |  27 +-
 .../deliveryservice/servers/servers.go             |  23 +-
 .../deliveryservicesregexes.go                     |  14 +-
 .../traffic_ops_golang/division/divisions.go       | 152 ++--------
 .../traffic_ops_golang/division/divisions_test.go  |   8 +-
 traffic_ops/traffic_ops_golang/origin/origins.go   | 166 ++++-------
 .../traffic_ops_golang/origin/origins_test.go      |  30 +-
 .../traffic_ops_golang/parameter/parameters.go     | 149 +++-------
 .../parameter/parameters_test.go                   |   9 +-
 .../physlocation/phys_locations.go                 | 103 ++-----
 .../physlocation/phys_locations_test.go            |  14 +-
 traffic_ops/traffic_ops_golang/profile/profiles.go | 126 +++-----
 .../traffic_ops_golang/profile/profiles_test.go    |  14 +-
 .../profileparameter/profile_parameters.go         |  82 ++---
 .../profileparameter/profile_parameters_test.go    |   8 +-
 traffic_ops/traffic_ops_golang/region/regions.go   | 107 ++-----
 .../traffic_ops_golang/region/regions_test.go      |  10 +-
 traffic_ops/traffic_ops_golang/riaksvc/dsutil.go   |  51 ++++
 traffic_ops/traffic_ops_golang/role/roles.go       | 124 ++------
 traffic_ops/traffic_ops_golang/role/roles_test.go  |  19 +-
 traffic_ops/traffic_ops_golang/routes.go           | 239 ++++++++-------
 traffic_ops/traffic_ops_golang/server/servers.go   | 133 +++------
 .../traffic_ops_golang/server/servers_test.go      |   3 +-
 .../staticdnsentry/staticdnsentry.go               |  19 +-
 traffic_ops/traffic_ops_golang/status/statuses.go  | 106 ++-----
 .../traffic_ops_golang/status/statuses_test.go     |  10 +-
 .../traffic_ops_golang/systeminfo/system_info.go   |   8 +-
 traffic_ops/traffic_ops_golang/tenant/tenancy.go   | 143 ++++-----
 traffic_ops/traffic_ops_golang/test/helpers.go     |  13 +
 traffic_ops/traffic_ops_golang/types/types.go      | 105 ++-----
 traffic_ops/traffic_ops_golang/types/types_test.go |  14 +-
 60 files changed, 1623 insertions(+), 2530 deletions(-)


[trafficcontrol] 03/15: converted all packages to use db-less interfaces

Posted by da...@apache.org.
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 084310b24bff938bf2d8c0b8602adbe45ddd8186
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 17:08:29 2018 -0600

    converted all packages to use db-less interfaces
---
 .../traffic_ops_golang/api/shared_handlers.go      |  46 +++++++
 .../deliveryservice/deliveryservicesv12.go         |   4 +-
 .../deliveryservice/deliveryservicesv13.go         |   4 +-
 .../deliveryservice/servers/servers.go             |  18 ++-
 .../traffic_ops_golang/parameter/parameters.go     | 146 ++++++---------------
 .../parameter/parameters_test.go                   |   9 +-
 .../physlocation/phys_locations.go                 | 100 +++-----------
 .../physlocation/phys_locations_test.go            |  10 +-
 traffic_ops/traffic_ops_golang/profile/profiles.go | 108 ++++-----------
 .../traffic_ops_golang/profile/profiles_test.go    |  10 +-
 .../profileparameter/profile_parameters.go         |  79 +++--------
 .../profileparameter/profile_parameters_test.go    |   9 +-
 traffic_ops/traffic_ops_golang/region/regions.go   | 102 +++-----------
 .../traffic_ops_golang/region/regions_test.go      |  10 +-
 traffic_ops/traffic_ops_golang/role/roles.go       | 119 ++++-------------
 traffic_ops/traffic_ops_golang/role/roles_test.go  |   9 +-
 traffic_ops/traffic_ops_golang/routes.go           | 120 ++++++++---------
 traffic_ops/traffic_ops_golang/server/servers.go   | 114 ++++------------
 .../traffic_ops_golang/server/servers_test.go      |   3 +-
 .../staticdnsentry/staticdnsentry.go               |  19 ++-
 traffic_ops/traffic_ops_golang/status/statuses.go  | 103 +++------------
 .../traffic_ops_golang/status/statuses_test.go     |  10 +-
 traffic_ops/traffic_ops_golang/types/types.go      | 102 +++-----------
 traffic_ops/traffic_ops_golang/types/types_test.go |  11 +-
 24 files changed, 407 insertions(+), 858 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index b01eb85..4a10cdb 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -162,6 +162,52 @@ func ReadHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 	}
 }
 
+//this creates a handler function from the pointer to a struct implementing the Reader interface
+//      this handler retrieves the user from the context
+//      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 ReadOnlyHandler(typeFactory func(reqInfo *APIInfo) Reader) 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()
+
+		// Load the PathParams into the query parameters for pass through
+		params, err := GetCombinedParams(r)
+		if err != nil {
+			log.Errorf("unable to get parameters from request: %s", err)
+			handleErrs(http.StatusInternalServerError, err)
+		}
+
+		reader := typeFactory(inf)
+
+		results, errs, errType := reader.Read(params)
+		if len(errs) > 0 {
+			tc.HandleErrorsWithType(errs, errType, handleErrs)
+			return
+		}
+		resp := struct {
+			Response []interface{} `json:"response"`
+		}{results}
+
+		respBts, err := json.Marshal(resp)
+		if err != nil {
+			handleErrs(http.StatusInternalServerError, err)
+			return
+		}
+
+		w.Header().Set("Content-Type", "application/json")
+		fmt.Fprintf(w, "%s", respBts)
+	}
+}
+
 //this creates a handler function from the pointer to a struct implementing the Updater interface
 //it must be immediately assigned to a local variable
 //   this generic handler encapsulates the logic for handling:
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index a783a90..aa0d36a 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -126,8 +126,8 @@ 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, tx *sqlx.Tx) (bool, error) {
-	return isTenantAuthorized(user, tx, &ds.DeliveryServiceNullableV12)
+func (ds *TODeliveryServiceV12) IsTenantAuthorized(user *auth.CurrentUser) (bool, error) {
+	return isTenantAuthorized(user, ds.ReqInfo.Tx, &ds.DeliveryServiceNullableV12)
 }
 
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 994d498..23d9d01 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -579,8 +579,8 @@ func (ds *TODeliveryServiceV13) Delete() (error, tc.ApiErrorType) {
 }
 
 // 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)
+func (ds *TODeliveryServiceV13) IsTenantAuthorized(user *auth.CurrentUser) (bool, error) {
+	return ds.V12().IsTenantAuthorized(user)
 }
 
 func filterAuthorized(dses []tc.DeliveryServiceNullableV13, user *auth.CurrentUser, db *sqlx.DB) ([]tc.DeliveryServiceNullableV13, error) {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index a497946..48406af 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -713,16 +713,20 @@ func dssSelectQuery() string {
 	return selectStmt
 }
 
-type TODSSDeliveryService tc.DSSDeliveryService
-
-var dserviceRef = TODSSDeliveryService(tc.DSSDeliveryService{})
+type TODSSDeliveryService struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.DSSDeliveryService
+}
 
-func GetDServiceRef() *TODSSDeliveryService {
-	return &dserviceRef
+func GetDSSDeliveryServiceReaderSingleton() func(reqInfo *api.APIInfo)api.Reader {
+	return func(reqInfo *api.APIInfo)api.Reader {
+		toReturn := TODSSDeliveryService{reqInfo, tc.DSSDeliveryService{}}
+		return &toReturn
+	}
 }
 
 // Read shows all of the delivery services associated with the specified server.
-func (dss *TODSSDeliveryService) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (dss *TODSSDeliveryService) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var err error = nil
 	orderby := params["orderby"]
 	serverId := params["id"]
@@ -734,7 +738,7 @@ func (dss *TODSSDeliveryService) Read(db *sqlx.DB, params map[string]string, use
 	query := SDSSelectQuery()
 	log.Debugln("Query is ", query)
 
-	rows, err := db.Queryx(query, serverId)
+	rows, err := dss.ReqInfo.Tx.Queryx(query, serverId)
 	if err != nil {
 		log.Errorf("Error querying DeliveryserviceServers: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters.go b/traffic_ops/traffic_ops_golang/parameter/parameters.go
index 87057e5..a948c01 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters.go
@@ -49,57 +49,59 @@ var (
 )
 
 //we need a type alias to define functions on
-type TOParameter tc.ParameterNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOParameter(tc.ParameterNullable{})
+type TOParameter struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.ParameterNullable
+}
 
-func GetRefType() *TOParameter {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOParameter{reqInfo, tc.ParameterNullable{}}
+		return &toReturn
+	}
 }
 
-func (parameter TOParameter) GetKeyFieldsInfo() []api.KeyFieldInfo {
+func (param TOParameter) GetKeyFieldsInfo() []api.KeyFieldInfo {
 	return []api.KeyFieldInfo{{IDQueryParam, api.GetIntKey}}
 }
 
 //Implementation of the Identifier, Validator interface functions
-func (parameter TOParameter) GetKeys() (map[string]interface{}, bool) {
-	if parameter.ID == nil {
+func (param TOParameter) GetKeys() (map[string]interface{}, bool) {
+	if param.ID == nil {
 		return map[string]interface{}{IDQueryParam: 0}, false
 	}
-	return map[string]interface{}{IDQueryParam: *parameter.ID}, true
+	return map[string]interface{}{IDQueryParam: *param.ID}, true
 }
 
-func (parameter *TOParameter) SetKeys(keys map[string]interface{}) {
+func (param *TOParameter) SetKeys(keys map[string]interface{}) {
 	i, _ := keys[IDQueryParam].(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.
-	parameter.ID = &i
+	param.ID = &i
 }
 
-func (parameter *TOParameter) GetAuditName() string {
-	if parameter.Name != nil {
-		return *parameter.Name
+func (param *TOParameter) GetAuditName() string {
+	if param.Name != nil {
+		return *param.Name
 	}
-	if parameter.ID != nil {
-		return strconv.Itoa(*parameter.ID)
+	if param.ID != nil {
+		return strconv.Itoa(*param.ID)
 	}
 	return "unknown"
 }
 
-func (parameter *TOParameter) GetType() string {
-	return "parameter"
+func (param *TOParameter) GetType() string {
+	return "param"
 }
 
 // Validate fulfills the api.Validator interface
-func (parameter TOParameter) Validate(db *sqlx.DB) []error {
-
+func (param TOParameter) Validate() []error {
 	// Test
 	// - Secure Flag is always set to either 1/0
 	// - Admin rights only
 	// - Do not allow duplicate parameters by name+config_file+value
 	errs := validation.Errors{
-		NameQueryParam:       validation.Validate(parameter.Name, validation.Required),
-		ConfigFileQueryParam: validation.Validate(parameter.ConfigFile, validation.Required),
-		ValueQueryParam:      validation.Validate(parameter.Value, validation.Required),
+		NameQueryParam:       validation.Validate(param.Name, validation.Required),
+		ConfigFileQueryParam: validation.Validate(param.ConfigFile, validation.Required),
+		ValueQueryParam:      validation.Validate(param.Value, validation.Required),
 	}
 
 	return tovalidate.ToErrors(errs)
@@ -112,24 +114,8 @@ func (parameter TOParameter) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted parameter and have
 //to be added to the struct
-func (pl *TOParameter) 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(), pl)
+func (param *TOParameter) Create() (error, tc.ApiErrorType) {
+	resultRows, err := param.ReqInfo.Tx.NamedQuery(insertQuery(), param)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -164,14 +150,9 @@ func (pl *TOParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 		return tc.DBError, tc.SystemError
 	}
 
-	pl.SetKeys(map[string]interface{}{IDQueryParam: id})
-	pl.LastUpdated = &lastUpdated
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
+	param.SetKeys(map[string]interface{}{IDQueryParam: id})
+	param.LastUpdated = &lastUpdated
+
 	return nil, tc.NoError
 }
 
@@ -188,10 +169,10 @@ secure) VALUES (
 	return query
 }
 
-func (parameter *TOParameter) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (param *TOParameter) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
-	privLevel := user.PrivLevel
+	privLevel := param.ReqInfo.User.PrivLevel
 
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
@@ -210,7 +191,7 @@ func (parameter *TOParameter) Read(db *sqlx.DB, parameters map[string]string, us
 	query := selectQuery() + where + ParametersGroupBy() + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := param.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Parameters: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -244,25 +225,9 @@ func (parameter *TOParameter) Read(db *sqlx.DB, parameters map[string]string, us
 //ParsePQUniqueConstraintError is used to determine if a parameter 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 (pl *TOParameter) 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
-	}
-	log.Debugf("about to run exec query: %s with parameter: %++v", updateQuery(), pl)
-	resultRows, err := tx.NamedQuery(updateQuery(), pl)
+func (param *TOParameter) Update() (error, tc.ApiErrorType) {
+	log.Debugf("about to run exec query: %s with parameter: %++v", updateQuery(), param)
+	resultRows, err := param.ReqInfo.Tx.NamedQuery(updateQuery(), param)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -287,43 +252,22 @@ func (pl *TOParameter) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 		}
 	}
 	log.Debugf("lastUpdated: %++v", lastUpdated)
-	pl.LastUpdated = &lastUpdated
+	param.LastUpdated = &lastUpdated
 	if rowsAffected != 1 {
 		if rowsAffected < 1 {
 			return errors.New("no parameter found with this id"), tc.DataMissingError
 		}
 		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 Parameter implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (pl *TOParameter) 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
-	}
-	log.Debugf("about to run exec query: %s with parameter: %++v", deleteQuery(), pl)
-	result, err := tx.NamedExec(deleteQuery(), pl)
+func (param *TOParameter) Delete() (error, tc.ApiErrorType) {
+	log.Debugf("about to run exec query: %s with parameter: %++v", deleteQuery(), param)
+	result, err := param.ReqInfo.Tx.NamedExec(deleteQuery(), param)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -339,12 +283,6 @@ func (pl *TOParameter) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 		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/parameter/parameters_test.go b/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
index c638af3..9b465f9 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
@@ -24,14 +24,15 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
 	"encoding/json"
 
 	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 )
 
 func getTestParameters() []tc.ParameterNullable {
@@ -90,10 +91,14 @@ func TestGetParameters(t *testing.T) {
 			ts.Value,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"name": "1"}
+	user := auth.CurrentUser{PrivLevel:30}
 
-	pps, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false),User:&user}
+	pps, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("parameter.Read expected: no errors, actual: %v", errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
index c204c9e..b579a99 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
@@ -28,7 +28,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"
@@ -37,13 +36,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOPhysLocation tc.PhysLocationNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOPhysLocation(tc.PhysLocationNullable{})
+type TOPhysLocation struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.PhysLocationNullable
+}
 
-func GetRefType() *TOPhysLocation {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOPhysLocation{reqInfo, tc.PhysLocationNullable{}}
+		return &toReturn
+	}
 }
 
 func (pl TOPhysLocation) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -77,7 +79,7 @@ func (pl *TOPhysLocation) GetType() string {
 	return "physLocation"
 }
 
-func (pl *TOPhysLocation) Validate(db *sqlx.DB) []error {
+func (pl *TOPhysLocation) Validate() []error {
 	errs := validation.Errors{
 		"address":   validation.Validate(pl.Address, validation.Required),
 		"city":      validation.Validate(pl.City, validation.Required),
@@ -93,7 +95,7 @@ func (pl *TOPhysLocation) Validate(db *sqlx.DB) []error {
 	return nil
 }
 
-func (pl *TOPhysLocation) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (pl *TOPhysLocation) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -111,7 +113,7 @@ func (pl *TOPhysLocation) Read(db *sqlx.DB, parameters map[string]string, user a
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := pl.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying PhysLocations: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -160,25 +162,9 @@ JOIN region r ON pl.region = r.id`
 //ParsePQUniqueConstraintError is used to determine if a phys_location 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 (pl *TOPhysLocation) 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 (pl *TOPhysLocation) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with phys_location: %++v", updateQuery(), pl)
-	resultRows, err := tx.NamedQuery(updateQuery(), pl)
+	resultRows, err := pl.ReqInfo.Tx.NamedQuery(updateQuery(), pl)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -210,12 +196,6 @@ func (pl *TOPhysLocation) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.
 		}
 		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
 }
 
@@ -226,24 +206,8 @@ func (pl *TOPhysLocation) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted phys_location and have
 //to be added to the struct
-func (pl *TOPhysLocation) 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(), pl)
+func (pl *TOPhysLocation) Create() (error, tc.ApiErrorType) {
+	resultRows, err := pl.ReqInfo.Tx.NamedQuery(insertQuery(), pl)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -280,36 +244,16 @@ func (pl *TOPhysLocation) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.
 
 	pl.SetKeys(map[string]interface{}{"id": id})
 	pl.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
 }
 
 //The PhysLocation implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (pl *TOPhysLocation) 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()))
-		}
-	}()
+func (pl *TOPhysLocation) Delete() (error, tc.ApiErrorType) {
 
-	if err != nil {
-		log.Error.Printf("could not begin transaction: %v", err)
-		return tc.DBError, tc.SystemError
-	}
 	log.Debugf("about to run exec query: %s with phys_location: %++v", deleteQuery(), pl)
-	result, err := tx.NamedExec(deleteQuery(), pl)
+	result, err := pl.ReqInfo.Tx.NamedExec(deleteQuery(), pl)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -325,12 +269,6 @@ func (pl *TOPhysLocation) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.
 		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/physlocation/phys_locations_test.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
index f9161f7..3cad4ec 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
@@ -26,8 +26,8 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -92,10 +92,14 @@ func TestGetPhysLocations(t *testing.T) {
 			ts.Zip,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	physLocations, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+	physLocations, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("physLocation.Read expected: no errors, actual: %v", errs)
 	}
@@ -129,7 +133,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOPhysLocation{}
-	errs := test.SortErrors(p.Validate(nil))
+	errs := test.SortErrors(p.Validate())
 	expected := test.SortErrors([]error{
 		errors.New("'state' cannot be blank"),
 		errors.New("'zip' cannot be blank"),
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles.go b/traffic_ops/traffic_ops_golang/profile/profiles.go
index b7c6862..6b39006 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles.go
@@ -47,14 +47,16 @@ const (
 )
 
 //we need a type alias to define functions on
-type TOProfile v13.ProfileNullable
-type TOParameter v13.ParameterNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOProfile{}
+type TOProfile struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.ProfileNullable
+}
 
-func GetRefType() *TOProfile {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOProfile{reqInfo, v13.ProfileNullable{}}
+		return &toReturn
+	}
 }
 
 func (prof TOProfile) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -88,7 +90,7 @@ func (prof *TOProfile) GetType() string {
 	return "profile"
 }
 
-func (prof *TOProfile) Validate(db *sqlx.DB) []error {
+func (prof *TOProfile) Validate() []error {
 	errs := validation.Errors{
 		NameQueryParam:        validation.Validate(prof.Name, validation.Required),
 		DescriptionQueryParam: validation.Validate(prof.Description, validation.Required),
@@ -101,7 +103,7 @@ func (prof *TOProfile) Validate(db *sqlx.DB) []error {
 	return nil
 }
 
-func (prof *TOProfile) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (prof *TOProfile) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -118,7 +120,7 @@ func (prof *TOProfile) Read(db *sqlx.DB, parameters map[string]string, user auth
 	query := selectProfilesQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := prof.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Profile: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -135,7 +137,7 @@ func (prof *TOProfile) Read(db *sqlx.DB, parameters map[string]string, user auth
 
 		// Attach Parameters if the 'id' parameter is sent
 		if _, ok := parameters[IDQueryParam]; ok {
-			params, err := ReadParameters(db, parameters, user, p)
+			params, err := ReadParameters(prof.ReqInfo.Tx, parameters, prof.ReqInfo.User, p)
 			p.Parameters = params
 			if len(errs) > 0 {
 				log.Errorf("Error getting Parameters: %v", err)
@@ -166,7 +168,7 @@ LEFT JOIN cdn c ON prof.cdn = c.id`
 	return query
 }
 
-func ReadParameters(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser, profile v13.ProfileNullable) ([]v13.ParameterNullable, []error) {
+func ReadParameters(tx *sqlx.Tx, parameters map[string]string, user *auth.CurrentUser, profile v13.ProfileNullable) ([]v13.ParameterNullable, []error) {
 
 	var rows *sqlx.Rows
 	privLevel := user.PrivLevel
@@ -174,7 +176,7 @@ func ReadParameters(db *sqlx.DB, parameters map[string]string, user auth.Current
 	queryValues["profile_id"] = *profile.ID
 
 	query := selectParametersQuery()
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Parameter: %v", err)
 		return nil, []error{tc.DBError}
@@ -221,25 +223,9 @@ WHERE pp.profile = :profile_id`
 //ParsePQUniqueConstraintError is used to determine if a profile 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 (prof *TOProfile) 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 (prof *TOProfile) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with profile: %++v", updateQuery(), prof)
-	resultRows, err := tx.NamedQuery(updateQuery(), prof)
+	resultRows, err := prof.ReqInfo.Tx.NamedQuery(updateQuery(), prof)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -270,12 +256,7 @@ func (prof *TOProfile) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 		}
 		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
 }
 
@@ -286,24 +267,8 @@ func (prof *TOProfile) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted profile and have
 //to be added to the struct
-func (prof *TOProfile) 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(), prof)
+func (prof *TOProfile) Create() (error, tc.ApiErrorType) {
+	resultRows, err := prof.ReqInfo.Tx.NamedQuery(insertQuery(), prof)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -340,36 +305,15 @@ func (prof *TOProfile) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 
 	prof.SetKeys(map[string]interface{}{IDQueryParam: id})
 	prof.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
 }
 
 //The Profile implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (prof *TOProfile) 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 (prof *TOProfile) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with profile: %++v", deleteQuery(), prof)
-	result, err := tx.NamedExec(deleteQuery(), prof)
+	result, err := prof.ReqInfo.Tx.NamedExec(deleteQuery(), prof)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -385,12 +329,6 @@ func (prof *TOProfile) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Api
 		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/profile/profiles_test.go b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
index bf2fe08..b00ea6c 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles_test.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
@@ -26,8 +26,8 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -93,10 +93,14 @@ func TestGetProfiles(t *testing.T) {
 			ts.Type,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"name": "1"}
 
-	profiles, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+	profiles, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("profile.Read expected: no errors, actual: %v", errs)
 	}
@@ -130,7 +134,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOProfile{}
-	errs := test.SortErrors(p.Validate(nil))
+	errs := test.SortErrors(p.Validate())
 	expected := test.SortErrors([]error{
 		errors.New("'cdn' cannot be blank"),
 		errors.New("'description' cannot be blank"),
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
index 9920228..703d40c 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
@@ -29,7 +29,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"
@@ -43,13 +42,16 @@ const (
 )
 
 //we need a type alias to define functions on
-type TOProfileParameter v13.ProfileParameterNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOProfileParameter(v13.ProfileParameterNullable{})
+type TOProfileParameter struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.ProfileParameterNullable
+}
 
-func GetRefType() *TOProfileParameter {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOProfileParameter{reqInfo, v13.ProfileParameterNullable{}}
+		return &toReturn
+	}
 }
 
 func (pp TOProfileParameter) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -93,7 +95,7 @@ func (pp *TOProfileParameter) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (pp *TOProfileParameter) Validate(db *sqlx.DB) []error {
+func (pp *TOProfileParameter) Validate() []error {
 
 	errs := validation.Errors{
 		"profile":   validation.Validate(pp.ProfileID, validation.Required),
@@ -110,24 +112,8 @@ func (pp *TOProfileParameter) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the profile and lastUpdated values of the newly inserted profileparameter and have
 //to be added to the struct
-func (pp *TOProfileParameter) 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(), pp)
+func (pp *TOProfileParameter) Create() (error, tc.ApiErrorType) {
+	resultRows, err := pp.ReqInfo.Tx.NamedQuery(insertQuery(), pp)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -164,13 +150,6 @@ func (pp *TOProfileParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error,
 	}
 
 	pp.SetKeys(map[string]interface{}{ProfileIDQueryParam: profile, ParameterIDQueryParam: parameter})
-	pp.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
 }
 
@@ -183,7 +162,11 @@ parameter) VALUES (
 	return query
 }
 
-func (pp *TOProfileParameter) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (pp *TOProfileParameter) Update() (error, tc.ApiErrorType) {
+	return errors.New("unimplemented"), tc.SystemError
+}
+
+func (pp *TOProfileParameter) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -202,7 +185,7 @@ func (pp *TOProfileParameter) Read(db *sqlx.DB, parameters map[string]string, us
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := pp.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Parameters: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -225,25 +208,9 @@ func (pp *TOProfileParameter) Read(db *sqlx.DB, parameters map[string]string, us
 
 //The Parameter implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (pp *TOProfileParameter) 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 (pp *TOProfileParameter) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with parameter: %++v", deleteQuery(), pp)
-	result, err := tx.NamedExec(deleteQuery(), pp)
+	result, err := pp.ReqInfo.Tx.NamedExec(deleteQuery(), pp)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -259,12 +226,6 @@ func (pp *TOProfileParameter) Delete(db *sqlx.DB, user auth.CurrentUser) (error,
 		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/profileparameter/profile_parameters_test.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
index b11f64b..013ce3a 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
@@ -25,8 +25,8 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -78,10 +78,15 @@ func TestGetProfileParameters(t *testing.T) {
 			ts.ParameterID,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"profile": "1"}
 
-	pps, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+
+	pps, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("profileparameter.Read expected: no errors, actual: %v", errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/region/regions.go b/traffic_ops/traffic_ops_golang/region/regions.go
index 730870a..b54598c 100644
--- a/traffic_ops/traffic_ops_golang/region/regions.go
+++ b/traffic_ops/traffic_ops_golang/region/regions.go
@@ -26,20 +26,22 @@ 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/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
 //we need a type alias to define functions on
-type TORegion tc.Region
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TORegion(tc.Region{})
+type TORegion struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.Region
+}
 
-func GetRefType() *TORegion {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TORegion{reqInfo, tc.Region{}}
+		return &toReturn
+	}
 }
 
 func (region TORegion) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -64,7 +66,7 @@ func (region *TORegion) GetType() string {
 	return "region"
 }
 
-func (region *TORegion) Validate(db *sqlx.DB) []error {
+func (region *TORegion) Validate() []error {
 	errs := []error{}
 	if len(region.Name) < 1 {
 		errs = append(errs, errors.New(`Region 'name' is required.`))
@@ -72,7 +74,7 @@ func (region *TORegion) Validate(db *sqlx.DB) []error {
 	return errs
 }
 
-func (region *TORegion) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (region *TORegion) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -90,7 +92,7 @@ func (region *TORegion) Read(db *sqlx.DB, parameters map[string]string, user aut
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := region.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Regions: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -128,25 +130,9 @@ JOIN division d ON r.division = d.id`
 //ParsePQUniqueConstraintError is used to determine if a region 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 (region *TORegion) 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 (region *TORegion) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with region: %++v", updateQuery(), region)
-	resultRows, err := tx.NamedQuery(updateQuery(), region)
+	resultRows, err := region.ReqInfo.Tx.NamedQuery(updateQuery(), region)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -177,12 +163,7 @@ func (region *TORegion) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 		}
 		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
 }
 
@@ -193,24 +174,8 @@ func (region *TORegion) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted region and have
 //to be added to the struct
-func (region *TORegion) 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(), region)
+func (region *TORegion) Create() (error, tc.ApiErrorType) {
+	resultRows, err := region.ReqInfo.Tx.NamedQuery(insertQuery(), region)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -246,36 +211,15 @@ func (region *TORegion) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 	region.SetKeys(map[string]interface{}{"id": id})
 	region.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
 }
 
 //The Region implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (region *TORegion) 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 (region *TORegion) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with region: %++v", deleteQuery(), region)
-	result, err := tx.NamedExec(deleteQuery(), region)
+	result, err := region.ReqInfo.Tx.NamedExec(deleteQuery(), region)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -291,12 +235,6 @@ func (region *TORegion) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 		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/region/regions_test.go b/traffic_ops/traffic_ops_golang/region/regions_test.go
index 854f838..ad7cb6c 100644
--- a/traffic_ops/traffic_ops_golang/region/regions_test.go
+++ b/traffic_ops/traffic_ops_golang/region/regions_test.go
@@ -24,8 +24,8 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -59,8 +59,6 @@ func TestReadRegions(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-	refType := GetRefType()
-
 	testRegions := getTestRegions()
 	cols := test.ColsFromStructByTag("db", tc.Region{})
 	rows := sqlmock.NewRows(cols)
@@ -73,10 +71,14 @@ func TestReadRegions(t *testing.T) {
 			ts.Name,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	regions, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+	regions, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("region.Read expected: no errors, actual: %v", errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/role/roles.go b/traffic_ops/traffic_ops_golang/role/roles.go
index 3871c8b..2cc8a37 100644
--- a/traffic_ops/traffic_ops_golang/role/roles.go
+++ b/traffic_ops/traffic_ops_golang/role/roles.go
@@ -29,7 +29,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,13 +37,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TORole v13.Role
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TORole{}
+type TORole struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.Role
+}
 
-func GetRefType() *TORole {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TORole{reqInfo, v13.Role{}}
+		return &toReturn
+	}
 }
 
 func (role TORole) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -79,7 +81,7 @@ func (role *TORole) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (role TORole) Validate(db *sqlx.DB) []error {
+func (role TORole) Validate() []error {
 	errs := validation.Errors{
 		"name":        validation.Validate(role.Name, validation.Required),
 		"description": validation.Validate(role.Description, validation.Required),
@@ -88,8 +90,8 @@ func (role TORole) Validate(db *sqlx.DB) []error {
 	errsToReturn := tovalidate.ToErrors(errs)
 	checkCaps := `SELECT cap FROM UNNEST($1::text[]) AS cap WHERE NOT cap =  ANY(ARRAY(SELECT c.name FROM capability AS c WHERE c.name = ANY($1)))`
 	var badCaps []string
-	if db != nil {
-		err := db.Select(&badCaps, checkCaps, pq.Array(role.Capabilities))
+	if role.ReqInfo.Tx != nil {
+		err := role.ReqInfo.Tx.Select(&badCaps, checkCaps, pq.Array(role.Capabilities))
 		if err != nil {
 			log.Errorf("got error from selecting bad capabilities: %v", err)
 			return []error{tc.DBError}
@@ -108,27 +110,11 @@ func (role TORole) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted role and have
 //to be added to the struct
-func (role *TORole) 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
-	}
-	if *role.PrivLevel > user.PrivLevel {
+func (role *TORole) Create() (error, tc.ApiErrorType) {
+	if *role.PrivLevel > role.ReqInfo.User.PrivLevel {
 		return errors.New("can not create a role with a higher priv level than your own"), tc.ForbiddenError
 	}
-	resultRows, err := tx.NamedQuery(insertQuery(), role)
+	resultRows, err := role.ReqInfo.Tx.NamedQuery(insertQuery(), role)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -163,17 +149,11 @@ func (role *TORole) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErr
 	}
 	role.SetKeys(map[string]interface{}{"id": id})
 	//after we have role ID we can associate the capabilities:
-	err, errType := role.createRoleCapabilityAssociations(tx)
+	err, errType := role.createRoleCapabilityAssociations(role.ReqInfo.Tx)
 	if err != nil {
 		return err, errType
 	}
 
-	err = tx.Commit()
-	if err != nil {
-		log.Errorln("Could not commit transaction: ", err)
-		return tc.DBError, tc.SystemError
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
@@ -209,7 +189,7 @@ func (role *TORole) deleteRoleCapabilityAssociations(tx *sqlx.Tx) (error, tc.Api
 	return nil, tc.NoError
 }
 
-func (role *TORole) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (role *TORole) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -226,7 +206,7 @@ func (role *TORole) Read(db *sqlx.DB, parameters map[string]string, user auth.Cu
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := role.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Roles: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -253,30 +233,13 @@ func (role *TORole) Read(db *sqlx.DB, parameters map[string]string, user auth.Cu
 //ParsePQUniqueConstraintError is used to determine if a role 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 (role *TORole) 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
-	}
-
-	if *role.PrivLevel > user.PrivLevel {
+func (role *TORole) Update() (error, tc.ApiErrorType) {
+	if *role.PrivLevel > role.ReqInfo.User.PrivLevel {
 		return errors.New("can not create a role with a higher priv level than your own"), tc.ForbiddenError
 	}
 
 	log.Debugf("about to run exec query: %s with role: %++v\n", updateQuery(), role)
-	result, err := tx.NamedExec(updateQuery(), role)
+	result, err := role.ReqInfo.Tx.NamedExec(updateQuery(), role)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -303,46 +266,24 @@ func (role *TORole) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErr
 		}
 	}
 	//remove associations
-	err, errType := role.deleteRoleCapabilityAssociations(tx)
+	err, errType := role.deleteRoleCapabilityAssociations(role.ReqInfo.Tx)
 	if err != nil {
 		return err, errType
 	}
 	//create new associations
-	err, errType = role.createRoleCapabilityAssociations(tx)
+	err, errType = role.createRoleCapabilityAssociations(role.ReqInfo.Tx)
 	if err != nil {
 		return err, errType
 	}
 
-	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 Role implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (role *TORole) 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 (role *TORole) Delete() (error, tc.ApiErrorType) {
 	assignedUsers := 0
-	err = tx.Get(&assignedUsers, "SELECT COUNT(id) FROM tm_user WHERE role=$1", role.ID)
+	err := role.ReqInfo.Tx.Get(&assignedUsers, "SELECT COUNT(id) FROM tm_user WHERE role=$1", role.ID)
 	if err != nil {
 		log.Errorf("received error: %++v from assigned users check", err)
 		return tc.DBError, tc.SystemError
@@ -352,7 +293,7 @@ func (role *TORole) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErr
 	}
 
 	log.Debugf("about to run exec query: %s with role: %++v", deleteQuery(), role)
-	result, err := tx.NamedExec(deleteQuery(), role)
+	result, err := role.ReqInfo.Tx.NamedExec(deleteQuery(), role)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -369,17 +310,11 @@ func (role *TORole) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErr
 		}
 	}
 	//remove associations
-	err, errType := role.deleteRoleCapabilityAssociations(tx)
+	err, errType := role.deleteRoleCapabilityAssociations(role.ReqInfo.Tx)
 	if err != nil {
 		return err, errType
 	}
 
-	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/role/roles_test.go b/traffic_ops/traffic_ops_golang/role/roles_test.go
index 0c49961..7807371 100644
--- a/traffic_ops/traffic_ops_golang/role/roles_test.go
+++ b/traffic_ops/traffic_ops_golang/role/roles_test.go
@@ -96,8 +96,9 @@ func TestInterfaces(t *testing.T) {
 func TestValidate(t *testing.T) {
 	// invalid name, empty domainname
 	n := "not_a_valid_role"
-	r := TORole{Name: &n}
-	errs := test.SortErrors(r.Validate(nil))
+	reqInfo := api.APIInfo{}
+	r := TORole{ReqInfo: &reqInfo, Role: v13.Role{Name: &n}}
+	errs := test.SortErrors(r.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'description' cannot be blank`),
@@ -109,9 +110,9 @@ func TestValidate(t *testing.T) {
 	}
 
 	//  name,  domainname both valid
-	r = TORole{Name: stringAddr("this is a valid name"), Description: stringAddr("this is a description"), PrivLevel: intAddr(30)}
+	r = TORole{ReqInfo: &reqInfo, Role: v13.Role{Name: stringAddr("this is a valid name"), Description: stringAddr("this is a description"), PrivLevel: intAddr(30)}}
 	expectedErrs = []error{}
-	errs = r.Validate(nil)
+	errs = r.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index ed93d74..00edc23 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -46,16 +46,16 @@ import (
 	"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/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"
@@ -139,37 +139,37 @@ 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `parameters/{id}$`, api.ReadHandler(parameter.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `parameters/{id}$`, api.UpdateHandler(parameter.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `parameters/?$`, api.CreateHandler(parameter.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `parameters/{id}$`, api.DeleteHandler(parameter.GetTypeSingleton()), 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.GetTypeSingleton()), 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `phys_locations/{id}$`, api.UpdateHandler(physlocation.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `phys_locations/?$`, api.CreateHandler(physlocation.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `phys_locations/{id}$`, api.DeleteHandler(physlocation.GetTypeSingleton()), 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},
-
-		//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, `profiles/?(\.json)?$`, api.ReadHandler(profile.GetTypeSingleton()), 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `profiles/{id}$`, api.UpdateHandler(profile.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profiles/?$`, api.CreateHandler(profile.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `profiles/{id}$`, api.DeleteHandler(profile.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+
+		//Region: CRUDs
+		{1.1, http.MethodGet, `regions/?(\.json)?$`, api.ReadHandler(region.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `regions/{id}$`, api.ReadHandler(region.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `regions/{id}$`, api.UpdateHandler(region.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `regions/?$`, api.CreateHandler(region.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `regions/{id}$`, api.DeleteHandler(region.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		{1.1, http.MethodDelete, `deliveryservice_server/{dsid}/{serverid}`, dsserver.Delete, auth.PrivLevelReadOnly, Authenticated, nil},
 
@@ -178,10 +178,10 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{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, `servers/{id}/deliveryservices$`, api.ReadOnlyHandler(dsserver.GetDSSDeliveryServiceReaderSingleton()),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},
+		//{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{}},
@@ -193,28 +193,28 @@ 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `servers/{id}$`, api.ReadHandler(server.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `servers/{id}$`, api.UpdateHandler(server.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `servers/?$`, api.CreateHandler(server.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `servers/{id}$`, api.DeleteHandler(server.GetTypeSingleton()), 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `statuses/{id}$`, api.ReadHandler(status.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `statuses/{id}$`, api.UpdateHandler(status.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `statuses/?$`, api.CreateHandler(status.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `statuses/{id}$`, api.DeleteHandler(status.GetTypeSingleton()), 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `types/{id}$`, api.ReadHandler(types.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `types/{id}$`, api.UpdateHandler(types.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `types/?$`, api.CreateHandler(types.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `types/{id}$`, api.DeleteHandler(types.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//About
 		{1.3, http.MethodGet, `about/?(\.json)?$`, about.Handler(), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -280,10 +280,10 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.3, http.MethodDelete, `origins/?$`, api.DeleteHandler(origin.GetTypeSingleton()), 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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `roles/?$`, api.UpdateHandler(role.GetTypeSingleton()), auth.PrivLevelAdmin, Authenticated, nil},
+		{1.3, http.MethodPost, `roles/?$`, api.CreateHandler(role.GetTypeSingleton()), auth.PrivLevelAdmin, Authenticated, nil},
+		{1.3, http.MethodDelete, `roles/?$`, api.DeleteHandler(role.GetTypeSingleton()), auth.PrivLevelAdmin, Authenticated, nil},
 
 		//Delivery Services Regexes
 		{1.1, http.MethodGet, `deliveryservices_regexes/?(\.json)?$`, deliveryservicesregexes.Get(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -297,20 +297,20 @@ 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.ReadOnlyHandler(staticdnsentry.GetReaderSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//ProfileParameters
 		{1.1, http.MethodGet, `profiles/{id}/parameters/?(\.json)?$`, profileparameter.GetProfileID(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `profiles/{id}/unassigned_parameters/?(\.json)?$`, profileparameter.GetUnassigned(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `profiles/name/{name}/parameters/?(\.json)?$`, profileparameter.GetProfileName(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `parameters/profile/{name}/?(\.json)?$`, profileparameter.GetProfileName(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPost, `profiles/name/{name}/parameters/?$`, profileparameter.PostProfileParamsByName, auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `profiles/{id}/parameters/?$`, profileparameter.PostProfileParamsByID, auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodGet, `profileparameters/?(\.json)?$`, api.ReadHandler(profileparameter.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPost, `profileparameters/?$`, api.CreateHandler(profileparameter.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `profileparameter/?$`, profileparameter.PostProfileParam, auth.PrivLevelOperations, Authenticated, nil},
-		{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},
+		{1.1, http.MethodPost, `profiles/name/{name}/parameters/?$`, profileparameter.PostProfileParamsByName(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profiles/{id}/parameters/?$`, profileparameter.PostProfileParamsByID(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodGet, `profileparameters/?(\.json)?$`, api.ReadHandler(profileparameter.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPost, `profileparameters/?$`, api.CreateHandler(profileparameter.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profileparameter/?$`, profileparameter.PostProfileParam(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `parameterprofile/?$`, profileparameter.PostParamProfile(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `profileparameters/{profileId}/{parameterId}$`, api.DeleteHandler(profileparameter.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 
 		//Tenants
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index cd82dc8..1ad8eea 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -39,13 +39,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOServer v13.ServerNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOServer{}
+type TOServer struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.ServerNullable
+}
 
-func GetRefType() *TOServer {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOServer{reqInfo, v13.ServerNullable{}}
+		return &toReturn
+	}
 }
 
 func (server TOServer) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -79,7 +82,7 @@ func (server *TOServer) GetType() string {
 	return "server"
 }
 
-func (server *TOServer) Validate(db *sqlx.DB) []error {
+func (server *TOServer) Validate() []error {
 
 	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
 
@@ -104,7 +107,7 @@ func (server *TOServer) Validate(db *sqlx.DB) []error {
 		return errs
 	}
 
-	rows, err := db.Query("select use_in_table from type where id=$1", server.TypeID)
+	rows, err := server.ReqInfo.Tx.Query("select use_in_table from type where id=$1", server.TypeID)
 	if err != nil {
 		log.Error.Printf("could not execute select use_in_table from type: %s\n", err)
 		errs = append(errs, tc.DBError)
@@ -123,7 +126,7 @@ func (server *TOServer) Validate(db *sqlx.DB) []error {
 		errs = append(errs, errors.New("invalid server type"))
 	}
 
-	rows, err = db.Query("select cdn from profile where id=$1", server.ProfileID)
+	rows, err = server.ReqInfo.Tx.Query("select cdn from profile where id=$1", server.ProfileID)
 	if err != nil {
 		log.Error.Printf("could not execute select cdnID from profile: %s\n", err)
 		errs = append(errs, tc.DBError)
@@ -173,12 +176,12 @@ func (server TOServer) ChangeLogMessage(action string) (string, error) {
 	return message, nil
 }
 
-func (server *TOServer) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (server *TOServer) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
 
-	privLevel := user.PrivLevel
+	privLevel := server.ReqInfo.User.PrivLevel
 
-	servers, errs, errType := getServers(params, db, privLevel)
+	servers, errs, errType := getServers(params, server.ReqInfo.Tx, privLevel)
 	if len(errs) > 0 {
 		for _, err := range errs {
 			if err.Error() == `id cannot parse to integer` {
@@ -195,7 +198,7 @@ func (server *TOServer) Read(db *sqlx.DB, params map[string]string, user auth.Cu
 	return returnable, nil, tc.NoError
 }
 
-func getServers(params map[string]string, db *sqlx.DB, privLevel int) ([]tc.ServerNullable, []error, tc.ApiErrorType) {
+func getServers(params map[string]string, tx *sqlx.Tx, privLevel int) ([]tc.ServerNullable, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 	var err error
 
@@ -221,7 +224,7 @@ func getServers(params map[string]string, db *sqlx.DB, privLevel int) ([]tc.Serv
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err = db.NamedQuery(query, queryValues)
+	rows, err = tx.NamedQuery(query, queryValues)
 	if err != nil {
 		return nil, []error{fmt.Errorf("querying: %v", err)}, tc.SystemError
 	}
@@ -313,26 +316,9 @@ JOIN type t ON s.type = t.id`
 //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 (server *TOServer) 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 (server *TOServer) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with server: %++v", updateQuery(), server)
-	resultRows, err := tx.NamedQuery(updateQuery(), server)
+	resultRows, err := server.ReqInfo.Tx.NamedQuery(updateQuery(), server)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -369,12 +355,7 @@ func (server *TOServer) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 	server.SetKeys(map[string]interface{}{"id": id})
 	server.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
 }
 
@@ -424,30 +405,13 @@ WHERE id=:id RETURNING last_updated`
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted server and have
 //to be added to the struct
-func (server *TOServer) 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 (server *TOServer) Create() (error, tc.ApiErrorType) {
 	if server.XMPPID == nil || *server.XMPPID == "" {
 		hostName := *server.HostName
 		server.XMPPID = &hostName
 	}
 
-	resultRows, err := tx.NamedQuery(insertQuery(), server)
+	resultRows, err := server.ReqInfo.Tx.NamedQuery(insertQuery(), server)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -483,12 +447,7 @@ func (server *TOServer) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 	server.SetKeys(map[string]interface{}{"id": id})
 	server.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
 }
 
@@ -565,25 +524,9 @@ xmpp_passwd
 
 //The Server implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (server *TOServer) 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 (server *TOServer) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with server: %++v", deleteServerQuery(), server)
-	result, err := tx.NamedExec(deleteServerQuery(), server)
+	result, err := server.ReqInfo.Tx.NamedExec(deleteServerQuery(), server)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -599,12 +542,7 @@ func (server *TOServer) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 			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/server/servers_test.go b/traffic_ops/traffic_ops_golang/server/servers_test.go
index bb4f008..43d4e05 100644
--- a/traffic_ops/traffic_ops_golang/server/servers_test.go
+++ b/traffic_ops/traffic_ops_golang/server/servers_test.go
@@ -157,10 +157,11 @@ func TestGetServersByCachegroup(t *testing.T) {
 			ts.XMPPPasswd,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
 	v := map[string]string{"cachegroup": "2"}
 
-	servers, errs, errType := getServers(v, db, auth.PrivLevelAdmin)
+	servers, errs, errType := getServers(v, db.MustBegin(), auth.PrivLevelAdmin)
 	log.Debugln("%v-->", servers)
 	if len(errs) > 0 {
 		t.Errorf("getServers expected: no errors, actual: %v with error type: %s", errs, errType.String())
diff --git a/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go b/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
index ac9f3b1..2c06d35 100644
--- a/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
+++ b/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
@@ -22,16 +22,23 @@ package staticdnsentry
 import (
 	"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/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
-	"github.com/jmoiron/sqlx"
 )
 
-type TOStaticDNSEntry tc.StaticDNSEntry
+type TOStaticDNSEntry struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.StaticDNSEntry
+}
 
-func GetRefType() *TOStaticDNSEntry { return &TOStaticDNSEntry{} }
+func GetReaderSingleton() func(reqInfo *api.APIInfo)api.Reader {
+	return func(reqInfo *api.APIInfo)api.Reader {
+		toReturn := TOStaticDNSEntry{reqInfo, tc.StaticDNSEntry{}}
+		return &toReturn
+	}
+}
 
-func (staticDNSEntry *TOStaticDNSEntry) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (staticDNSEntry *TOStaticDNSEntry) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
 		"deliveryservice": dbhelpers.WhereColumnInfo{"deliveryservice", nil}, // order by
 	}
@@ -42,7 +49,7 @@ func (staticDNSEntry *TOStaticDNSEntry) Read(db *sqlx.DB, parameters map[string]
 	}
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := staticDNSEntry.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying StaticDNSEntries: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
diff --git a/traffic_ops/traffic_ops_golang/status/statuses.go b/traffic_ops/traffic_ops_golang/status/statuses.go
index a9fa43b..d00aed9 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses.go
@@ -28,7 +28,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"
@@ -37,13 +36,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOStatus tc.StatusNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOStatus(tc.StatusNullable{})
+type TOStatus struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.StatusNullable
+}
 
-func GetRefType() *TOStatus {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOStatus{reqInfo, tc.StatusNullable{}}
+		return &toReturn
+	}
 }
 
 func (status TOStatus) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -77,14 +79,14 @@ func (status TOStatus) GetType() string {
 	return "status"
 }
 
-func (status TOStatus) Validate(db *sqlx.DB) []error {
+func (status TOStatus) Validate() []error {
 	errs := validation.Errors{
 		"name": validation.Validate(status.Name, validation.NotNil, validation.Required),
 	}
 	return tovalidate.ToErrors(errs)
 }
 
-func (status *TOStatus) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (status *TOStatus) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -102,7 +104,7 @@ func (status *TOStatus) Read(db *sqlx.DB, parameters map[string]string, user aut
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := status.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Status: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -139,25 +141,9 @@ FROM status s`
 //ParsePQUniqueConstraintError is used to determine if a status 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 (status *TOStatus) 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 (status *TOStatus) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with status: %++v", updateQuery(), status)
-	resultRows, err := tx.NamedQuery(updateQuery(), status)
+	resultRows, err := status.ReqInfo.Tx.NamedQuery(updateQuery(), status)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -190,12 +176,7 @@ func (status *TOStatus) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 			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
 }
 
@@ -206,24 +187,8 @@ func (status *TOStatus) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted status and have
 //to be added to the struct
-func (status *TOStatus) 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(), status)
+func (status *TOStatus) Create() (error, tc.ApiErrorType) {
+	resultRows, err := status.ReqInfo.Tx.NamedQuery(insertQuery(), status)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -259,36 +224,15 @@ func (status *TOStatus) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 	status.SetKeys(map[string]interface{}{"id": id})
 	status.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
 }
 
 //The Status implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (status *TOStatus) 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 (status *TOStatus) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with status: %++v", deleteQuery(), status)
-	result, err := tx.NamedExec(deleteQuery(), status)
+	result, err := status.ReqInfo.Tx.NamedExec(deleteQuery(), status)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -304,12 +248,7 @@ func (status *TOStatus) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 			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/status/statuses_test.go b/traffic_ops/traffic_ops_golang/status/statuses_test.go
index 3127450..7a684f1 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses_test.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses_test.go
@@ -24,8 +24,8 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -60,8 +60,6 @@ func TestReadStatuses(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-	refType := GetRefType()
-
 	testStatuses := getTestStatuses()
 	cols := test.ColsFromStructByTag("db", tc.Status{})
 	rows := sqlmock.NewRows(cols)
@@ -74,10 +72,14 @@ func TestReadStatuses(t *testing.T) {
 			ts.Name,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	statuses, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+	statuses, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("status.Read expected: no errors, actual: %v", errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/types/types.go b/traffic_ops/traffic_ops_golang/types/types.go
index 7baaf52..f160f8b 100644
--- a/traffic_ops/traffic_ops_golang/types/types.go
+++ b/traffic_ops/traffic_ops_golang/types/types.go
@@ -28,7 +28,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"
@@ -37,13 +36,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOType tc.TypeNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOType(tc.TypeNullable{})
+type TOType struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.TypeNullable
+}
 
-func GetRefType() *TOType {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOType{reqInfo, tc.TypeNullable{}}
+		return &toReturn
+	}
 }
 
 func (typ TOType) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -77,7 +79,7 @@ func (typ *TOType) GetType() string {
 	return "type"
 }
 
-func (typ *TOType) Validate(db *sqlx.DB) []error {
+func (typ *TOType) Validate() []error {
 	errs := validation.Errors{
 		"name":         validation.Validate(typ.Name, validation.Required),
 		"description":  validation.Validate(typ.Description, validation.Required),
@@ -89,7 +91,7 @@ func (typ *TOType) Validate(db *sqlx.DB) []error {
 	return nil
 }
 
-func (typ *TOType) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (typ *TOType) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -107,7 +109,7 @@ func (typ *TOType) Read(db *sqlx.DB, parameters map[string]string, user auth.Cur
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := typ.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Types: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -145,25 +147,9 @@ FROM type typ`
 //ParsePQUniqueConstraintError is used to determine if a type 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 (typ *TOType) 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 (typ *TOType) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with type: %++v", updateQuery(), typ)
-	resultRows, err := tx.NamedQuery(updateQuery(), typ)
+	resultRows, err := typ.ReqInfo.Tx.NamedQuery(updateQuery(), typ)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -194,12 +180,7 @@ func (typ *TOType) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErro
 		}
 		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
 }
 
@@ -210,24 +191,8 @@ func (typ *TOType) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErro
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted type and have
 //to be added to the struct
-func (typ *TOType) 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(), typ)
+func (typ *TOType) Create() (error, tc.ApiErrorType) {
+	resultRows, err := typ.ReqInfo.Tx.NamedQuery(insertQuery(), typ)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -264,36 +229,15 @@ func (typ *TOType) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErro
 
 	typ.SetKeys(map[string]interface{}{"id": id})
 	typ.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
 }
 
 //The Type implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (typ *TOType) 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 (typ *TOType) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with type: %++v", deleteQuery(), typ)
-	result, err := tx.NamedExec(deleteQuery(), typ)
+	result, err := typ.ReqInfo.Tx.NamedExec(deleteQuery(), typ)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -309,12 +253,6 @@ func (typ *TOType) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErro
 		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/types/types_test.go b/traffic_ops/traffic_ops_golang/types/types_test.go
index f69d5fe..252bed2 100644
--- a/traffic_ops/traffic_ops_golang/types/types_test.go
+++ b/traffic_ops/traffic_ops_golang/types/types_test.go
@@ -26,8 +26,8 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -81,10 +81,15 @@ func TestGetType(t *testing.T) {
 			ts.UseInTable,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	types, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+
+
+	types, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("type.Read expected: no errors, actual: %v", errs)
 	}
@@ -118,7 +123,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOType{}
-	errs := test.SortErrors(p.Validate(nil))
+	errs := test.SortErrors(p.Validate())
 	expected := test.SortErrors([]error{
 		errors.New("'name' cannot be blank"),
 		errors.New("'description' cannot be blank"),


[trafficcontrol] 10/15: convert tenancy package to new interfaces

Posted by da...@apache.org.
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 840729ac74e150edba557151bb14b59680eaabec
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Wed Jun 27 12:16:08 2018 -0600

    convert tenancy package to new interfaces
---
 lib/go-tc/parameters.go                          | 52 +++++++------
 traffic_ops/traffic_ops_golang/routes.go         | 19 +++--
 traffic_ops/traffic_ops_golang/tenant/tenancy.go | 96 ++++--------------------
 3 files changed, 51 insertions(+), 116 deletions(-)

diff --git a/lib/go-tc/parameters.go b/lib/go-tc/parameters.go
index 9bbf3b9..1151f30 100644
--- a/lib/go-tc/parameters.go
+++ b/lib/go-tc/parameters.go
@@ -79,7 +79,7 @@ type ProfileParameterByNamePost struct {
 	Value      *string `json:"value"`
 }
 
-func (p *ProfileParameterByNamePost) Validate(tx *sql.Tx) error {
+func (p *ProfileParameterByNamePost) Validate(tx *sql.Tx) []error {
 	errs := []string{}
 	if p.ConfigFile == nil || *p.ConfigFile == "" {
 		errs = append(errs, "configFile")
@@ -94,7 +94,7 @@ func (p *ProfileParameterByNamePost) Validate(tx *sql.Tx) error {
 		errs = append(errs, "value")
 	}
 	if len(errs) > 0 {
-		return errors.New("required fields missing: " + strings.Join(errs, ", "))
+		return []error{errors.New("required fields missing: " + strings.Join(errs, ", "))}
 	}
 	return nil
 }
@@ -121,16 +121,18 @@ func (pp *ProfileParametersByNamePost) UnmarshalJSON(bts []byte) error {
 	return nil
 }
 
-func (pp *ProfileParametersByNamePost) Validate(tx *sql.Tx) error {
-	errs := []string{}
+func (pp *ProfileParametersByNamePost) Validate(tx *sql.Tx) []error {
+	errs := []error{}
 	ppArr := ([]ProfileParameterByNamePost)(*pp)
 	for i, profileParam := range ppArr {
-		if err := profileParam.Validate(tx); err != nil {
-			errs = append(errs, "object "+strconv.Itoa(i)+": "+err.Error())
+		if ppErrs := profileParam.Validate(tx); len(ppErrs) > 0 {
+			for _, err := range ppErrs {
+				errs = append(errs, errors.New("object "+strconv.Itoa(i)+": "+err.Error()))
+			}
 		}
 	}
 	if len(errs) > 0 {
-		return errors.New("validate errors: " + strings.Join(errs, "; "))
+		return errs
 	}
 	return nil
 }
@@ -158,25 +160,25 @@ func (pp *PostProfileParam) Sanitize(tx *sql.Tx) {
 	}
 }
 
-func (pp *PostProfileParam) Validate(tx *sql.Tx) error {
+func (pp *PostProfileParam) Validate(tx *sql.Tx) []error {
 	pp.Sanitize(tx)
-	errs := []string{}
+	errs := []error{}
 	if pp.ProfileID == nil {
-		errs = append(errs, "profileId missing")
+		errs = append(errs, errors.New("profileId missing"))
 	} else if ok, err := ProfileExists(*pp.ProfileID, tx); err != nil {
-		errs = append(errs, "checking profile ID "+strconv.Itoa(int(*pp.ProfileID))+" existence: "+err.Error())
+		errs = append(errs, errors.New("checking profile ID "+strconv.Itoa(int(*pp.ProfileID))+" existence: "+err.Error()))
 	} else if !ok {
-		errs = append(errs, "no profile with ID "+strconv.Itoa(int(*pp.ProfileID))+" exists")
+		errs = append(errs, errors.New("no profile with ID "+strconv.Itoa(int(*pp.ProfileID))+" exists"))
 	}
 	if pp.ParamIDs == nil {
-		errs = append(errs, "paramIds missing")
+		errs = append(errs, errors.New("paramIds missing"))
 	} else if ok, err := ParamsExist(*pp.ParamIDs, tx); err != nil {
-		errs = append(errs, fmt.Sprintf("checking parameter IDs %v existence: "+err.Error(), *pp.ParamIDs))
+		errs = append(errs, errors.New(fmt.Sprintf("checking parameter IDs %v existence: "+err.Error(), *pp.ParamIDs)))
 	} else if !ok {
-		errs = append(errs, fmt.Sprintf("parameters with IDs %v don't all exist", *pp.ParamIDs))
+		errs = append(errs, errors.New(fmt.Sprintf("parameters with IDs %v don't all exist", *pp.ParamIDs)))
 	}
 	if len(errs) > 0 {
-		return errors.New("validate errors: " + strings.Join(errs, ", "))
+		return errs
 	}
 	return nil
 }
@@ -193,26 +195,26 @@ func (pp *PostParamProfile) Sanitize(tx *sql.Tx) {
 	}
 }
 
-func (pp *PostParamProfile) Validate(tx *sql.Tx) error {
+func (pp *PostParamProfile) Validate(tx *sql.Tx) []error {
 	pp.Sanitize(tx)
 
-	errs := []string{}
+	errs := []error{}
 	if pp.ParamID == nil {
-		errs = append(errs, "paramId missing")
+		errs = append(errs, errors.New("paramId missing"))
 	} else if ok, err := ParamExists(*pp.ParamID, tx); err != nil {
-		errs = append(errs, "checking param ID "+strconv.Itoa(int(*pp.ParamID))+" existence: "+err.Error())
+		errs = append(errs, errors.New("checking param ID "+strconv.Itoa(int(*pp.ParamID))+" existence: "+err.Error()))
 	} else if !ok {
-		errs = append(errs, "no parameter with ID "+strconv.Itoa(int(*pp.ParamID))+" exists")
+		errs = append(errs, errors.New("no parameter with ID "+strconv.Itoa(int(*pp.ParamID))+" exists"))
 	}
 	if pp.ProfileIDs == nil {
-		errs = append(errs, "profileIds missing")
+		errs = append(errs, errors.New("profileIds missing"))
 	} else if ok, err := ProfilesExist(*pp.ProfileIDs, tx); err != nil {
-		errs = append(errs, fmt.Sprintf("checking profiles IDs %v existence: "+err.Error(), *pp.ProfileIDs))
+		errs = append(errs, errors.New(fmt.Sprintf("checking profiles IDs %v existence: "+err.Error(), *pp.ProfileIDs)))
 	} else if !ok {
-		errs = append(errs, fmt.Sprintf("profiles with IDs %v don't all exist", *pp.ProfileIDs))
+		errs = append(errs, errors.New(fmt.Sprintf("profiles with IDs %v don't all exist", *pp.ProfileIDs)))
 	}
 	if len(errs) > 0 {
-		return errors.New("validate errors: " + strings.Join(errs, ", "))
+		return errs
 	}
 	return nil
 }
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 6b3b215..d26503d 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -304,21 +304,20 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.1, http.MethodGet, `profiles/{id}/unassigned_parameters/?(\.json)?$`, profileparameter.GetUnassigned(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `profiles/name/{name}/parameters/?(\.json)?$`, profileparameter.GetProfileName(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodGet, `parameters/profile/{name}/?(\.json)?$`, profileparameter.GetProfileName(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
-		{1.1, http.MethodPost, `profiles/name/{name}/parameters/?$`, profileparameter.PostProfileParamsByName(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `profiles/{id}/parameters/?$`, profileparameter.PostProfileParamsByID(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profiles/name/{name}/parameters/?$`, profileparameter.PostProfileParamsByName, auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profiles/{id}/parameters/?$`, profileparameter.PostProfileParamsByID, auth.PrivLevelOperations, Authenticated, nil},
 		{1.1, http.MethodGet, `profileparameters/?(\.json)?$`, api.ReadHandler(profileparameter.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.1, http.MethodPost, `profileparameters/?$`, api.CreateHandler(profileparameter.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `profileparameter/?$`, profileparameter.PostProfileParam(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `parameterprofile/?$`, profileparameter.PostParamProfile(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `profileparameter/?$`, profileparameter.PostProfileParam, auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `parameterprofile/?$`, profileparameter.PostParamProfile, auth.PrivLevelOperations, Authenticated, nil},
 		{1.1, http.MethodDelete, `profileparameters/{profileId}/{parameterId}$`, api.DeleteHandler(profileparameter.GetTypeSingleton()), 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},
-		{1.1, http.MethodPut, `tenants/{id}$`, api.UpdateHandler(tenant.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodPost, `tenants/?$`, api.CreateHandler(tenant.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
-		{1.1, http.MethodDelete, `tenants/{id}$`, api.DeleteHandler(tenant.GetRefType(), d.DB), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodGet, `tenants/?(\.json)?$`, api.ReadHandler(tenant.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `tenants/{id}$`, api.ReadHandler(tenant.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `tenants/{id}$`, api.UpdateHandler(tenant.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `tenants/?$`, api.CreateHandler(tenant.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `tenants/{id}$`, api.DeleteHandler(tenant.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//CRConfig
 		{1.1, http.MethodGet, `cdns/{cdn}/snapshot/?$`, crconfig.SnapshotGetHandler(d.DB, d.Config), auth.PrivLevelReadOnly, Authenticated, nil},
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index cd8edd3..6b66141 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -117,7 +117,7 @@ func CheckID(tx *sql.Tx, user *auth.CurrentUser, dsID int) (error, error, int) {
 // NOTE: This method does not use the use_tenancy parameter and if this method is being used
 // to control tenancy the parameter must be checked. The method IsResourceAuthorizedToUser checks the use_tenancy parameter
 // and should be used for this purpose in most cases.
-func GetUserTenantList(user auth.CurrentUser, db *sqlx.DB) ([]TOTenant, error) {
+func GetUserTenantList(user auth.CurrentUser, db *sqlx.DB) ([]tc.TenantNullable, error) {
 	query := `WITH RECURSIVE q AS (SELECT id, name, active, parent_id, last_updated FROM tenant WHERE id = $1
 	UNION SELECT t.id, t.name, t.active, t.parent_id, t.last_updated  FROM tenant t JOIN q ON q.id = t.parent_id)
 	SELECT id, name, active, parent_id, last_updated FROM q;`
@@ -136,14 +136,14 @@ func GetUserTenantList(user auth.CurrentUser, db *sqlx.DB) ([]TOTenant, error) {
 	}
 	defer rows.Close()
 
-	tenants := []TOTenant{}
+	tenants := []tc.TenantNullable{}
 
 	for rows.Next() {
 		if err := rows.Scan(&tenantID, &name, &active, &parentID, &lastUpdated); err != nil {
 			return nil, err
 		}
 
-		tenants = append(tenants, TOTenant{ID: &tenantID, Name: &name, Active: &active, ParentID: &parentID})
+		tenants = append(tenants, tc.TenantNullable{ID: &tenantID, Name: &name, Active: &active, ParentID: &parentID})
 	}
 
 	return tenants, nil
@@ -292,7 +292,6 @@ func IsResourceAuthorizedToUserTx(resourceTenantID int, user *auth.CurrentUser,
 	}
 }
 
-
 // TOTenant provides a local type against which to define methods
 type TOTenant struct {
 	ReqInfo *api.APIInfo `json:"-"`
@@ -353,7 +352,7 @@ func (ten *TOTenant) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (ten TOTenant) Validate(db *sqlx.DB) []error {
+func (ten TOTenant) Validate() []error {
 	errs := validation.Errors{
 		"name":     validation.Validate(ten.Name, validation.Required),
 		"active":   validation.Validate(ten.Active), // only validate it's boolean
@@ -369,24 +368,8 @@ func (ten TOTenant) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted tenant and have
 //to be added to the struct
-func (ten *TOTenant) 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(), ten)
+func (ten *TOTenant) Create() (error, tc.ApiErrorType) {
+	resultRows, err := ten.ReqInfo.Tx.NamedQuery(insertQuery(), ten)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -421,17 +404,12 @@ func (ten *TOTenant) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 	}
 	ten.SetKeys(map[string]interface{}{"id": id})
 	ten.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
 }
 
 // Read implements the tc.Reader interface
-func (ten *TOTenant) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (ten *TOTenant) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -451,7 +429,7 @@ func (ten *TOTenant) Read(db *sqlx.DB, parameters map[string]string, user auth.C
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := ten.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying tenants: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -476,25 +454,9 @@ func (ten *TOTenant) Read(db *sqlx.DB, parameters map[string]string, user auth.C
 //ParsePQUniqueConstraintError is used to determine if a tenant 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 (ten *TOTenant) 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 (ten *TOTenant) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with tenant: %++v", updateQuery(), ten)
-	resultRows, err := tx.NamedQuery(updateQuery(), ten)
+	resultRows, err := ten.ReqInfo.Tx.NamedQuery(updateQuery(), ten)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -525,41 +487,19 @@ func (ten *TOTenant) 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
 }
 
 //Delete implements the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (ten *TOTenant) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (ten *TOTenant) Delete() (error, tc.ApiErrorType) {
 	if ten.ID == nil {
 		// should never happen...
 		return errors.New("invalid tenant: id is nil"), tc.SystemError
 	}
-	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
-	}
 
 	log.Debugf("about to run exec query: %s with tenant: %++v", deleteQuery(), ten)
-	result, err := tx.NamedExec(deleteQuery(), ten)
+	result, err := ten.ReqInfo.Tx.NamedExec(deleteQuery(), ten)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err = fmt.Errorf("pqErr is %++v\n", pqErr)
@@ -579,7 +519,7 @@ func (ten *TOTenant) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 
 			// another query to get tenant name for the error message
 			name := strconv.Itoa(*ten.ID)
-			if err := db.QueryRow(`SELECT name FROM tenant WHERE id = $1`, *ten.ID).Scan(&name); err != nil {
+			if err := ten.ReqInfo.Tx.QueryRow(`SELECT name FROM tenant WHERE id = $1`, *ten.ID).Scan(&name); err != nil {
 				// use ID as a backup for name the error -- this should never happen
 				log.Debugf("error getting tenant name: %++v", err)
 			}
@@ -600,12 +540,7 @@ func (ten *TOTenant) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiEr
 		}
 		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
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
@@ -667,4 +602,3 @@ func getDSTenantIDByIDTx(tx *sql.Tx, id int) (*int, bool, error) {
 	}
 	return tenantID, true, nil
 }
-


[trafficcontrol] 02/15: all packages through origins (alphabetically) refactored

Posted by da...@apache.org.
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 43f00c39d8fdf7241d4e80d9ebec45086f77c620
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 16:56:56 2018 -0600

    all packages through origins (alphabetically) refactored
---
 traffic_ops/traffic_ops_golang/api/change_log.go   |  14 +-
 .../traffic_ops_golang/api/change_log_test.go      |   3 +-
 .../traffic_ops_golang/api/shared_handlers.go      |  12 +-
 .../traffic_ops_golang/api/shared_handlers_test.go | 109 +++++++--------
 .../traffic_ops_golang/api/shared_interfaces.go    |   4 +-
 traffic_ops/traffic_ops_golang/asn/asns_test.go    |  10 +-
 .../traffic_ops_golang/cachegroup/cachegroups.go   |   2 +-
 .../cachegroup/cachegroups_test.go                 |  20 +--
 .../traffic_ops_golang/cachegroup/queueupdate.go   |   2 +-
 traffic_ops/traffic_ops_golang/cdn/cdns_test.go    |  20 +--
 traffic_ops/traffic_ops_golang/cdn/queue.go        |   2 +-
 .../traffic_ops_golang/coordinate/coordinates.go   | 100 +++-----------
 .../coordinate/coordinates_test.go                 |  33 ++---
 traffic_ops/traffic_ops_golang/crconfig/handler.go |   4 +-
 .../deliveryservice/deliveryservicesv12.go         |  10 +-
 .../deliveryservice/deliveryservicesv13.go         |  16 +--
 .../deliveryservice/request/comment/comments.go    | 105 +++------------
 .../request/comment/comments_test.go               |   7 +-
 .../deliveryservice/request/requests.go            |   4 +-
 .../deliveryservice/request/requests_test.go       |   4 +-
 .../deliveryservicesregexes.go                     |  14 +-
 .../traffic_ops_golang/division/divisions.go       | 149 +++------------------
 .../traffic_ops_golang/division/divisions_test.go  |   8 +-
 traffic_ops/traffic_ops_golang/origin/origins.go   | 148 ++++++--------------
 .../traffic_ops_golang/origin/origins_test.go      |  17 +--
 traffic_ops/traffic_ops_golang/routes.go           |  58 ++++----
 traffic_ops/traffic_ops_golang/tenant/tenancy.go   |   6 +-
 27 files changed, 289 insertions(+), 592 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/change_log.go b/traffic_ops/traffic_ops_golang/api/change_log.go
index 99f1c5e..b8117ec 100644
--- a/traffic_ops/traffic_ops_golang/api/change_log.go
+++ b/traffic_ops/traffic_ops_golang/api/change_log.go
@@ -51,7 +51,7 @@ const (
 	Deleted   = "Deleted"
 )
 
-func CreateChangeLog(level string, action string, i Identifier, user auth.CurrentUser, tx *sqlx.Tx) 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()
@@ -66,7 +66,7 @@ func CreateChangeLog(level string, action string, i Identifier, user auth.Curren
 	return CreateChangeLogRawErr(level, msg, user, tx.Tx)
 }
 
-func CreateChangeLogBuildMsg(level string, action string, user auth.CurrentUser, tx *sqlx.Tx, 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) + " "
@@ -76,12 +76,14 @@ func CreateChangeLogBuildMsg(level string, action string, user auth.CurrentUser,
 	return CreateChangeLogRawErr(level, msg, user, tx.Tx)
 }
 
-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 CreateChangeLogRawErr(level string, msg string, user *auth.CurrentUser, tx *sql.Tx) error {
+	 if _, err := tx.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID); err != nil {
+		 return errors.New("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
+	 }
+	 return nil
 }
 
-func CreateChangeLogRaw(level string, msg string, user auth.CurrentUser, db *sql.DB) {
+func CreateChangeLogRaw(level string, msg string, user *auth.CurrentUser, db *sql.DB) {
 	if _, err := db.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID); err != nil {
 		log.Errorln("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
 	}
diff --git a/traffic_ops/traffic_ops_golang/api/change_log_test.go b/traffic_ops/traffic_ops_golang/api/change_log_test.go
index 190dfd8..38a44d5 100644
--- a/traffic_ops/traffic_ops_golang/api/change_log_test.go
+++ b/traffic_ops/traffic_ops_golang/api/change_log_test.go
@@ -62,9 +62,10 @@ func TestCreateChangeLog(t *testing.T) {
 	keys, _ := i.GetKeys()
 	expectedMessage := Created + " " + i.GetType() + ": " + i.GetAuditName() + " keys: { id:" + strconv.Itoa(keys["id"].(int)) + " }"
 
+	mock.ExpectBegin()
 	mock.ExpectExec("INSERT").WithArgs(ApiChange, expectedMessage, 1).WillReturnResult(sqlmock.NewResult(1, 1))
 	user := auth.CurrentUser{ID: 1}
-	err = CreateChangeLog(ApiChange, Created, &i, user, db)
+	err = CreateChangeLog(ApiChange, Created, &i, &user, db.MustBegin())
 	if err != nil {
 		t.Fatal(err)
 	}
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index fc0e6bb..b01eb85 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -238,7 +238,7 @@ func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) 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)
+			authorized, err := t.IsTenantAuthorized(user)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -256,7 +256,7 @@ func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 			return
 		}
 		//auditing here
-		err = CreateChangeLog(ApiChange, Updated, u, *inf.User, inf.Tx)
+		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
@@ -327,7 +327,7 @@ func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) 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(*inf.User)
+			authorized, err := t.IsTenantAuthorized(inf.User)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -347,7 +347,7 @@ func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 		}
 		//audit here
 		log.Debugf("changelog for delete on object")
-		err = CreateChangeLog(ApiChange, Deleted, d, *inf.User, inf.Tx)
+		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
@@ -402,7 +402,7 @@ func CreateHandler(typeConstructor func(reqInfo *APIInfo) CRUDer) http.HandlerFu
 
 		// 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(*inf.User)
+			authorized, err := t.IsTenantAuthorized(inf.User)
 			if err != nil {
 				handleErrs(http.StatusBadRequest, err)
 				return
@@ -419,7 +419,7 @@ func CreateHandler(typeConstructor func(reqInfo *APIInfo) CRUDer) http.HandlerFu
 			return
 		}
 
-		err = CreateChangeLog(ApiChange, Created, i, *inf.User, inf.Tx)
+		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
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
index d0827ba..56880d4 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
@@ -30,6 +30,7 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/config"
 	"github.com/jmoiron/sqlx"
 
 	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
@@ -41,7 +42,14 @@ type tester struct {
 	errorType tc.ApiErrorType //only for testing
 }
 
-type emptyTester tester
+func GetTypeSingleton() func(apiInfo *APIInfo) CRUDer{
+	return func(apiInfo *APIInfo) CRUDer {
+		tester := tester{}
+		return &tester
+	}
+}
+
+var cfg = config.Config{}
 
 func (i tester) GetKeyFieldsInfo() []KeyFieldInfo {
 	return []KeyFieldInfo{{"id", GetIntKey}}
@@ -66,7 +74,7 @@ func (i *tester) GetAuditName() string {
 }
 
 //Validator interface function
-func (v *tester) Validate(db *sqlx.DB) []error {
+func (v *tester) Validate() []error {
 	if v.ID < 1 {
 		return []error{errors.New("ID is too low")}
 	}
@@ -74,27 +82,22 @@ func (v *tester) Validate(db *sqlx.DB) []error {
 }
 
 //Creator interface functions
-func (i *tester) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (i *tester) Create() (error, tc.ApiErrorType) {
 	return i.error, i.errorType
 }
 
 //Reader interface functions
-func (i *tester) Read(db *sqlx.DB, v map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (i *tester) Read(v map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	return []interface{}{tester{ID: 1}}, nil, tc.NoError
 }
 
-//Reader interface functions
-func (i *emptyTester) Read(db *sqlx.DB, v map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	return []interface{}{}, nil, tc.NoError
-}
-
 //Updater interface functions
-func (i *tester) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (i *tester) Update() (error, tc.ApiErrorType) {
 	return i.error, i.errorType
 }
 
 //Deleter interface functions
-func (i *tester) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (i *tester) Delete() (error, tc.ApiErrorType) {
 	return i.error, i.errorType
 }
 
@@ -123,16 +126,23 @@ func TestCreateHandler(t *testing.T) {
 	ctx := r.Context()
 	ctx = context.WithValue(ctx, auth.CurrentUserKey,
 		auth.CurrentUser{UserName: "username", ID: 1, PrivLevel: auth.PrivLevelAdmin})
+	ctx = context.WithValue(ctx, DBContextKey, db)
+	ctx = context.WithValue(ctx, ConfigContextKey, &cfg)
+	ctx = context.WithValue(ctx, ReqIDContextKey, uint64(0))
+	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id":"1"})
 
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{}
-	createFunc := CreateHandler(&typeRef, db)
+	typeRef := tester{ID:1}
+
+	createFunc := CreateHandler(GetTypeSingleton())
 
 	//verifies we get the right changelog insertion
-	expectedMessage := Created + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " id: 1"
+	expectedMessage := Created + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " keys: { id:1 }"
+	mock.ExpectBegin()
 	mock.ExpectExec("INSERT").WithArgs(ApiChange, expectedMessage, 1).WillReturnResult(sqlmock.NewResult(1, 1))
+	mock.ExpectCommit()
 
 	createFunc(w, r)
 
@@ -143,43 +153,8 @@ func TestCreateHandler(t *testing.T) {
 	}
 }
 
-func TestEmptyReadHandler(t *testing.T) {
-	mockDB, _, err := sqlmock.New()
-	if err != nil {
-		t.Fatalf("an error '%s' was not expected when opening a stub database connection", err)
-	}
-	defer mockDB.Close()
-
-	db := sqlx.NewDb(mockDB, "sqlmock")
-	defer db.Close()
-
-	w := httptest.NewRecorder()
-	r, err := http.NewRequest("", "", nil)
-	if err != nil {
-		t.Error("Error creating new request")
-	}
-
-	ctx := r.Context()
-	ctx = context.WithValue(ctx, auth.CurrentUserKey,
-		auth.CurrentUser{UserName: "username", ID: 1, PrivLevel: auth.PrivLevelAdmin})
-	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{})
-	// Add our context to the request
-	r = r.WithContext(ctx)
-
-	typeRef := emptyTester{}
-	readFunc := ReadHandler(&typeRef, db)
-
-	readFunc(w, r)
-
-	//verifies the body is in the expected format
-	body := `{"response":[]}`
-	if w.Body.String() != body {
-		t.Error("Expected body", body, "got", w.Body.String())
-	}
-}
-
 func TestReadHandler(t *testing.T) {
-	mockDB, _, err := sqlmock.New()
+	mockDB, mock, err := sqlmock.New()
 	if err != nil {
 		t.Fatalf("an error '%s' was not expected when opening a stub database connection", err)
 	}
@@ -198,11 +173,17 @@ func TestReadHandler(t *testing.T) {
 	ctx = context.WithValue(ctx, auth.CurrentUserKey,
 		auth.CurrentUser{UserName: "username", ID: 1, PrivLevel: auth.PrivLevelAdmin})
 	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id": "1"})
+	ctx = context.WithValue(ctx, DBContextKey, db)
+	ctx = context.WithValue(ctx, ConfigContextKey, &cfg)
+	ctx = context.WithValue(ctx, ReqIDContextKey, uint64(0))
+
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{}
-	readFunc := ReadHandler(&typeRef, db)
+	readFunc := ReadHandler(GetTypeSingleton())
+
+	mock.ExpectBegin()
+	mock.ExpectCommit()
 
 	readFunc(w, r)
 
@@ -233,15 +214,21 @@ func TestUpdateHandler(t *testing.T) {
 	ctx = context.WithValue(ctx, auth.CurrentUserKey,
 		auth.CurrentUser{UserName: "username", ID: 1, PrivLevel: auth.PrivLevelAdmin})
 	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id": "1"})
+	ctx = context.WithValue(ctx, DBContextKey, db)
+	ctx = context.WithValue(ctx, ConfigContextKey, &cfg)
+	ctx = context.WithValue(ctx, ReqIDContextKey, uint64(0))
+
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{}
-	updateFunc := UpdateHandler(&typeRef, db)
+	typeRef := tester{ID:1}
+	updateFunc := UpdateHandler(GetTypeSingleton())
 
 	//verifies we get the right changelog insertion
-	expectedMessage := Updated + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " id: 1"
+	expectedMessage := Updated + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " keys: { id:1 }"
+	mock.ExpectBegin()
 	mock.ExpectExec("INSERT").WithArgs(ApiChange, expectedMessage, 1).WillReturnResult(sqlmock.NewResult(1, 1))
+	mock.ExpectCommit()
 
 	updateFunc(w, r)
 
@@ -272,16 +259,20 @@ func TestDeleteHandler(t *testing.T) {
 	ctx = context.WithValue(ctx, auth.CurrentUserKey,
 		auth.CurrentUser{UserName: "username", ID: 1, PrivLevel: auth.PrivLevelAdmin})
 	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id": "1"})
+	ctx = context.WithValue(ctx, DBContextKey, db)
+	ctx = context.WithValue(ctx, ConfigContextKey, &cfg)
+	ctx = context.WithValue(ctx, ReqIDContextKey, uint64(0))
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{}
-	deleteFunc := DeleteHandler(&typeRef, db)
+	typeRef := tester{ID:1}
+	deleteFunc := DeleteHandler(GetTypeSingleton())
 
 	//verifies we get the right changelog insertion
-	expectedMessage := Deleted + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " id: 1"
+	expectedMessage := Deleted + " " + typeRef.GetType() + ": " + typeRef.GetAuditName() + " keys: { id:1 }"
+	mock.ExpectBegin()
 	mock.ExpectExec("INSERT").WithArgs(ApiChange, expectedMessage, 1).WillReturnResult(sqlmock.NewResult(1, 1))
-
+	mock.ExpectCommit()
 	deleteFunc(w, r)
 
 	//verifies the body is in the expected format
diff --git a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
index 515a1d1..a8df96f 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
@@ -39,7 +39,6 @@ type Updater interface {
 
 type Identifier interface {
 	GetKeys() (map[string]interface{}, bool)
-	SetKeys(map[string]interface{})
 	GetType() string
 	GetAuditName() string
 	GetKeyFieldsInfo() []KeyFieldInfo
@@ -47,6 +46,7 @@ type Identifier interface {
 
 type Creator interface {
 	Create() (error, tc.ApiErrorType)
+	SetKeys(map[string]interface{})
 }
 
 type Deleter interface {
@@ -58,7 +58,7 @@ type Validator interface {
 }
 
 type Tenantable interface {
-	IsTenantAuthorized(user auth.CurrentUser) (bool, error)
+	IsTenantAuthorized(user *auth.CurrentUser) (bool, error)
 }
 
 type Reader interface {
diff --git a/traffic_ops/traffic_ops_golang/asn/asns_test.go b/traffic_ops/traffic_ops_golang/asn/asns_test.go
index 12e8de4..5ad859a 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns_test.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns_test.go
@@ -27,11 +27,11 @@ import (
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
 	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
+	"github.com/apache/trafficcontrol/lib/go-util"
 )
 
 func getTestASNs() []tc.ASNNullable {
@@ -65,7 +65,7 @@ func TestGetASNs(t *testing.T) {
 	defer db.Close()
 
 	testCase := getTestASNs()
-	cols := test.ColsFromStructByTag("db", TOASNV11{})
+	cols := test.ColsFromStructByTag("db", tc.ASNNullable{})
 	rows := sqlmock.NewRows(cols)
 
 	for _, ts := range testCase {
@@ -76,10 +76,12 @@ func TestGetASNs(t *testing.T) {
 			*ts.LastUpdated,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
-
-	asns, errs, _ := GetTypeSingleton(db)().Read(v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	asns, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 
 	if len(errs) > 0 {
 		t.Errorf("asn.Read expected: no errors, actual: %v", errs)
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index a43df6f..03da6d2 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -42,7 +42,7 @@ type TOCacheGroup struct{
 	v13.CacheGroupNullable
 	}
 
-func GetV11TypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCacheGroup{reqInfo, v13.CacheGroupNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
index 0bd2066..1c4eaff 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
@@ -28,8 +28,8 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -79,7 +79,6 @@ func TestReadCacheGroups(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-	refType := GetRefType()
 
 	testCGs := getTestCacheGroups()
 	cols := test.ColsFromStructByTag("db", v13.CacheGroup{})
@@ -99,10 +98,13 @@ func TestReadCacheGroups(t *testing.T) {
 			ts.LastUpdated,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	cachegroups, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	cachegroups, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("cdn.Read expected: no errors, actual: %v", errs)
 	}
@@ -158,7 +160,7 @@ func TestValidate(t *testing.T) {
 	ty := "EDGE_LOC"
 	ti := 6
 	lu := tc.TimeNoMod{Time: time.Now()}
-	c := TOCacheGroup{ID: &id,
+	c := TOCacheGroup{CacheGroupNullable: v13.CacheGroupNullable{ID: &id,
 		Name:        &nm,
 		ShortName:   &sn,
 		Latitude:    &la,
@@ -166,8 +168,8 @@ func TestValidate(t *testing.T) {
 		Type:        &ty,
 		TypeID:      &ti,
 		LastUpdated: &lu,
-	}
-	errs := test.SortErrors(c.Validate(nil))
+	}}
+	errs := test.SortErrors(c.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'latitude' Must be a floating point number within the range +-90`),
@@ -185,7 +187,7 @@ func TestValidate(t *testing.T) {
 	sn = `awesome-cachegroup`
 	la = 90.0
 	lo = 90.0
-	c = TOCacheGroup{ID: &id,
+	c = TOCacheGroup{CacheGroupNullable: v13.CacheGroupNullable{ID: &id,
 		Name:        &nm,
 		ShortName:   &sn,
 		Latitude:    &la,
@@ -193,9 +195,9 @@ func TestValidate(t *testing.T) {
 		Type:        &ty,
 		TypeID:      &ti,
 		LastUpdated: &lu,
-	}
+	}}
 	expectedErrs = []error{}
-	errs = c.Validate(nil)
+	errs = c.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/queueupdate.go b/traffic_ops/traffic_ops_golang/cachegroup/queueupdate.go
index 7ef8d1c..30bc947 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/queueupdate.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/queueupdate.go
@@ -92,7 +92,7 @@ func QueueUpdates(db *sql.DB) http.HandlerFunc {
 			CDN:            *reqObj.CDN,
 			CacheGroupID:   cgID,
 		})
-		api.CreateChangeLogRaw(api.ApiChange, "Server updates "+reqObj.Action+"d for "+string(cgName), *user, db)
+		api.CreateChangeLogRaw(api.ApiChange, "Server updates "+reqObj.Action+"d for "+string(cgName), user, db)
 	}
 }
 
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
index f4910e9..c09529e 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
@@ -27,9 +27,9 @@ import (
 	"time"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -65,8 +65,6 @@ func TestReadCDNs(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-	refType := GetRefType()
-
 	testCDNs := getTestCDNs()
 	cols := test.ColsFromStructByTag("db", v13.CDN{})
 	rows := sqlmock.NewRows(cols)
@@ -80,10 +78,13 @@ func TestReadCDNs(t *testing.T) {
 			ts.Name,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
-	v := map[string]string{"dsId": "1"}
+	mock.ExpectCommit()
 
-	servers, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	v := map[string]string{"dsId": "1"}
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	servers, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("cdn.Read expected: no errors, actual: %v", errs)
 	}
@@ -132,8 +133,8 @@ func TestInterfaces(t *testing.T) {
 func TestValidate(t *testing.T) {
 	// invalid name, empty domainname
 	n := "not_a_valid_cdn"
-	c := TOCDN{Name: &n}
-	errs := test.SortErrors(c.Validate(nil))
+	c := TOCDN{CDNNullable: v13.CDNNullable{Name: &n}}
+	errs := test.SortErrors(c.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'domainName' cannot be blank`),
@@ -147,11 +148,10 @@ func TestValidate(t *testing.T) {
 	//  name,  domainname both valid
 	n = "This.is.2.a-Valid---CDNNAME."
 	d := `awesome-cdn.example.net`
-	c = TOCDN{Name: &n, DomainName: &d}
+	c = TOCDN{CDNNullable: v13.CDNNullable{Name: &n, DomainName: &d}}
 	expectedErrs = []error{}
-	errs = c.Validate(nil)
+	errs = c.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
-
 }
diff --git a/traffic_ops/traffic_ops_golang/cdn/queue.go b/traffic_ops/traffic_ops_golang/cdn/queue.go
index 4e7cc6e..b8a4b82 100644
--- a/traffic_ops/traffic_ops_golang/cdn/queue.go
+++ b/traffic_ops/traffic_ops_golang/cdn/queue.go
@@ -64,7 +64,7 @@ func Queue(db *sql.DB) http.HandlerFunc {
 			return
 		}
 		api.WriteResp(w, r, QueueResp{Action: reqObj.Action, CDNID: int64(intParams["id"])})
-		api.CreateChangeLogRaw(api.ApiChange, "Server updates "+reqObj.Action+"d for cdn "+params["id"], *user, db)
+		api.CreateChangeLogRaw(api.ApiChange, "Server updates "+reqObj.Action+"d for cdn "+params["id"], user, db)
 	}
 }
 
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index 4b852d7..2707519 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.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"
@@ -39,13 +38,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOCoordinate v13.CoordinateNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOCoordinate{}
+type TOCoordinate struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.CoordinateNullable
+}
 
-func GetRefType() *TOCoordinate {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOCoordinate{reqInfo, v13.CoordinateNullable{}}
+		return &toReturn
+	}
 }
 
 func (coordinate TOCoordinate) GetKeyFieldsInfo() []api.KeyFieldInfo {
@@ -102,7 +104,7 @@ func IsValidCoordinateName(str string) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (coordinate TOCoordinate) Validate(db *sqlx.DB) []error {
+func (coordinate TOCoordinate) Validate() []error {
 	validName := validation.NewStringRule(IsValidCoordinateName, "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"
@@ -121,24 +123,8 @@ func (coordinate TOCoordinate) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted coordinate and have
 //to be added to the struct
-func (coordinate *TOCoordinate) 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(), coordinate)
+func (coordinate *TOCoordinate) Create() (error, tc.ApiErrorType) {
+	resultRows, err := coordinate.ReqInfo.Tx.NamedQuery(insertQuery(), coordinate)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -174,16 +160,10 @@ func (coordinate *TOCoordinate) Create(db *sqlx.DB, user auth.CurrentUser) (erro
 	}
 	coordinate.SetKeys(map[string]interface{}{"id": id})
 	coordinate.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 (coordinate *TOCoordinate) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (coordinate *TOCoordinate) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -200,7 +180,7 @@ func (coordinate *TOCoordinate) 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 := coordinate.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Coordinate: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -225,25 +205,9 @@ func (coordinate *TOCoordinate) Read(db *sqlx.DB, parameters map[string]string,
 //ParsePQUniqueConstraintError is used to determine if a coordinate 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 (coordinate *TOCoordinate) 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 (coordinate *TOCoordinate) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with coordinate: %++v", updateQuery(), coordinate)
-	resultRows, err := tx.NamedQuery(updateQuery(), coordinate)
+	resultRows, err := coordinate.ReqInfo.Tx.NamedQuery(updateQuery(), coordinate)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -276,36 +240,14 @@ func (coordinate *TOCoordinate) Update(db *sqlx.DB, user auth.CurrentUser) (erro
 			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 Coordinate implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (coordinate *TOCoordinate) 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 (coordinate *TOCoordinate) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with coordinate: %++v", deleteQuery(), coordinate)
-	result, err := tx.NamedExec(deleteQuery(), coordinate)
+	result, err := coordinate.ReqInfo.Tx.NamedExec(deleteQuery(), coordinate)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -321,12 +263,6 @@ func (coordinate *TOCoordinate) Delete(db *sqlx.DB, user auth.CurrentUser) (erro
 			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
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
index 16b3837..6d3275b 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
@@ -26,10 +26,10 @@ import (
 	"testing"
 	"time"
 
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"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/test"
 	"github.com/jmoiron/sqlx"
 
@@ -69,8 +69,6 @@ func TestReadCoordinates(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-	refType := GetRefType()
-
 	testCoords := getTestCoordinates()
 	cols := test.ColsFromStructByTag("db", v13.Coordinate{})
 	rows := sqlmock.NewRows(cols)
@@ -84,10 +82,13 @@ func TestReadCoordinates(t *testing.T) {
 			ts.LastUpdated,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	coordinates, errs, _ := refType.Read(db, v, auth.CurrentUser{})
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	coordinates, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("coordinate.Read expected: no errors, actual: %v", errs)
 	}
@@ -140,13 +141,13 @@ func TestValidate(t *testing.T) {
 	la := -190.0
 	lo := -190.0
 	lu := tc.TimeNoMod{Time: time.Now()}
-	c := TOCoordinate{ID: &id,
-		Name:        &nm,
-		Latitude:    &la,
-		Longitude:   &lo,
+	c := TOCoordinate{ CoordinateNullable: v13.CoordinateNullable{ID: &id,
+		Name: &nm,
+		Latitude: &la,
+		Longitude: &lo,
 		LastUpdated: &lu,
-	}
-	errs := test.SortErrors(c.Validate(nil))
+	}}
+	errs := test.SortErrors(c.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'latitude' Must be a floating point number within the range +-90`),
@@ -162,14 +163,14 @@ func TestValidate(t *testing.T) {
 	nm = "This.is.2.a-Valid---Coordinate."
 	la = 90.0
 	lo = 90.0
-	c = TOCoordinate{ID: &id,
-		Name:        &nm,
-		Latitude:    &la,
-		Longitude:   &lo,
+	c = TOCoordinate{ CoordinateNullable: v13.CoordinateNullable{ID: &id,
+		Name: &nm,
+		Latitude: &la,
+		Longitude: &lo,
 		LastUpdated: &lu,
-	}
+	}}
 	expectedErrs = []error{}
-	errs = c.Validate(nil)
+	errs = c.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/crconfig/handler.go b/traffic_ops/traffic_ops_golang/crconfig/handler.go
index b1cafa1..646fe13 100644
--- a/traffic_ops/traffic_ops_golang/crconfig/handler.go
+++ b/traffic_ops/traffic_ops_golang/crconfig/handler.go
@@ -191,7 +191,7 @@ func SnapshotHandler(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
-		api.CreateChangeLogRaw(api.ApiChange, "Snapshot of CRConfig performed for "+cdn, *user, db.DB)
+		api.CreateChangeLogRaw(api.ApiChange, "Snapshot of CRConfig performed for "+cdn, user, db.DB)
 		w.WriteHeader(http.StatusOK) // TODO change to 204 No Content in new version
 	}
 }
@@ -236,7 +236,7 @@ func SnapshotOldGUIHandler(db *sqlx.DB, cfg config.Config) http.HandlerFunc {
 			writePerlHTMLErr(w, r, err)
 			return
 		}
-		api.CreateChangeLogRaw(api.ApiChange, "Snapshot of CRConfig performed for "+cdn, *user, db.DB)
+		api.CreateChangeLogRaw(api.ApiChange, "Snapshot of CRConfig performed for "+cdn, user, db.DB)
 		http.Redirect(w, r, "/tools/flash_and_close/"+url.PathEscape("Successfully wrote the CRConfig.json!"), http.StatusFound)
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index 09c4f3c..a783a90 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -126,7 +126,7 @@ 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, tx *sqlx.Tx) (bool, error) {
+func (ds *TODeliveryServiceV12) IsTenantAuthorized(user *auth.CurrentUser, tx *sqlx.Tx) (bool, error) {
 	return isTenantAuthorized(user, tx, &ds.DeliveryServiceNullableV12)
 }
 
@@ -144,7 +144,7 @@ func getTenantID(tx *sql.Tx, ds *tc.DeliveryServiceNullableV12) (*int, error) {
 	return existingID, err
 }
 
-func isTenantAuthorized(user auth.CurrentUser, tx *sqlx.Tx, ds *tc.DeliveryServiceNullableV12) (bool, error) {
+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())
@@ -197,7 +197,7 @@ func CreateV12() http.HandlerFunc {
 			return
 		}
 		dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
-		if authorized, err := isTenantAuthorized(*inf.User, inf.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 {
@@ -317,14 +317,14 @@ func UpdateV12() http.HandlerFunc {
 		return
 	}
 	dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
-	if authorized, err := isTenantAuthorized(*inf.User, inf.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
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 12d7312..994d498 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -133,7 +133,7 @@ func CreateV13() http.HandlerFunc {
 			return
 		}
 
-		if authorized, err := isTenantAuthorized(*inf.User, inf.Tx, &ds.DeliveryServiceNullableV12); err != nil {
+		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 {
@@ -240,7 +240,7 @@ func create(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds tc.Deliver
 	}
 
 	ds.LastUpdated = &lastUpdated
-	if err := api.CreateChangeLogRawErr(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), *user, tx); err!= nil {
+	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
@@ -410,7 +410,7 @@ func UpdateV13() http.HandlerFunc {
 			return
 		}
 
-		if authorized, err := isTenantAuthorized(*inf.User, inf.Tx, &ds.DeliveryServiceNullableV12); err != nil {
+		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 {
@@ -418,7 +418,7 @@ func UpdateV13() http.HandlerFunc {
 			return
 		}
 
-		ds, errCode, userErr, sysErr = update(inf.Tx.Tx, *inf.Config, *inf.User, &ds)
+		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
@@ -440,7 +440,7 @@ func getDSType(tx *sql.Tx, xmlid string) (tc.DSType, bool, error) {
 	return tc.DSTypeFromString(name), true, nil
 }
 
-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
 	}
@@ -560,7 +560,7 @@ func update(tx *sql.Tx, cfg config.Config, user auth.CurrentUser, ds *tc.Deliver
 		return tc.DeliveryServiceNullableV13{}, http.StatusInternalServerError, nil, errors.New("creating mid cacheurl parameters: " + err.Error())
 	}
 
-	if err := updatePrimaryOrigin(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())
 	}
 
@@ -579,11 +579,11 @@ func (ds *TODeliveryServiceV13) Delete() (error, tc.ApiErrorType) {
 }
 
 // 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) {
+func (ds *TODeliveryServiceV13) IsTenantAuthorized(user *auth.CurrentUser, tx *sqlx.Tx) (bool, error) {
 	return ds.V12().IsTenantAuthorized(user, tx)
 }
 
-func filterAuthorized(dses []tc.DeliveryServiceNullableV13, user auth.CurrentUser, db *sqlx.DB) ([]tc.DeliveryServiceNullableV13, error) {
+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
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 5824af1..50ca589 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -28,7 +28,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"
 
 	"github.com/go-ozzo/ozzo-validation"
@@ -38,13 +37,13 @@ import (
 
 //we need a type alias to define functions on
 type TODeliveryServiceRequestComment struct{
-	DB *sqlx.DB `json:"-"`
+	ReqInfo *api.APIInfo `json:"-"`
 	tc.DeliveryServiceRequestCommentNullable
 }
 
-func GetTypeSingleton(db *sqlx.DB) func()api.CRUDer {
-	return func()api.CRUDer {
-		toReturn := TODeliveryServiceRequestComment{db, tc.DeliveryServiceRequestCommentNullable{}}
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TODeliveryServiceRequestComment{reqInfo, tc.DeliveryServiceRequestCommentNullable{}}
 		return &toReturn
 	}
 }
@@ -77,7 +76,7 @@ func (comment TODeliveryServiceRequestComment) GetType() string {
 	return "deliveryservice_request_comment"
 }
 
-func (comment TODeliveryServiceRequestComment) Validate(db *sqlx.DB) []error {
+func (comment TODeliveryServiceRequestComment) Validate() []error {
 	errs := validation.Errors{
 		"deliveryServiceRequestId": validation.Validate(comment.DeliveryServiceRequestID, validation.NotNil),
 		"value":                    validation.Validate(comment.Value, validation.NotNil),
@@ -85,28 +84,11 @@ func (comment TODeliveryServiceRequestComment) Validate(db *sqlx.DB) []error {
 	return tovalidate.ToErrors(errs)
 }
 
-func (comment *TODeliveryServiceRequestComment) 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
-	}
-
-	userID := tc.IDNoMod(user.ID)
+func (comment *TODeliveryServiceRequestComment) Create() (error, tc.ApiErrorType) {
+	userID := tc.IDNoMod(comment.ReqInfo.User.ID)
 	comment.AuthorID = &userID
 
-	resultRows, err := tx.NamedQuery(insertQuery(), comment)
+	resultRows, err := comment.ReqInfo.Tx.NamedQuery(insertQuery(), comment)
 
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
@@ -143,16 +125,11 @@ func (comment *TODeliveryServiceRequestComment) Create(db *sqlx.DB, user auth.Cu
 	}
 	comment.SetKeys(map[string]interface{}{"id": id})
 	comment.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 (comment *TODeliveryServiceRequestComment) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (comment *TODeliveryServiceRequestComment) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 
 	// Query Parameters to Database Query column mappings
@@ -171,7 +148,7 @@ func (comment *TODeliveryServiceRequestComment) Read(db *sqlx.DB, parameters map
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := comment.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying delivery service request comments: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -191,38 +168,22 @@ func (comment *TODeliveryServiceRequestComment) Read(db *sqlx.DB, parameters map
 	return comments, []error{}, tc.NoError
 }
 
-func (comment *TODeliveryServiceRequestComment) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (comment *TODeliveryServiceRequestComment) Update() (error, tc.ApiErrorType) {
 
 	var current TODeliveryServiceRequestComment
-	err := db.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
+	err := comment.ReqInfo.Tx.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequestComments: %v", err)
 		return err, tc.SystemError
 	}
 
-	userID := tc.IDNoMod(user.ID)
+	userID := tc.IDNoMod(comment.ReqInfo.User.ID)
 	if *current.AuthorID != userID {
 		return errors.New("Comments can only be updated by the author"), 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
-	}
 	log.Debugf("about to run exec query: %s with comment: %++v", updateQuery(), comment)
-	resultRows, err := tx.NamedQuery(updateQuery(), comment)
+	resultRows, err := comment.ReqInfo.Tx.NamedQuery(updateQuery(), comment)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -255,47 +216,25 @@ func (comment *TODeliveryServiceRequestComment) Update(db *sqlx.DB, user auth.Cu
 			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
 }
 
-func (comment *TODeliveryServiceRequestComment) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (comment *TODeliveryServiceRequestComment) Delete() (error, tc.ApiErrorType) {
 
 	var current TODeliveryServiceRequestComment
-	err := db.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
+	err :=comment.ReqInfo.Tx.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequestComments: %v", err)
 		return err, tc.SystemError
 	}
 
-	userID := tc.IDNoMod(user.ID)
+	userID := tc.IDNoMod(comment.ReqInfo.User.ID)
 	if *current.AuthorID != userID {
 		return errors.New("Comments can only be deleted by the author"), 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
-	}
 	log.Debugf("about to run exec query: %s with comment: %++v", deleteQuery(), comment)
-	result, err := tx.NamedExec(deleteQuery(), comment)
+	result, err := comment.ReqInfo.Tx.NamedExec(deleteQuery(), comment)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -311,12 +250,6 @@ func (comment *TODeliveryServiceRequestComment) Delete(db *sqlx.DB, user auth.Cu
 			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
-	}
-	rollbackTransaction = false
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
index b785536..17b436c 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
@@ -27,6 +27,7 @@ import (
 
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/test"
+	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 func TestFuncs(t *testing.T) {
@@ -67,7 +68,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	c := TODeliveryServiceRequestComment{}
-	errs := test.SortErrors(c.Validate(nil))
+	errs := test.SortErrors(c.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'deliveryServiceRequestId' is required`),
@@ -80,9 +81,9 @@ func TestValidate(t *testing.T) {
 
 	v := "the comment value"
 	d := 1
-	c = TODeliveryServiceRequestComment{DeliveryServiceRequestID: &d, Value: &v}
+	c = TODeliveryServiceRequestComment{DeliveryServiceRequestCommentNullable: tc.DeliveryServiceRequestCommentNullable{DeliveryServiceRequestID: &d, Value: &v}}
 	expectedErrs = []error{}
-	errs = c.Validate(nil)
+	errs = c.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index 753127d..00a3d9d 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -122,7 +122,7 @@ func (req *TODeliveryServiceRequest) Read(parameters map[string]string) ([]inter
 		}
 
 		// TODO: combine tenancy with the query above so there's a single db call
-		t, err := s.IsTenantAuthorized(*req.ReqInfo.User)
+		t, err := s.IsTenantAuthorized(req.ReqInfo.User)
 		if err != nil {
 			log.Errorf("error checking tenancy: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
@@ -161,7 +161,7 @@ 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) (bool, error) {
+func (req TODeliveryServiceRequest) IsTenantAuthorized(user *auth.CurrentUser) (bool, error) {
 
 	ds := req.DeliveryService
 	if ds == 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 b4aa80e..b69b9b9 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
@@ -58,7 +58,7 @@ func TestGetDeliveryServiceRequest(t *testing.T) {
 	b := true
 	u := "UPDATE"
 	st := tc.RequestStatusSubmitted
-	r := &TODeliveryServiceRequest{
+	r := &TODeliveryServiceRequest{DeliveryServiceRequestNullable: tc.DeliveryServiceRequestNullable{
 		ChangeType: &u,
 		Status:     &st,
 		DeliveryService: &tc.DeliveryServiceNullableV13{
@@ -74,7 +74,7 @@ func TestGetDeliveryServiceRequest(t *testing.T) {
 				},
 			},
 		},
-	}
+	}}
 
 	expectedErrors := []string{
 		/*
diff --git a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
index 20e6485..b091ea9 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
@@ -72,7 +72,7 @@ JOIN type as rt ON r.type = rt.id
 				handleErrs(http.StatusInternalServerError, errors.New("querying: "+err.Error()))
 				return
 			}
-			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 				continue
 			} else if err != nil {
 				handleErrs(http.StatusInternalServerError, errors.New("checking tenancy: "+err.Error()))
@@ -153,7 +153,7 @@ ORDER BY dsr.set_number ASC
 				handleErrs(http.StatusInternalServerError, errors.New("querying: "+err.Error()))
 				return
 			}
-			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 				continue
 			} else if err != nil {
 				handleErrs(http.StatusInternalServerError, errors.New("checking tenancy: "+err.Error()))
@@ -245,7 +245,7 @@ ORDER BY dsr.set_number ASC
 				handleErrs(http.StatusInternalServerError, errors.New("querying: "+err.Error()))
 				return
 			}
-			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+			if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 				continue
 			} else if err != nil {
 				handleErrs(http.StatusInternalServerError, errors.New("checking tenancy: "+err.Error()))
@@ -301,7 +301,7 @@ func Post(dbx *sqlx.DB) http.HandlerFunc {
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
-		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 			handleErrs(http.StatusInternalServerError, errors.New("unauthorized"))
 			return
 		} else if err != nil {
@@ -403,7 +403,7 @@ func Put(dbx *sqlx.DB) http.HandlerFunc {
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
-		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 			handleErrs(http.StatusInternalServerError, errors.New("unauthorized"))
 			return
 		} else if err != nil {
@@ -525,7 +525,7 @@ func Delete(dbx *sqlx.DB) http.HandlerFunc {
 			handleErrs(http.StatusInternalServerError, err)
 			return
 		}
-		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, *user, dbx); !ok {
+		if ok, err := tenant.IsResourceAuthorizedToUser(dsTenantID, user, dbx); !ok {
 			handleErrs(http.StatusUnauthorized, errors.New("unauthorized"))
 			return
 		} else if err != nil {
@@ -557,7 +557,7 @@ func Delete(dbx *sqlx.DB) http.HandlerFunc {
 		}
 
 		log.Debugf("changelog for delete on object")
-		api.CreateChangeLogRaw(api.ApiChange,fmt.Sprintf(`deleted deliveryservice_regex {"ds": %d, "regex": %d}`, dsID, regexID), *user, dbx.DB)
+		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/division/divisions.go b/traffic_ops/traffic_ops_golang/division/divisions.go
index 70f864e..fa9d3a3 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions.go
@@ -29,22 +29,23 @@ 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"
-	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
 //we need a type alias to define functions on
-type TODivision tc.DivisionNullable
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TODivision(tc.DivisionNullable{})
+type TODivision struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.DivisionNullable
+}
 
-func GetRefType() *TODivision {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TODivision{reqInfo, tc.DivisionNullable{}}
+		return &toReturn
+	}
 }
 
 func (division TODivision) GetAuditName() string {
@@ -78,7 +79,7 @@ func (division TODivision) GetType() string {
 	return "division"
 }
 
-func (division TODivision) Validate(db *sqlx.DB) []error {
+func (division TODivision) Validate() []error {
 	errs := validation.Errors{
 		"name": validation.Validate(division.Name, validation.NotNil, validation.Required),
 	}
@@ -92,24 +93,8 @@ func (division TODivision) Validate(db *sqlx.DB) []error {
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted division and have
 //to be added to the struct
-func (division *TODivision) 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(), division)
+func (division *TODivision) Create() (error, tc.ApiErrorType) {
+	resultRows, err := division.ReqInfo.Tx.NamedQuery(insertQuery(), division)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -145,16 +130,10 @@ func (division *TODivision) Create(db *sqlx.DB, user auth.CurrentUser) (error, t
 	}
 	division.SetKeys(map[string]interface{}{"id": id})
 	division.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 (division *TODivision) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (division *TODivision) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	if strings.HasSuffix(parameters["name"], ".json") {
 		parameters["name"] = parameters["name"][:len(parameters["name"])-len(".json")]
 	}
@@ -172,7 +151,7 @@ func (division *TODivision) Read(db *sqlx.DB, parameters map[string]string, user
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := division.ReqInfo.Tx.NamedQuery(query, queryValues)
 	if err != nil {
 		log.Errorf("Error querying Divisions: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
@@ -197,25 +176,9 @@ func (division *TODivision) Read(db *sqlx.DB, parameters map[string]string, user
 //ParsePQUniqueConstraintError is used to determine if a division 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 (division *TODivision) 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 (division *TODivision) Update() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with division: %++v", updateQuery(), division)
-	resultRows, err := tx.NamedQuery(updateQuery(), division)
+	resultRows, err := division.ReqInfo.Tx.NamedQuery(updateQuery(), division)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -248,84 +211,14 @@ func (division *TODivision) Update(db *sqlx.DB, user auth.CurrentUser) (error, t
 			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
 }
 
-func getDivisionsResponse(params map[string]string, db *sqlx.DB) (*tc.DivisionsResponse, []error, tc.ApiErrorType) {
-	divisions, errs, errType := getDivisions(params, db)
-	if len(errs) > 0 {
-		return nil, errs, errType
-	}
-
-	resp := tc.DivisionsResponse{
-		Response: divisions,
-	}
-	return &resp, nil, tc.NoError
-}
-
-func getDivisions(params map[string]string, db *sqlx.DB) ([]tc.Division, []error, tc.ApiErrorType) {
-	var rows *sqlx.Rows
-	var err error
-
-	// Query Parameters to Database Query column mappings
-	// see the fields mapped in the SQL query
-	queryParamsToSQLCols := map[string]dbhelpers.WhereColumnInfo{
-		"id":   dbhelpers.WhereColumnInfo{"id", api.IsInt},
-		"name": dbhelpers.WhereColumnInfo{"name", nil},
-	}
-
-	where, orderBy, queryValues, errs := dbhelpers.BuildWhereAndOrderBy(params, queryParamsToSQLCols)
-	if len(errs) > 0 {
-		return nil, errs, tc.DataConflictError
-	}
-
-	query := selectQuery() + where + orderBy
-	log.Debugln("Query is ", query)
-
-	rows, err = db.NamedQuery(query, queryValues)
-	if err != nil {
-		return nil, []error{err}, tc.SystemError
-	}
-	defer rows.Close()
-
-	o := []tc.Division{}
-	for rows.Next() {
-		var d tc.Division
-		if err = rows.StructScan(&d); err != nil {
-			return nil, []error{fmt.Errorf("getting divisions: %v", err)}, tc.SystemError
-		}
-		o = append(o, d)
-	}
-	return o, nil, tc.NoError
-}
-
 //The Division implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (division *TODivision) 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 (division *TODivision) Delete() (error, tc.ApiErrorType) {
 	log.Debugf("about to run exec query: %s with division: %++v", deleteQuery(), division)
-	result, err := tx.NamedExec(deleteQuery(), division)
+	result, err := division.ReqInfo.Tx.NamedExec(deleteQuery(), division)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -341,12 +234,6 @@ func (division *TODivision) Delete(db *sqlx.DB, user auth.CurrentUser) (error, t
 			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/division/divisions_test.go b/traffic_ops/traffic_ops_golang/division/divisions_test.go
index ce2e99d..eedcbba 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions_test.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions_test.go
@@ -25,6 +25,7 @@ import (
 	"time"
 
 	"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/test"
 	"github.com/jmoiron/sqlx"
@@ -69,10 +70,13 @@ func TestGetDivisions(t *testing.T) {
 			ts.Name,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
+	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	servers, errs, errType := getDivisions(v, db)
+	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	servers, errs, errType := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("getDivisions expected: no errors, actual: %v with error type: %s", errs, errType.String())
 	}
@@ -106,7 +110,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidation(t *testing.T) {
 	div := TODivision{}
-	errs := test.SortErrors(div.Validate(nil))
+	errs := test.SortErrors(div.Validate())
 	expected := []error{}
 
 	if reflect.DeepEqual(expected, errs) {
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index a9b9ee7..73f4a4a 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -41,13 +41,16 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOOrigin v13.Origin
-
-//the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOOrigin{}
+type TOOrigin struct{
+	ReqInfo *api.APIInfo `json:"-"`
+	v13.Origin
+}
 
-func GetRefType() *TOOrigin {
-	return &refType
+func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
+	return func(reqInfo *api.APIInfo)api.CRUDer {
+		toReturn := TOOrigin{reqInfo, v13.Origin{}}
+		return &toReturn
+	}
 }
 
 func (origin *TOOrigin) SetID(i int) {
@@ -85,7 +88,7 @@ func (origin *TOOrigin) GetType() string {
 	return "origin"
 }
 
-func (origin *TOOrigin) Validate(db *sqlx.DB) []error {
+func (origin *TOOrigin) Validate() []error {
 
 	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
 	validProtocol := validation.NewStringRule(tovalidate.IsOneOfStringICase("http", "https"), "must be http or https")
@@ -108,10 +111,10 @@ func (origin *TOOrigin) Validate(db *sqlx.DB) []error {
 }
 
 // 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) {
+func (origin *TOOrigin) GetTenantID(tx *sqlx.Tx) (*int, error) {
 	if origin.ID != nil {
 		var tenantID *int
-		if err := db.QueryRow(`SELECT tenant FROM origin where id = $1`, *origin.ID).Scan(&tenantID); err != nil {
+		if err := tx.QueryRow(`SELECT tenant FROM origin where id = $1`, *origin.ID).Scan(&tenantID); err != nil {
 			if err == sql.ErrNoRows {
 				return nil, nil
 			}
@@ -122,21 +125,21 @@ func (origin *TOOrigin) GetTenantID(db *sqlx.DB) (*int, error) {
 	return nil, nil
 }
 
-func (origin *TOOrigin) IsTenantAuthorized(user auth.CurrentUser, db *sqlx.DB) (bool, error) {
-	currentTenantID, err := origin.GetTenantID(db)
+func (origin *TOOrigin) IsTenantAuthorized(user *auth.CurrentUser) (bool, error) {
+	currentTenantID, err := origin.GetTenantID(origin.ReqInfo.Tx)
 	if err != nil {
 		return false, err
 	}
 
-	if currentTenantID != nil && tenant.IsTenancyEnabled(db) {
-		return tenant.IsResourceAuthorizedToUser(*currentTenantID, user, db)
+	if currentTenantID != nil && tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx) {
+		return tenant.IsResourceAuthorizedToUserTx(*currentTenantID, user, origin.ReqInfo.Tx.Tx)
 	}
 
 	return true, nil
 }
 
 // filterAuthorized will filter a slice of Origins based upon tenant. It assumes that tenancy is enabled
-func filterAuthorized(origins []v13.Origin, user auth.CurrentUser, db *sqlx.DB) ([]v13.Origin, error) {
+func filterAuthorized(origins []v13.Origin, user *auth.CurrentUser, tx *sqlx.Tx) ([]v13.Origin, error) {
 	newOrigins := []v13.Origin{}
 	for _, origin := range origins {
 		if origin.TenantID == nil {
@@ -147,7 +150,7 @@ func filterAuthorized(origins []v13.Origin, user auth.CurrentUser, db *sqlx.DB)
 			}
 		}
 		// TODO add/use a helper func to make a single SQL call, for performance
-		ok, err := tenant.IsResourceAuthorizedToUser(*origin.TenantID, user, db)
+		ok, err := tenant.IsResourceAuthorizedToUserTx(*origin.TenantID, user, tx.Tx)
 		if err != nil {
 			if origin.ID == nil {
 				return nil, errors.New("isResourceAuthorized for origin with nil ID: " + err.Error())
@@ -163,19 +166,19 @@ func filterAuthorized(origins []v13.Origin, user auth.CurrentUser, db *sqlx.DB)
 	return newOrigins, nil
 }
 
-func (origin *TOOrigin) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+func (origin *TOOrigin) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
 
-	privLevel := user.PrivLevel
+	privLevel := origin.ReqInfo.User.PrivLevel
 
-	origins, errs, errType := getOrigins(params, db, privLevel)
+	origins, errs, errType := getOrigins(params, origin.ReqInfo.Tx, privLevel)
 	if len(errs) > 0 {
 		return nil, errs, errType
 	}
 
 	var err error
-	if tenant.IsTenancyEnabled(db) {
-		origins, err = filterAuthorized(origins, user, db)
+	if tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx) {
+		origins, err = filterAuthorized(origins, origin.ReqInfo.User, origin.ReqInfo.Tx)
 		if err != nil {
 			log.Errorln("Checking tenancy: " + err.Error())
 			return nil, []error{errors.New("Error checking tenancy.")}, tc.SystemError
@@ -189,7 +192,7 @@ func (origin *TOOrigin) Read(db *sqlx.DB, params map[string]string, user auth.Cu
 	return returnable, nil, tc.NoError
 }
 
-func getOrigins(params map[string]string, db *sqlx.DB, privLevel int) ([]v13.Origin, []error, tc.ApiErrorType) {
+func getOrigins(params map[string]string, tx *sqlx.Tx, privLevel int) ([]v13.Origin, []error, tc.ApiErrorType) {
 	var rows *sqlx.Rows
 	var err error
 
@@ -214,7 +217,7 @@ func getOrigins(params map[string]string, db *sqlx.DB, privLevel int) ([]v13.Ori
 	query := selectQuery() + where + orderBy
 	log.Debugln("Query is ", query)
 
-	rows, err = db.NamedQuery(query, queryValues)
+	rows, err = tx.NamedQuery(query, queryValues)
 	if err != nil {
 		return nil, []error{fmt.Errorf("querying: %v", err)}, tc.SystemError
 	}
@@ -266,12 +269,12 @@ LEFT JOIN tenant t ON o.tenant = t.id`
 	return selectStmt
 }
 
-func checkTenancy(originTenantID, deliveryserviceID *int, db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
-	if tenant.IsTenancyEnabled(db) {
+func checkTenancy(originTenantID, deliveryserviceID *int, tx *sqlx.Tx, user *auth.CurrentUser) (error, tc.ApiErrorType) {
+	if tenant.IsTenancyEnabledTx(tx) {
 		if originTenantID == nil {
 			return tc.NilTenantError, tc.ForbiddenError
 		}
-		authorized, err := tenant.IsResourceAuthorizedToUser(*originTenantID, user, db)
+		authorized, err := tenant.IsResourceAuthorizedToUserTx(*originTenantID, user, tx.Tx)
 		if err != nil {
 			return err, tc.SystemError
 		}
@@ -281,7 +284,7 @@ func checkTenancy(originTenantID, deliveryserviceID *int, db *sqlx.DB, user auth
 
 		if deliveryserviceID != nil {
 			var deliveryserviceTenantID *int
-			if err := db.QueryRow(`SELECT tenant_id FROM deliveryservice where id = $1`, *deliveryserviceID).Scan(&deliveryserviceTenantID); err != nil {
+			if err := tx.QueryRow(`SELECT tenant_id FROM deliveryservice where id = $1`, *deliveryserviceID).Scan(&deliveryserviceTenantID); err != nil {
 				if err == sql.ErrNoRows {
 					return errors.New("checking tenancy: requested delivery service does not exist"), tc.DataConflictError
 				}
@@ -289,7 +292,7 @@ func checkTenancy(originTenantID, deliveryserviceID *int, db *sqlx.DB, user auth
 				return err, tc.SystemError
 			}
 			if deliveryserviceTenantID != nil {
-				authorized, err := tenant.IsResourceAuthorizedToUser(*deliveryserviceTenantID, user, db)
+				authorized, err := tenant.IsResourceAuthorizedToUserTx(*deliveryserviceTenantID, user, tx.Tx)
 				if err != nil {
 					return err, tc.SystemError
 				}
@@ -307,34 +310,17 @@ func checkTenancy(originTenantID, deliveryserviceID *int, db *sqlx.DB, user auth
 //ParsePQUniqueConstraintError is used to determine if an origin 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 (origin *TOOrigin) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (origin *TOOrigin) Update() (error, tc.ApiErrorType) {
 	// TODO: enhance tenancy framework to handle this in isTenantAuthorized()
-	err, errType := checkTenancy(origin.TenantID, origin.DeliveryServiceID, db, user)
+	err, errType := checkTenancy(origin.TenantID, origin.DeliveryServiceID, origin.ReqInfo.Tx, origin.ReqInfo.User)
 	if err != nil {
 		return err, errType
 	}
 
-	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
-	}
-
 	isPrimary := false
 	ds := 0
 	q := `SELECT is_primary, deliveryservice FROM origin WHERE id = $1`
-	if err := tx.QueryRow(q, *origin.ID).Scan(&isPrimary, &ds); err != nil {
+	if err := origin.ReqInfo.Tx.QueryRow(q, *origin.ID).Scan(&isPrimary, &ds); err != nil {
 		log.Errorf("updating origin %d, received error in select: %v", *origin.ID, err)
 		return tc.DBError, tc.SystemError
 	}
@@ -343,7 +329,7 @@ func (origin *TOOrigin) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 
 	log.Debugf("about to run exec query: %s with origin: %++v", updateQuery(), origin)
-	resultRows, err := tx.NamedQuery(updateQuery(), origin)
+	resultRows, err := origin.ReqInfo.Tx.NamedQuery(updateQuery(), origin)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -378,12 +364,6 @@ func (origin *TOOrigin) Update(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 		return tc.DBError, tc.SystemError
 	}
 	origin.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
 }
 
@@ -412,31 +392,14 @@ WHERE id=:id RETURNING last_updated`
 //generic error message returned
 //The insert sql returns the id and lastUpdated values of the newly inserted origin and have
 //to be added to the struct
-func (origin *TOOrigin) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
+func (origin *TOOrigin) Create() (error, tc.ApiErrorType) {
 	// TODO: enhance tenancy framework to handle this in isTenantAuthorized()
-	err, errType := checkTenancy(origin.TenantID, origin.DeliveryServiceID, db, user)
+	err, errType := checkTenancy(origin.TenantID, origin.DeliveryServiceID, origin.ReqInfo.Tx, origin.ReqInfo.User)
 	if err != nil {
 		return err, errType
 	}
 
-	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(), origin)
+	resultRows, err := origin.ReqInfo.Tx.NamedQuery(insertQuery(), origin)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -472,12 +435,7 @@ func (origin *TOOrigin) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 	origin.SetKeys(map[string]interface{}{"id": id})
 	origin.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
 }
 
@@ -510,27 +468,10 @@ tenant) VALUES (
 
 //The Origin implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
-func (origin *TOOrigin) 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 (origin *TOOrigin) Delete() (error, tc.ApiErrorType) {
 	isPrimary := false
 	q := `SELECT is_primary FROM origin WHERE id = $1`
-	if err := tx.QueryRow(q, *origin.ID).Scan(&isPrimary); err != nil {
+	if err := origin.ReqInfo.Tx.QueryRow(q, *origin.ID).Scan(&isPrimary); err != nil {
 		log.Errorf("deleting origin %d, received error selecting is_primary: %v", *origin.ID, err)
 		return tc.DBError, tc.SystemError
 	}
@@ -539,7 +480,7 @@ func (origin *TOOrigin) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 	}
 
 	log.Debugf("about to run exec query: %s with origin: %++v", deleteQuery(), origin)
-	result, err := tx.NamedExec(deleteQuery(), origin)
+	result, err := origin.ReqInfo.Tx.NamedExec(deleteQuery(), origin)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -555,12 +496,7 @@ func (origin *TOOrigin) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.Ap
 			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
-	}
-	rollbackTransaction = false
+
 	return nil, tc.NoError
 }
 
diff --git a/traffic_ops/traffic_ops_golang/origin/origins_test.go b/traffic_ops/traffic_ops_golang/origin/origins_test.go
index 7036171..ed5db33 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins_test.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins_test.go
@@ -113,10 +113,11 @@ func TestReadOrigins(t *testing.T) {
 			to.TenantID,
 		)
 	}
+	mock.ExpectBegin()
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
 	v := map[string]string{}
 
-	origins, errs, errType := getOrigins(v, db, auth.PrivLevelAdmin)
+	origins, errs, errType := getOrigins(v, db.MustBegin(), auth.PrivLevelAdmin)
 	log.Debugln("%v-->", origins)
 	if len(errs) > 0 {
 		t.Errorf("getOrigins expected: no errors, actual: %v with error type: %s", errs, errType.String())
@@ -175,13 +176,13 @@ func TestValidate(t *testing.T) {
 	const ip6Err = `'ip6Address' must be a valid IPv6 address`
 
 	// verify that non-null fields are invalid
-	c := TOOrigin{ID: nil,
+	c := TOOrigin{Origin: v13.Origin{ID: nil,
 		Name:              nil,
 		DeliveryServiceID: nil,
 		FQDN:              nil,
 		Protocol:          nil,
-	}
-	errs := test.SortErrors(c.Validate(nil))
+	}}
+	errs := test.SortErrors(c.Validate())
 
 	expectedErrs := []error{
 		errors.New(`'deliveryServiceId' is required`),
@@ -203,7 +204,7 @@ func TestValidate(t *testing.T) {
 	port := 65535
 	pro := "http"
 	lu := tc.TimeNoMod{Time: time.Now()}
-	c = TOOrigin{ID: &id,
+	c = TOOrigin{Origin:v13.Origin{ID: &id,
 		Name:              &nm,
 		DeliveryServiceID: &id,
 		FQDN:              &fqdn,
@@ -212,9 +213,9 @@ func TestValidate(t *testing.T) {
 		Port:              &port,
 		Protocol:          &pro,
 		LastUpdated:       &lu,
-	}
+	}}
 	expectedErrs = []error{}
-	errs = c.Validate(nil)
+	errs = c.Validate()
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
 	}
@@ -319,7 +320,7 @@ func TestValidate(t *testing.T) {
 				c.IP6Address = &tc.Str
 				value = tc.Str
 			}
-			errs = test.SortErrors(c.Validate(nil))
+			errs = test.SortErrors(c.Validate())
 			if !reflect.DeepEqual(tc.ExpectedErrors, errs) {
 				t.Errorf("given: '%v', expected %s, got %s", value, tc.ExpectedErrors, errs)
 			}
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 67fe91c..ed93d74 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -36,16 +36,16 @@ 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/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"
@@ -93,11 +93,11 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 
 		//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.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.MethodGet, `cachegroups/?(\.json)?$`, api.ReadHandler(cachegroup.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `cachegroups/{id}$`, api.ReadHandler(cachegroup.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `cachegroups/{id}$`, api.UpdateHandler(cachegroup.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `cachegroups/?$`, api.CreateHandler(cachegroup.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `cachegroups/{id}$`, api.DeleteHandler(cachegroup.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		{1.1, http.MethodPost, `cachegroups/{id}/queue_update$`, cachegroup.QueueUpdates(d.DB.DB), auth.PrivLevelOperations, Authenticated, nil},
 
@@ -125,12 +125,12 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `divisions/{id}$`, api.ReadHandler(division.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodPut, `divisions/{id}$`, api.UpdateHandler(division.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodPost, `divisions/?$`, api.CreateHandler(division.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodDelete, `divisions/{id}$`, api.DeleteHandler(division.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.1, http.MethodGet, `divisions/name/{name}/?(\.json)?$`, api.ReadHandler(division.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//HWInfo
 		{1.1, http.MethodGet, `hwinfo-wip/?(\.json)?$`, hwinfo.HWInfoHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
@@ -220,11 +220,11 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{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.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},
+		{1.3, http.MethodGet, `coordinates/?(\.json)?$`, api.ReadHandler(coordinate.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `coordinates/?$`, api.ReadHandler(coordinate.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `coordinates/?$`, api.UpdateHandler(coordinate.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPost, `coordinates/?$`, api.CreateHandler(coordinate.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodDelete, `coordinates/?$`, api.DeleteHandler(coordinate.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Servers
 		// explicitly passed to legacy system until fully implemented.  Auth handled by legacy system.
@@ -261,10 +261,10 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{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.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservice_request_comments/?$`, api.UpdateHandler(comment.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
+		{1.3, http.MethodPost, `deliveryservice_request_comments/?$`, api.CreateHandler(comment.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
+		{1.3, http.MethodDelete, `deliveryservice_request_comments/?$`, api.DeleteHandler(comment.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
 
 		//Delivery service uri signing keys: CRUD
 		{1.3, http.MethodGet, `deliveryservices/{xmlID}/urisignkeys$`, getURIsignkeysHandler, auth.PrivLevelAdmin, Authenticated, nil},
@@ -273,11 +273,11 @@ 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.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},
+		{1.3, http.MethodGet, `origins/?(\.json)?$`, api.ReadHandler(origin.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodGet, `origins/?$`, api.ReadHandler(origin.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `origins/?$`, api.UpdateHandler(origin.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodPost, `origins/?$`, api.CreateHandler(origin.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
+		{1.3, http.MethodDelete, `origins/?$`, api.DeleteHandler(origin.GetTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},
 
 		//Roles
 		//{1.3, http.MethodGet, `roles/?(\.json)?$`, api.ReadHandler(role.GetRefType(), d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index abd3209..fad4cdc 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
@@ -233,7 +233,7 @@ func IsTenancyEnabledTx(tx *sql.Tx) (bool, error) {
 
 // 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 IsResourceAuthorizedToUser(resourceTenantID int, user auth.CurrentUser, db *sqlx.DB) (bool, error) {
+func IsResourceAuthorizedToUser(resourceTenantID int, user *auth.CurrentUser, db *sqlx.DB) (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),
@@ -264,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),


[trafficcontrol] 07/15: fix tenants test by resetting id_seq and fixing error message

Posted by da...@apache.org.
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 6056ee0ffef1ed508195edd7f87caa8f87700f2f
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 23:41:20 2018 -0600

    fix tenants test by resetting id_seq and fixing error message
---
 traffic_ops/testing/api/v13/tenants_test.go | 3 +--
 traffic_ops/testing/api/v13/todb.go         | 5 +++--
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/traffic_ops/testing/api/v13/tenants_test.go b/traffic_ops/testing/api/v13/tenants_test.go
index 99bbbf2..2a4003a 100644
--- a/traffic_ops/testing/api/v13/tenants_test.go
+++ b/traffic_ops/testing/api/v13/tenants_test.go
@@ -22,7 +22,6 @@ import (
 )
 
 func TestTenants(t *testing.T) {
-
 	CreateTestTenants(t)
 	UpdateTestTenants(t)
 	GetTestTenants(t)
@@ -120,7 +119,7 @@ func DeleteTestTenants(t *testing.T) {
 	if err == nil {
 		t.Errorf("%s has child tenants -- should not be able to delete", t1)
 	}
-	expected := "Tenant 'tenant1' has child tenants"
+	expected := `Tenant '`+ strconv.Itoa(tenant1.ID) + `' has child tenants. Please update these child tenants and retry.`
 	if !strings.Contains(err.Error(), expected) {
 		t.Errorf("expected error: %s;  got %s", expected, err.Error())
 	}
diff --git a/traffic_ops/testing/api/v13/todb.go b/traffic_ops/testing/api/v13/todb.go
index 0ebea91..4e25ec2 100644
--- a/traffic_ops/testing/api/v13/todb.go
+++ b/traffic_ops/testing/api/v13/todb.go
@@ -156,8 +156,8 @@ func SetupTenants(db *sql.DB) error {
 	// TODO: root tenant must be present in initial database.  "badtenant" is needed for now so tests can be done
 	// with a tenant outside the user's tenant.  That should be removed once User API tests are in place rather than the SetupUsers defined above.
 	sqlStmt := `
-INSERT INTO tenant (id, name, active, parent_id, last_updated) VALUES (1, 'root', true, null, '2018-01-19 19:01:21.327262');
-INSERT INTO tenant (id, name, active, parent_id, last_updated) VALUES (2, 'badtenant', true, 1, '2018-01-19 19:01:21.327262');
+INSERT INTO tenant (name, active, parent_id, last_updated) VALUES ('root', true, null, '2018-01-19 19:01:21.327262');
+INSERT INTO tenant (name, active, parent_id, last_updated) VALUES ('badtenant', true, 1, '2018-01-19 19:01:21.327262');
 `
 	err := execSQL(db, sqlStmt, "tenant")
 	if err != nil {
@@ -363,6 +363,7 @@ func Teardown(db *sql.DB) error {
 	DELETE FROM snapshot;
 	DELETE FROM cdn;
 	DELETE FROM tenant;
+	ALTER SEQUENCE tenant_id_seq RESTART WITH 1;
 `
 	err := execSQL(db, sqlStmt, "Tearing down")
 	if err != nil {


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

Posted by da...@apache.org.
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
 }
+


[trafficcontrol] 04/15: fix cachegroup where clause info to match select statement

Posted by da...@apache.org.
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 4d980d931cd308beb85b707afcc3cd097bdd9ffe
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 19:58:51 2018 -0600

    fix cachegroup where clause info to match select statement
---
 traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index 03da6d2..784e15f 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -289,10 +289,10 @@ func (cg *TOCacheGroup) Read(parameters map[string]string) ([]interface{}, []err
 	// Query Parameters to Database Query column mappings
 	// see the fields mapped in the SQL query
 	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
-		"id":        dbhelpers.WhereColumnInfo{"cg.id", api.IsInt},
-		"name":      dbhelpers.WhereColumnInfo{"cg.name", nil},
+		"id":        dbhelpers.WhereColumnInfo{"cachegroup.id", api.IsInt},
+		"name":      dbhelpers.WhereColumnInfo{"cachegroup.name", nil},
 		"shortName": dbhelpers.WhereColumnInfo{"short_name", nil},
-		"type":      dbhelpers.WhereColumnInfo{"cg.type", nil},
+		"type":      dbhelpers.WhereColumnInfo{"cachegroup.type", nil},
 	}
 	where, orderBy, queryValues, errs := dbhelpers.BuildWhereAndOrderBy(parameters, queryParamsToQueryCols)
 	if len(errs) > 0 {


[trafficcontrol] 13/15: documentation cleanup and code simplifications

Posted by da...@apache.org.
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 2c495a9a05f20aa99604f3394f13e777233db64d
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Mon Jul 2 07:31:42 2018 -0600

    documentation cleanup and code simplifications
---
 .../traffic_ops_golang/api/shared_handlers.go      | 29 +++-------
 traffic_ops/traffic_ops_golang/asn/asns.go         | 66 +++++++++++-----------
 traffic_ops/traffic_ops_golang/asn/asns_test.go    |  8 +--
 .../deliveryservice/deliveryservicesv12.go         |  6 +-
 .../deliveryservice/deliveryservicesv13.go         | 18 ++----
 .../deliveryservice/request/requests.go            |  3 +-
 traffic_ops/traffic_ops_golang/routes.go           |  2 +-
 7 files changed, 54 insertions(+), 78 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index b2b6465..9beee6e 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -102,10 +102,7 @@ func GetCombinedParams(r *http.Request) (map[string]string, error) {
 	return combinedParams, nil
 }
 
-//decodes and validates a pointer to a struct implementing the Validator interface
-//      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
+// decodeAndValidateRequestBody decodes and validates a pointer to a struct implementing the Validator interface
 func decodeAndValidateRequestBody(r *http.Request, v Validator) error {
 	defer r.Body.Close()
 
@@ -115,7 +112,7 @@ func decodeAndValidateRequestBody(r *http.Request, v Validator) error {
 	return v.Validate()
 }
 
-//this creates a handler function from the pointer to a struct implementing the Reader interface
+// ReadHandler creates a handler function from the pointer to a struct implementing the Reader interface
 //      this handler retrieves the user from the context
 //      combines the path and query parameters
 //      produces the proper status code based on the error code returned
@@ -162,7 +159,7 @@ func ReadHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	}
 }
 
-//this creates a handler function from the pointer to a struct implementing the Reader interface
+// ReadOnlyHandler creates a handler function from the pointer to a struct implementing the Reader interface
 //      this handler retrieves the user from the context
 //      combines the path and query parameters
 //      produces the proper status code based on the error code returned
@@ -208,8 +205,7 @@ func ReadOnlyHandler(typeFactory func(reqInfo *APIInfo) Reader) http.HandlerFunc
 	}
 }
 
-//this creates a handler function from the pointer to a struct implementing the Updater interface
-//it must be immediately assigned to a local variable
+// UpdateHandler creates a handler function from the pointer to a struct implementing the Updater interface
 //   this generic handler encapsulates the logic for handling:
 //   *fetching the id from the path parameter
 //   *current user
@@ -245,8 +241,6 @@ func UpdateHandler(typeFactory CRUDFactory) http.HandlerFunc {
 			return
 		}
 
-		//create local instance of the shared typeRef pointer
-		//no operations should be made on the typeRef
 		//decode the body and validate the request struct
 		err = decodeAndValidateRequestBody(r, u)
 		if err != nil {
@@ -302,8 +296,7 @@ func UpdateHandler(typeFactory CRUDFactory) http.HandlerFunc {
 			return
 		}
 		//auditing here
-		err = CreateChangeLog(ApiChange, Updated, u, inf.User, inf.Tx)
-		if err != nil {
+		if err := CreateChangeLog(ApiChange, Updated, u, inf.User, inf.Tx); err != nil {
 			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
@@ -325,8 +318,7 @@ func UpdateHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	}
 }
 
-//this creates a handler function from the pointer to a struct implementing the Deleter interface
-//it must be immediately assigned to a local variable
+// DeleteHandler creates a handler function from the pointer to a struct implementing the Deleter interface
 //   this generic handler encapsulates the logic for handling:
 //   *fetching the id from the path parameter
 //   *current user
@@ -392,8 +384,7 @@ func DeleteHandler(typeFactory CRUDFactory) http.HandlerFunc {
 		}
 		//audit here
 		log.Debugf("changelog for delete on object")
-		err = CreateChangeLog(ApiChange, Deleted, d, inf.User, inf.Tx)
-		if err != nil {
+		if err = CreateChangeLog(ApiChange, Deleted, d, inf.User, inf.Tx); err != nil {
 			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
@@ -414,8 +405,7 @@ func DeleteHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	}
 }
 
-//this creates a handler function from the pointer to a struct implementing the Creator interface
-//it must be immediately assigned to a local variable
+// CreateHandler creates a handler function from the pointer to a struct implementing the Creator interface
 //   this generic handler encapsulates the logic for handling:
 //   *fetching the id from the path parameter
 //   *current user
@@ -464,8 +454,7 @@ func CreateHandler(typeConstructor CRUDFactory) http.HandlerFunc {
 			return
 		}
 
-		err = CreateChangeLog(ApiChange, Created, i, inf.User, inf.Tx)
-		if err != nil {
+		if err = CreateChangeLog(ApiChange, Created, i, inf.User, inf.Tx); err != nil {
 			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index 3e790a7..f2a9657 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -159,45 +159,43 @@ func (asn *TOASNV11) Read(parameters map[string]string) ([]interface{}, []error,
 }
 
 // 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() http.HandlerFunc {
-	return func(w http.ResponseWriter, r *http.Request) {
-		handleErrs := tc.GetHandleErrorsFunc(w, r)
-
-		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()
+func V11ReadAll(w http.ResponseWriter, r *http.Request) {
+	handleErrs := tc.GetHandleErrorsFunc(w, r)
 
-		params, err := api.GetCombinedParams(r)
-		if err != nil {
-			handleErrs(http.StatusInternalServerError, err)
-			return
-		}
+	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()
 
-		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 []tc.ASNNullable `json:"asns"`
-			} `json:"response"`
-		}{Response: struct {
+	params, err := api.GetCombinedParams(r)
+	if err != nil {
+		handleErrs(http.StatusInternalServerError, err)
+		return
+	}
+
+	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 []tc.ASNNullable `json:"asns"`
-		}{ASNs: asns}}
+		} `json:"response"`
+	}{Response: struct {
+		ASNs []tc.ASNNullable `json:"asns"`
+	}{ASNs: asns}}
 
-		respBts, err := json.Marshal(resp)
-		if err != nil {
-			handleErrs(http.StatusInternalServerError, err)
-			return
-		}
-		w.Header().Set("Content-Type", "application/json")
-		fmt.Fprintf(w, "%s", respBts)
+	respBts, err := json.Marshal(resp)
+	if err != nil {
+		handleErrs(http.StatusInternalServerError, err)
+		return
 	}
+	w.Header().Set("Content-Type", "application/json")
+	fmt.Fprintf(w, "%s", respBts)
 }
 
 func read(tx *sqlx.Tx, parameters map[string]string) ([]tc.ASNNullable, []error, tc.ApiErrorType) {
diff --git a/traffic_ops/traffic_ops_golang/asn/asns_test.go b/traffic_ops/traffic_ops_golang/asn/asns_test.go
index 3c1c159..00963f0 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns_test.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns_test.go
@@ -35,7 +35,7 @@ import (
 )
 
 func getTestASNs() []tc.ASNNullable {
-	ASNs := []tc.ASNNullable{}
+	asns := []tc.ASNNullable{}
 	i := 1
 	c := "Yukon"
 	testCase := tc.ASNNullable{
@@ -45,13 +45,13 @@ func getTestASNs() []tc.ASNNullable {
 		ID:           &i,
 		LastUpdated:  &tc.TimeNoMod{Time: time.Now()},
 	}
-	ASNs = append(ASNs, testCase)
+	asns = append(asns, testCase)
 
 	testCase2 := testCase
 	*testCase2.ASN = 2
-	ASNs = append(ASNs, testCase2)
+	asns = append(asns, testCase2)
 
-	return ASNs
+	return asns
 }
 
 func TestGetASNs(t *testing.T) {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index e1876bb..a84a14f 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -174,7 +174,7 @@ func (ds *TODeliveryServiceV12) Validate() error {
 	return ds.DeliveryServiceNullableV12.Validate(ds.ReqInfo.Tx.Tx)
 }
 
-// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow a create to return an array of one
+// Create is 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
 }
@@ -228,7 +228,7 @@ func (ds *TODeliveryServiceV12) Read(params map[string]string) ([]interface{}, [
 	return returnable, nil, tc.NoError
 }
 
-//The DeliveryService implementation of the Deleter interface
+//Delete is 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)
@@ -291,7 +291,7 @@ func (ds *TODeliveryServiceV12) Delete() (error, tc.ApiErrorType) {
 	return nil, tc.NoError
 }
 
-// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow an update to return an array of one
+// Update is 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
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index d7b0f7a..efc385d 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -93,20 +93,11 @@ func (ds *TODeliveryServiceV13) Validate() error {
 	return ds.DeliveryServiceNullableV13.Validate(ds.ReqInfo.Tx.Tx)
 }
 
-// unimplemented, needed to satisfy CRUDer, since the framework doesn't allow a create to return an array of one
+// Create is 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) {
@@ -148,8 +139,7 @@ func CreateV13() http.HandlerFunc {
 	}
 }
 
-// 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.
-
+// 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 code, 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) {
 	// TODO change DeepCachingType to implement sql.Valuer and sql.Scanner, so sqlx struct scan can be used.
 	deepCachingType := tc.DeepCachingType("").String()
@@ -377,7 +367,7 @@ 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
+// Update is 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
 }
@@ -567,7 +557,7 @@ func update(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds *tc.Delive
 	return *ds, http.StatusOK, nil, nil
 }
 
-//The DeliveryService implementation of the Deleter interface
+// Delete is the DeliveryService implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
 func (ds *TODeliveryServiceV13) Delete() (error, tc.ApiErrorType) {
 	return ds.V12().Delete()
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index 9dccd08..f367480 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -35,7 +35,7 @@ import (
 	"github.com/lib/pq"
 )
 
-//we need a type alias to define functions on
+// TODeliveryServiceRequest is the type alias to define functions on
 type TODeliveryServiceRequest struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.DeliveryServiceRequestNullable
@@ -52,7 +52,6 @@ func (req TODeliveryServiceRequest) GetKeyFieldsInfo() []api.KeyFieldInfo {
 	return []api.KeyFieldInfo{{"id", api.GetIntKey}}
 }
 
-//Implementation of the Identifier, Validator interface functions
 func (req TODeliveryServiceRequest) GetKeys() (map[string]interface{}, bool) {
 	if req.ID == nil {
 		return map[string]interface{}{"id": 0}, false
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index d26503d..9b1d696 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -85,7 +85,7 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 
 		//ASN: CRUD
 		{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/?(\.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},


[trafficcontrol] 11/15: define CRUDFactory function type to cleanup signatures

Posted by da...@apache.org.
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 0ad2e84d8debbf127f9e3dca4b60fb41b39cd645
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Wed Jun 27 14:50:41 2018 -0600

    define CRUDFactory function type to cleanup signatures
---
 traffic_ops/traffic_ops_golang/api/api.go                         | 2 ++
 traffic_ops/traffic_ops_golang/api/shared_handlers.go             | 8 ++++----
 traffic_ops/traffic_ops_golang/asn/asns.go                        | 2 +-
 traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go          | 2 +-
 traffic_ops/traffic_ops_golang/cdn/cdns.go                        | 2 +-
 traffic_ops/traffic_ops_golang/coordinate/coordinates.go          | 2 +-
 .../traffic_ops_golang/deliveryservice/deliveryservicesv12.go     | 2 +-
 .../traffic_ops_golang/deliveryservice/deliveryservicesv13.go     | 2 +-
 .../deliveryservice/request/comment/comments.go                   | 2 +-
 .../traffic_ops_golang/deliveryservice/request/requests.go        | 6 +++---
 traffic_ops/traffic_ops_golang/division/divisions.go              | 2 +-
 traffic_ops/traffic_ops_golang/origin/origins.go                  | 2 +-
 traffic_ops/traffic_ops_golang/parameter/parameters.go            | 2 +-
 traffic_ops/traffic_ops_golang/physlocation/phys_locations.go     | 2 +-
 traffic_ops/traffic_ops_golang/profile/profiles.go                | 2 +-
 .../traffic_ops_golang/profileparameter/profile_parameters.go     | 2 +-
 traffic_ops/traffic_ops_golang/region/regions.go                  | 2 +-
 traffic_ops/traffic_ops_golang/role/roles.go                      | 2 +-
 traffic_ops/traffic_ops_golang/server/servers.go                  | 2 +-
 traffic_ops/traffic_ops_golang/status/statuses.go                 | 2 +-
 traffic_ops/traffic_ops_golang/tenant/tenancy.go                  | 2 +-
 traffic_ops/traffic_ops_golang/types/types.go                     | 2 +-
 22 files changed, 28 insertions(+), 26 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/api.go b/traffic_ops/traffic_ops_golang/api/api.go
index 99a4d89..ec3f9f9 100644
--- a/traffic_ops/traffic_ops_golang/api/api.go
+++ b/traffic_ops/traffic_ops_golang/api/api.go
@@ -44,6 +44,8 @@ const DBContextKey = "db"
 const ConfigContextKey = "context"
 const ReqIDContextKey = "reqid"
 
+type CRUDFactory func(reqInfo *APIInfo) CRUDer
+
 // WriteResp takes any object, serializes it as JSON, and writes that to w. Any errors are logged and written to w via tc.GetHandleErrorsFunc.
 // This is a helper for the common case; not using this in unusual cases is perfectly acceptable.
 func WriteResp(w http.ResponseWriter, r *http.Request, v interface{}) {
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index 46f4b77..237cc60 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -120,7 +120,7 @@ func decodeAndValidateRequestBody(r *http.Request, v Validator) []error {
 //      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(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
+func ReadHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		//create error function with ResponseWriter and Request
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
@@ -216,7 +216,7 @@ func ReadOnlyHandler(typeFactory func(reqInfo *APIInfo) Reader) http.HandlerFunc
 //   *decoding and validating the struct
 //   *change log entry
 //   *forming and writing the body over the wire
-func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
+func UpdateHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		//create error function with ResponseWriter and Request
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
@@ -332,7 +332,7 @@ func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 //   *current user
 //   *change log entry
 //   *forming and writing the body over the wire
-func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
+func DeleteHandler(typeFactory CRUDFactory) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
 
@@ -422,7 +422,7 @@ func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 //   *decoding and validating the struct
 //   *change log entry
 //   *forming and writing the body over the wire
-func CreateHandler(typeConstructor func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
+func CreateHandler(typeConstructor CRUDFactory) http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
 		handleErrs := tc.GetHandleErrorsFunc(w, r)
 
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index 0886c6f..af59b18 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -46,7 +46,7 @@ type TOASNV11 struct {
 	tc.ASNNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOASNV11{reqInfo, tc.ASNNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index 505352b..1e12577 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -42,7 +42,7 @@ type TOCacheGroup struct {
 	v13.CacheGroupNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCacheGroup{reqInfo, v13.CacheGroupNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns.go b/traffic_ops/traffic_ops_golang/cdn/cdns.go
index 3412a4c..39e5492 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns.go
@@ -43,7 +43,7 @@ type TOCDN struct {
 	v13.CDNNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCDN{reqInfo, v13.CDNNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index eaabede..3b978ac 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
@@ -43,7 +43,7 @@ type TOCoordinate struct {
 	v13.CoordinateNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCoordinate{reqInfo, v13.CoordinateNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index 5c5d1d9..cd16e7e 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -41,7 +41,7 @@ type TODeliveryServiceV12 struct {
 	tc.DeliveryServiceNullableV12
 }
 
-func GetTypeV12Factory() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeV12Factory() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceV12{reqInfo, tc.DeliveryServiceNullableV12{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index fbd996e..d6170da 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -60,7 +60,7 @@ func (ds *TODeliveryServiceV13) UnmarshalJSON(data []byte) error {
 	return json.Unmarshal(data, ds.DeliveryServiceNullableV13)
 }
 
-func GetTypeV13Factory() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeV13Factory() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceV13{reqInfo, tc.DeliveryServiceNullableV13{}}
 		return &toReturn
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 5887b8f..3818266 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -41,7 +41,7 @@ type TODeliveryServiceRequestComment struct {
 	tc.DeliveryServiceRequestCommentNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceRequestComment{reqInfo, tc.DeliveryServiceRequestCommentNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index e2bd836..45c0ff9 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -41,7 +41,7 @@ type TODeliveryServiceRequest struct {
 	tc.DeliveryServiceRequestNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}
 		return &toReturn
@@ -447,7 +447,7 @@ WHERE id=:id`
 ////////////////////////////////////////////////////////////////
 // Assignment change
 
-func GetAssignmentTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetAssignmentTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := deliveryServiceRequestAssignment{TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}}
 		return &toReturn
@@ -537,7 +537,7 @@ type deliveryServiceRequestStatus struct {
 	TODeliveryServiceRequest
 }
 
-func GetStatusTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetStatusTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := deliveryServiceRequestStatus{TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/division/divisions.go b/traffic_ops/traffic_ops_golang/division/divisions.go
index 0e0e7e7..52a8e4c 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions.go
@@ -41,7 +41,7 @@ type TODivision struct {
 	tc.DivisionNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODivision{reqInfo, tc.DivisionNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index e760660..82eb3ea 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -46,7 +46,7 @@ type TOOrigin struct {
 	v13.Origin
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOOrigin{reqInfo, v13.Origin{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters.go b/traffic_ops/traffic_ops_golang/parameter/parameters.go
index c0363f9..e21011d 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters.go
@@ -54,7 +54,7 @@ type TOParameter struct {
 	tc.ParameterNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOParameter{reqInfo, tc.ParameterNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
index 8528341..b9de171 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
@@ -41,7 +41,7 @@ type TOPhysLocation struct {
 	tc.PhysLocationNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOPhysLocation{reqInfo, tc.PhysLocationNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles.go b/traffic_ops/traffic_ops_golang/profile/profiles.go
index 87c99a0..b3cbaa2 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles.go
@@ -52,7 +52,7 @@ type TOProfile struct {
 	v13.ProfileNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOProfile{reqInfo, v13.ProfileNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
index e6e3366..9d3c7b7 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
@@ -47,7 +47,7 @@ type TOProfileParameter struct {
 	v13.ProfileParameterNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOProfileParameter{reqInfo, v13.ProfileParameterNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/region/regions.go b/traffic_ops/traffic_ops_golang/region/regions.go
index f963923..a8d074b 100644
--- a/traffic_ops/traffic_ops_golang/region/regions.go
+++ b/traffic_ops/traffic_ops_golang/region/regions.go
@@ -37,7 +37,7 @@ type TORegion struct {
 	tc.Region
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TORegion{reqInfo, tc.Region{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/role/roles.go b/traffic_ops/traffic_ops_golang/role/roles.go
index 763ab62..ba5916f 100644
--- a/traffic_ops/traffic_ops_golang/role/roles.go
+++ b/traffic_ops/traffic_ops_golang/role/roles.go
@@ -42,7 +42,7 @@ type TORole struct {
 	v13.Role
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TORole{reqInfo, v13.Role{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index 7e51ab3..879d165 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -44,7 +44,7 @@ type TOServer struct {
 	v13.ServerNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOServer{reqInfo, v13.ServerNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/status/statuses.go b/traffic_ops/traffic_ops_golang/status/statuses.go
index 5821a35..9c405af 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses.go
@@ -41,7 +41,7 @@ type TOStatus struct {
 	tc.StatusNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOStatus{reqInfo, tc.StatusNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index 6b66141..202dd70 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -298,7 +298,7 @@ type TOTenant struct {
 	tc.TenantNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOTenant{reqInfo, tc.TenantNullable{}}
 		return &toReturn
diff --git a/traffic_ops/traffic_ops_golang/types/types.go b/traffic_ops/traffic_ops_golang/types/types.go
index ab95214..a8c809f 100644
--- a/traffic_ops/traffic_ops_golang/types/types.go
+++ b/traffic_ops/traffic_ops_golang/types/types.go
@@ -41,7 +41,7 @@ type TOType struct {
 	tc.TypeNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+func GetTypeSingleton() api.CRUDFactory {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOType{reqInfo, tc.TypeNullable{}}
 		return &toReturn


[trafficcontrol] 08/15: fix profiles by removing nested query with rows.Next loop and fix routes for deliveryservice_requests

Posted by da...@apache.org.
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 a323f5af8848a2e6376a5605882425c64f583e3d
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 23:44:20 2018 -0600

    fix profiles by removing nested query with rows.Next loop and fix routes for deliveryservice_requests
---
 traffic_ops/traffic_ops_golang/profile/profiles.go | 23 +++++++++++++---------
 traffic_ops/traffic_ops_golang/routes.go           |  6 +++---
 2 files changed, 17 insertions(+), 12 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/profile/profiles.go b/traffic_ops/traffic_ops_golang/profile/profiles.go
index 6b39006..87c99a0 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles.go
@@ -47,13 +47,13 @@ const (
 )
 
 //we need a type alias to define functions on
-type TOProfile struct{
+type TOProfile struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.ProfileNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOProfile{reqInfo, v13.ProfileNullable{}}
 		return &toReturn
 	}
@@ -127,27 +127,32 @@ func (prof *TOProfile) Read(parameters map[string]string) ([]interface{}, []erro
 	}
 	defer rows.Close()
 
-	profiles := []interface{}{}
+	profiles := []v13.ProfileNullable{}
+
 	for rows.Next() {
 		var p v13.ProfileNullable
 		if err = rows.StructScan(&p); err != nil {
 			log.Errorf("error parsing Profile rows: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
 		}
-
+		profiles = append(profiles, p)
+	}
+	rows.Close()
+	profileInterfaces := []interface{}{}
+	for _, profile := range profiles {
 		// Attach Parameters if the 'id' parameter is sent
 		if _, ok := parameters[IDQueryParam]; ok {
-			params, err := ReadParameters(prof.ReqInfo.Tx, parameters, prof.ReqInfo.User, p)
-			p.Parameters = params
+			params, err := ReadParameters(prof.ReqInfo.Tx, parameters, prof.ReqInfo.User, profile)
+			profile.Parameters = params
 			if len(errs) > 0 {
 				log.Errorf("Error getting Parameters: %v", err)
 				return nil, []error{tc.DBError}, tc.SystemError
 			}
 		}
-		profiles = append(profiles, p)
+		profileInterfaces = append(profileInterfaces, profile)
 	}
 
-	return profiles, []error{}, tc.NoError
+	return profileInterfaces, []error{}, tc.NoError
 
 }
 
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 00edc23..c270424 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -251,10 +251,10 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 
 		//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.MethodGet, `deliveryservice_requests/?$`, api.ReadHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.3, http.MethodPut, `deliveryservice_requests/?$`, 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},
+		{1.3, http.MethodDelete, `deliveryservice_requests/?$`, api.DeleteHandler(dsrequest.GetTypeSingleton()), auth.PrivLevelPortal, Authenticated, nil},
 
 		//Delivery service request: Actions
 		{1.3, http.MethodPut, `deliveryservice_requests/{id}/assign$`, api.UpdateHandler(dsrequest.GetAssignmentTypeSingleton()), auth.PrivLevelOperations, Authenticated, nil},


[trafficcontrol] 15/15: fix up places where IsTenancyEnabledTx now returns an error

Posted by da...@apache.org.
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 71972efed58a8670e1a76693e416ac962c8eb36d
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Mon Jul 2 17:31:46 2018 -0600

    fix up places where IsTenancyEnabledTx now returns an error
---
 .../deliveryservice/deliveryservicesv13.go            |  8 +++++++-
 .../deliveryservice/request/requests.go               |  7 ++++++-
 traffic_ops/traffic_ops_golang/origin/origins.go      | 19 +++++++++++++++----
 traffic_ops/traffic_ops_golang/test/helpers.go        |  3 +--
 4 files changed, 29 insertions(+), 8 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index efc385d..cd6bfa9 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -615,7 +615,13 @@ func readGetDeliveryServices(params map[string]string, tx *sqlx.Tx, user *auth.C
 		return nil, errs, tc.DataConflictError
 	}
 
-	if tenant.IsTenancyEnabledTx(tx) {
+	tenancyEnabled, err := tenant.IsTenancyEnabledTx(tx.Tx)
+	if err != nil {
+		log.Errorln("checking if tenancy is enabled: " + err.Error())
+		return nil, []error{tc.DBError}, tc.SystemError
+	}
+
+	if tenancyEnabled {
 		log.Debugln("Tenancy is enabled")
 		tenantIDs, err := tenant.GetUserTenantIDListTx(user, tx)
 		if err != nil {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index f367480..85a1389 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -101,7 +101,12 @@ func (req *TODeliveryServiceRequest) Read(parameters map[string]string) ([]inter
 	if len(errs) > 0 {
 		return nil, errs, tc.DataConflictError
 	}
-	if tenant.IsTenancyEnabledTx(req.ReqInfo.Tx) {
+	tenancyEnabled, err := tenant.IsTenancyEnabledTx(req.ReqInfo.Tx.Tx)
+	if err != nil {
+		log.Errorln("checking if tenancy is enabled: " + err.Error())
+		return nil, []error{tc.DBError}, tc.SystemError
+	}
+	if tenancyEnabled {
 		log.Debugln("Tenancy is enabled")
 		tenantIDs, err := tenant.GetUserTenantIDListTx(req.ReqInfo.User, req.ReqInfo.Tx)
 		if err != nil {
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index 0693ff5..16b32e7 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -131,8 +131,11 @@ func (origin *TOOrigin) IsTenantAuthorized(user *auth.CurrentUser) (bool, error)
 	if err != nil {
 		return false, err
 	}
-
-	if currentTenantID != nil && tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx) {
+	tenancyEnabled, err := tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx.Tx)
+	if err != nil {
+		return false, err
+	}
+	if currentTenantID != nil && tenancyEnabled {
 		return tenant.IsResourceAuthorizedToUserTx(*currentTenantID, user, origin.ReqInfo.Tx.Tx)
 	}
 
@@ -178,7 +181,11 @@ func (origin *TOOrigin) Read(params map[string]string) ([]interface{}, []error,
 	}
 
 	var err error
-	if tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx) {
+	tenancyEnabled, err := tenant.IsTenancyEnabledTx(origin.ReqInfo.Tx.Tx)
+	if err != nil {
+		return nil, []error{errors.New("Error checking if tenancy enabled.")}, tc.SystemError
+	}
+	if tenancyEnabled {
 		origins, err = filterAuthorized(origins, origin.ReqInfo.User, origin.ReqInfo.Tx)
 		if err != nil {
 			log.Errorln("Checking tenancy: " + err.Error())
@@ -271,7 +278,11 @@ LEFT JOIN tenant t ON o.tenant = t.id`
 }
 
 func checkTenancy(originTenantID, deliveryserviceID *int, tx *sqlx.Tx, user *auth.CurrentUser) (error, tc.ApiErrorType) {
-	if tenant.IsTenancyEnabledTx(tx) {
+	tenancyEnabled, err := tenant.IsTenancyEnabledTx(tx.Tx)
+	if err != nil {
+		return errors.New("Error checking if tenancy enabled."), tc.SystemError
+	}
+	if tenancyEnabled {
 		if originTenantID == nil {
 			return tc.NilTenantError, tc.ForbiddenError
 		}
diff --git a/traffic_ops/traffic_ops_golang/test/helpers.go b/traffic_ops/traffic_ops_golang/test/helpers.go
index 0c63660..8c0a57d 100644
--- a/traffic_ops/traffic_ops_golang/test/helpers.go
+++ b/traffic_ops/traffic_ops_golang/test/helpers.go
@@ -20,11 +20,10 @@ package test
  */
 
 import (
+	"errors"
 	"reflect"
 	"sort"
 	"strings"
-
-	"github.com/pkg/errors"
 )
 
 // Extract the tag annotations from a struct into a string array


[trafficcontrol] 14/15: fix tests broken by validation error change

Posted by da...@apache.org.
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 6dc69defe8aa506a5fc16b1421cc39c95f2182d7
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Mon Jul 2 15:15:58 2018 -0600

    fix tests broken by validation error change
---
 .../traffic_ops_golang/api/shared_handlers_test.go    |  4 ++--
 traffic_ops/traffic_ops_golang/asn/asns_test.go       |  6 +++---
 .../traffic_ops_golang/cachegroup/cachegroups_test.go | 13 ++++++-------
 traffic_ops/traffic_ops_golang/cdn/cdns_test.go       | 13 ++++++-------
 .../traffic_ops_golang/coordinate/coordinates_test.go | 13 ++++++-------
 .../deliveryservice/request/comment/comments_test.go  | 15 ++++++++-------
 .../traffic_ops_golang/division/divisions_test.go     |  2 +-
 traffic_ops/traffic_ops_golang/origin/origins_test.go | 19 +++++++++----------
 .../physlocation/phys_locations_test.go               |  6 +++---
 .../traffic_ops_golang/profile/profiles_test.go       |  6 +++---
 traffic_ops/traffic_ops_golang/role/roles_test.go     | 14 +++++++-------
 traffic_ops/traffic_ops_golang/test/helpers.go        | 14 ++++++++++++++
 traffic_ops/traffic_ops_golang/types/types_test.go    |  6 +++---
 13 files changed, 71 insertions(+), 60 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
index abf122f..c13543d 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
@@ -76,9 +76,9 @@ func (i *tester) GetAuditName() string {
 //Validator interface function
 func (v *tester) Validate() error {
 	if v.ID < 1 {
-		return []error{errors.New("ID is too low")}
+		return errors.New("ID is too low")
 	}
-	return []error{}
+	return nil
 }
 
 //Creator interface functions
diff --git a/traffic_ops/traffic_ops_golang/asn/asns_test.go b/traffic_ops/traffic_ops_golang/asn/asns_test.go
index 00963f0..f08697f 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns_test.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns_test.go
@@ -118,11 +118,11 @@ func TestValidate(t *testing.T) {
 	i := -99
 	asn := TOASNV11{nil, tc.ASNNullable{ASN: &i, CachegroupID: &i}}
 
-	errs := test.SortErrors(asn.Validate())
-	expected := []error{
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(asn.Validate())))
+	expected := util.JoinErrsStr([]error{
 		errors.New(`'asn' must be no less than 0`),
 		errors.New(`'cachegroupId' must be no less than 0`),
-	}
+	})
 	if !reflect.DeepEqual(expected, errs) {
 		t.Errorf(`expected %v,  got %v`, expected, errs)
 	}
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
index 1dade56..def84ab 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
@@ -168,14 +168,14 @@ func TestValidate(t *testing.T) {
 		TypeID:      &ti,
 		LastUpdated: &lu,
 	}}
-	errs := test.SortErrors(c.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'latitude' Must be a floating point number within the range +-90`),
 		errors.New(`'longitude' Must be a floating point number within the range +-180`),
 		errors.New(`'name' invalid characters found - Use alphanumeric . or - or _ .`),
 		errors.New(`'shortName' invalid characters found - Use alphanumeric . or - or _ .`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -195,9 +195,8 @@ func TestValidate(t *testing.T) {
 		TypeID:      &ti,
 		LastUpdated: &lu,
 	}}
-	expectedErrs = []error{}
-	errs = c.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	err := c.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
index 2bc4288..d7dd8f3 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
@@ -134,12 +134,12 @@ func TestValidate(t *testing.T) {
 	// invalid name, empty domainname
 	n := "not_a_valid_cdn"
 	c := TOCDN{CDNNullable: v13.CDNNullable{Name: &n}}
-	errs := test.SortErrors(c.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'domainName' cannot be blank`),
 		errors.New(`'name' invalid characters found - Use alphanumeric . or - .`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -149,9 +149,8 @@ func TestValidate(t *testing.T) {
 	n = "This.is.2.a-Valid---CDNNAME."
 	d := `awesome-cdn.example.net`
 	c = TOCDN{CDNNullable: v13.CDNNullable{Name: &n, DomainName: &d}}
-	expectedErrs = []error{}
-	errs = c.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	err := c.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
index b8ba402..10c15d5 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
@@ -147,13 +147,13 @@ func TestValidate(t *testing.T) {
 		Longitude:   &lo,
 		LastUpdated: &lu,
 	}}
-	errs := test.SortErrors(c.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'latitude' Must be a floating point number within the range +-90`),
 		errors.New(`'longitude' Must be a floating point number within the range +-180`),
 		errors.New(`'name' invalid characters found - Use alphanumeric . or - or _ .`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -169,9 +169,8 @@ func TestValidate(t *testing.T) {
 		Longitude:   &lo,
 		LastUpdated: &lu,
 	}}
-	expectedErrs = []error{}
-	errs = c.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	err := c.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
index fa1de48..b16ac6c 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
@@ -26,6 +26,7 @@ import (
 	"testing"
 
 	"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/test"
 )
@@ -68,12 +69,12 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	c := TODeliveryServiceRequestComment{}
-	errs := test.SortErrors(c.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'deliveryServiceRequestId' is required`),
 		errors.New(`'value' is required`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -82,10 +83,10 @@ func TestValidate(t *testing.T) {
 	v := "the comment value"
 	d := 1
 	c = TODeliveryServiceRequestComment{DeliveryServiceRequestCommentNullable: tc.DeliveryServiceRequestCommentNullable{DeliveryServiceRequestID: &d, Value: &v}}
-	expectedErrs = []error{}
-	errs = c.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+
+	err := c.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 
 }
diff --git a/traffic_ops/traffic_ops_golang/division/divisions_test.go b/traffic_ops/traffic_ops_golang/division/divisions_test.go
index a8d93a8..1b582d8 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions_test.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions_test.go
@@ -110,7 +110,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidation(t *testing.T) {
 	div := TODivision{}
-	errs := test.SortErrors(div.Validate())
+	errs := test.SortErrors(test.SplitErrors(div.Validate()))
 	expected := []error{}
 
 	if reflect.DeepEqual(expected, errs) {
diff --git a/traffic_ops/traffic_ops_golang/origin/origins_test.go b/traffic_ops/traffic_ops_golang/origin/origins_test.go
index 56d1b87..7280e60 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins_test.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins_test.go
@@ -182,14 +182,14 @@ func TestValidate(t *testing.T) {
 		FQDN:              nil,
 		Protocol:          nil,
 	}}
-	errs := test.SortErrors(c.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'deliveryServiceId' is required`),
 		errors.New(`'fqdn' cannot be blank`),
 		errors.New(`'name' cannot be blank`),
 		errors.New(`'protocol' cannot be blank`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -214,10 +214,9 @@ func TestValidate(t *testing.T) {
 		Protocol:          &pro,
 		LastUpdated:       &lu,
 	}}
-	expectedErrs = []error{}
-	errs = c.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	err := c.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 
 	type testCase struct {
@@ -320,9 +319,9 @@ func TestValidate(t *testing.T) {
 				c.IP6Address = &tc.Str
 				value = tc.Str
 			}
-			errs = test.SortErrors(c.Validate())
-			if !reflect.DeepEqual(tc.ExpectedErrors, errs) {
-				t.Errorf("given: '%v', expected %s, got %s", value, tc.ExpectedErrors, errs)
+			errStr := util.JoinErrsStr(test.SortErrors(test.SplitErrors(c.Validate())))
+			if !reflect.DeepEqual(util.JoinErrsStr(tc.ExpectedErrors), errStr) {
+				t.Errorf("given: '%v', expected %s, got %s", value, tc.ExpectedErrors, errStr)
 			}
 		}
 	}
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
index bb459c8..f88f193 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
@@ -133,8 +133,8 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOPhysLocation{}
-	errs := test.SortErrors(p.Validate())
-	expected := test.SortErrors([]error{
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(p.Validate())))
+	expected := util.JoinErrsStr(test.SortErrors([]error{
 		errors.New("'state' cannot be blank"),
 		errors.New("'zip' cannot be blank"),
 		errors.New("'address' cannot be blank"),
@@ -142,7 +142,7 @@ func TestValidate(t *testing.T) {
 		errors.New("'name' cannot be blank"),
 		errors.New("'regionId' cannot be blank"),
 		errors.New("'shortName' cannot be blank"),
-	})
+	}))
 
 	if !reflect.DeepEqual(expected, errs) {
 		t.Errorf("expected %++v,  got %++v", expected, errs)
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles_test.go b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
index 5ad1b95..8dc3177 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles_test.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
@@ -134,13 +134,13 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOProfile{}
-	errs := test.SortErrors(p.Validate())
-	expected := test.SortErrors([]error{
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(p.Validate())))
+	expected := util.JoinErrsStr(test.SortErrors([]error{
 		errors.New("'cdn' cannot be blank"),
 		errors.New("'description' cannot be blank"),
 		errors.New("'name' cannot be blank"),
 		errors.New("'type' cannot be blank"),
-	})
+	}))
 
 	if !reflect.DeepEqual(expected, errs) {
 		t.Errorf("expected %++v,  got %++v", expected, errs)
diff --git a/traffic_ops/traffic_ops_golang/role/roles_test.go b/traffic_ops/traffic_ops_golang/role/roles_test.go
index 7807371..5f9e9f7 100644
--- a/traffic_ops/traffic_ops_golang/role/roles_test.go
+++ b/traffic_ops/traffic_ops_golang/role/roles_test.go
@@ -26,6 +26,7 @@ import (
 	"testing"
 
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"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/test"
 )
@@ -98,12 +99,12 @@ func TestValidate(t *testing.T) {
 	n := "not_a_valid_role"
 	reqInfo := api.APIInfo{}
 	r := TORole{ReqInfo: &reqInfo, Role: v13.Role{Name: &n}}
-	errs := test.SortErrors(r.Validate())
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(r.Validate())))
 
-	expectedErrs := []error{
+	expectedErrs := util.JoinErrsStr([]error{
 		errors.New(`'description' cannot be blank`),
 		errors.New(`'privLevel' cannot be blank`),
-	}
+	})
 
 	if !reflect.DeepEqual(expectedErrs, errs) {
 		t.Errorf("expected %s, got %s", expectedErrs, errs)
@@ -111,10 +112,9 @@ func TestValidate(t *testing.T) {
 
 	//  name,  domainname both valid
 	r = TORole{ReqInfo: &reqInfo, Role: v13.Role{Name: stringAddr("this is a valid name"), Description: stringAddr("this is a description"), PrivLevel: intAddr(30)}}
-	expectedErrs = []error{}
-	errs = r.Validate()
-	if !reflect.DeepEqual(expectedErrs, errs) {
-		t.Errorf("expected %s, got %s", expectedErrs, errs)
+	err := r.Validate()
+	if err != nil {
+		t.Errorf("expected nil, got %s", err)
 	}
 
 }
diff --git a/traffic_ops/traffic_ops_golang/test/helpers.go b/traffic_ops/traffic_ops_golang/test/helpers.go
index 4b6ef9c..0c63660 100644
--- a/traffic_ops/traffic_ops_golang/test/helpers.go
+++ b/traffic_ops/traffic_ops_golang/test/helpers.go
@@ -23,6 +23,8 @@ import (
 	"reflect"
 	"sort"
 	"strings"
+
+	"github.com/pkg/errors"
 )
 
 // Extract the tag annotations from a struct into a string array
@@ -60,3 +62,15 @@ func SortErrors(p []error) []error {
 	sort.Sort(sortableErrors(p))
 	return p
 }
+
+func SplitErrors(err error) []error {
+	if err == nil {
+		return []error{}
+	}
+	strs := strings.Split(err.Error(), ", ")
+	errs := []error{}
+	for _, str := range strs {
+		errs = append(errs, errors.New(str))
+	}
+	return errs
+}
diff --git a/traffic_ops/traffic_ops_golang/types/types_test.go b/traffic_ops/traffic_ops_golang/types/types_test.go
index 780f893..b7fea61 100644
--- a/traffic_ops/traffic_ops_golang/types/types_test.go
+++ b/traffic_ops/traffic_ops_golang/types/types_test.go
@@ -122,12 +122,12 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	p := TOType{}
-	errs := test.SortErrors(p.Validate())
-	expected := test.SortErrors([]error{
+	errs := util.JoinErrsStr(test.SortErrors(test.SplitErrors(p.Validate())))
+	expected := util.JoinErrsStr(test.SortErrors([]error{
 		errors.New("'name' cannot be blank"),
 		errors.New("'description' cannot be blank"),
 		errors.New("'use_in_table' cannot be blank"),
-	})
+	}))
 
 	if !reflect.DeepEqual(expected, errs) {
 		t.Errorf("expected %++v,  got %++v", expected, errs)


[trafficcontrol] 05/15: remove nested tenancy check from deliveryservice request read method

Posted by da...@apache.org.
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 70fe7c02a8f7823fe3a44902cc3696fc9cd0dd35
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 20:50:57 2018 -0600

    remove nested tenancy check from deliveryservice request read method
---
 .../deliveryservice/deliveryservicesv12.go           |  2 +-
 .../deliveryservice/deliveryservicesv13.go           |  4 ++--
 .../deliveryservice/request/requests.go              | 20 ++++++++++----------
 traffic_ops/traffic_ops_golang/tenant/tenancy.go     |  2 +-
 4 files changed, 14 insertions(+), 14 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index aa0d36a..dd2d3cc 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -216,7 +216,7 @@ func CreateV12() http.HandlerFunc {
 
 	func(ds *TODeliveryServiceV12) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
-	dses, errs, errType := readGetDeliveryServices(params, ds.ReqInfo.Tx, *ds.ReqInfo.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` {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 23d9d01..10d3182 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -248,7 +248,7 @@ func create(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds tc.Deliver
 
 func (ds *TODeliveryServiceV13) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
-	dses, errs, errType := readGetDeliveryServices(params, ds.ReqInfo.Tx, *ds.ReqInfo.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
@@ -603,7 +603,7 @@ func filterAuthorized(dses []tc.DeliveryServiceNullableV13, user *auth.CurrentUs
 	return newDSes, nil
 }
 
-func readGetDeliveryServices(params map[string]string, tx *sqlx.Tx, 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")]
 	}
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index 00a3d9d..8ac9f2b 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -102,6 +102,15 @@ func (req *TODeliveryServiceRequest) Read(parameters map[string]string) ([]inter
 	if len(errs) > 0 {
 		return nil, errs, tc.DataConflictError
 	}
+	if tenant.IsTenancyEnabledTx(req.ReqInfo.Tx) {
+		log.Debugln("Tenancy is enabled")
+		tenantIDs, err := tenant.GetUserTenantIDListTx(req.ReqInfo.User, req.ReqInfo.Tx)
+		if err != nil {
+			log.Errorln("received error querying for user's tenants: " + err.Error())
+			return nil, []error{tc.DBError}, tc.SystemError
+		}
+		where, queryValues = dbhelpers.AddTenancyCheck(where, queryValues, "r.deliveryservice->>'tenantId", tenantIDs)
+	}
 
 	query := selectDeliveryServiceRequestsQuery() + where + orderBy
 	log.Debugln("Query is ", query)
@@ -120,16 +129,7 @@ func (req *TODeliveryServiceRequest) Read(parameters map[string]string) ([]inter
 			log.Errorf("error parsing DeliveryServiceRequest rows: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
 		}
-
-		// TODO: combine tenancy with the query above so there's a single db call
-		t, err := s.IsTenantAuthorized(req.ReqInfo.User)
-		if err != nil {
-			log.Errorf("error checking tenancy: %v", err)
-			return nil, []error{tc.DBError}, tc.SystemError
-		}
-		if t {
-			deliveryServiceRequests = append(deliveryServiceRequests, s)
-		}
+		deliveryServiceRequests = append(deliveryServiceRequests, s)
 	}
 
 	return deliveryServiceRequests, []error{}, tc.NoError
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index fad4cdc..ea6d5ce 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -183,7 +183,7 @@ func GetUserTenantIDList(user auth.CurrentUser, db *sqlx.DB) ([]int, error) {
 	return tenants, nil
 }
 
-func GetUserTenantIDListTx(user auth.CurrentUser, tx *sqlx.Tx) ([]int, error) {
+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;`


[trafficcontrol] 09/15: run goimports on all files

Posted by da...@apache.org.
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 ec1e7f8570e830b31688294ef84afa84c84ab396
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 23:49:01 2018 -0600

    run goimports on all files
---
 traffic_ops/traffic_ops_golang/api/change_log.go   |  10 +-
 .../traffic_ops_golang/api/shared_handlers.go      |   7 +-
 .../traffic_ops_golang/api/shared_handlers_test.go |  10 +-
 traffic_ops/traffic_ops_golang/asn/asns.go         |   7 +-
 traffic_ops/traffic_ops_golang/asn/asns_test.go    |   6 +-
 .../traffic_ops_golang/cachegroup/cachegroups.go   |   4 +-
 .../cachegroup/cachegroups_test.go                 |   3 +-
 traffic_ops/traffic_ops_golang/cdn/cdns.go         |   6 +-
 traffic_ops/traffic_ops_golang/cdn/cdns_test.go    |   4 +-
 .../traffic_ops_golang/coordinate/coordinates.go   |   6 +-
 .../coordinate/coordinates_test.go                 |  20 +--
 .../traffic_ops_golang/crconfig/edgelocations.go   |   1 -
 .../deliveryservice/deliveryservicesv12.go         | 191 ++++++++++-----------
 .../deliveryservice/deliveryservicesv13.go         |  16 +-
 .../deliveryservice/request/comment/comments.go    |   8 +-
 .../request/comment/comments_test.go               |   2 +-
 .../deliveryservice/request/requests.go            |  16 +-
 .../deliveryservice/request/requests_test.go       |  16 +-
 .../deliveryservice/request/validate.go            |   1 -
 .../deliveryservice/servers/servers.go             |   6 +-
 .../deliveryservicesregexes.go                     |   2 +-
 .../traffic_ops_golang/division/divisions.go       |   6 +-
 .../traffic_ops_golang/division/divisions_test.go  |   2 +-
 traffic_ops/traffic_ops_golang/origin/origins.go   |   6 +-
 .../traffic_ops_golang/origin/origins_test.go      |   2 +-
 .../traffic_ops_golang/parameter/parameters.go     |   6 +-
 .../parameter/parameters_test.go                   |   6 +-
 .../physlocation/phys_locations.go                 |   6 +-
 .../physlocation/phys_locations_test.go            |   2 +-
 .../traffic_ops_golang/profile/profiles_test.go    |   2 +-
 .../profileparameter/profile_parameters.go         |   6 +-
 .../profileparameter/profile_parameters_test.go    |   3 +-
 traffic_ops/traffic_ops_golang/region/regions.go   |   6 +-
 .../traffic_ops_golang/region/regions_test.go      |   2 +-
 traffic_ops/traffic_ops_golang/riaksvc/dsutil.go   |   3 +-
 traffic_ops/traffic_ops_golang/role/roles.go       |   6 +-
 traffic_ops/traffic_ops_golang/routes.go           |  12 +-
 traffic_ops/traffic_ops_golang/server/servers.go   |   6 +-
 .../staticdnsentry/staticdnsentry.go               |   6 +-
 traffic_ops/traffic_ops_golang/status/statuses.go  |   6 +-
 .../traffic_ops_golang/status/statuses_test.go     |   2 +-
 .../traffic_ops_golang/systeminfo/system_info.go   |   9 +-
 traffic_ops/traffic_ops_golang/tenant/tenancy.go   |  19 +-
 traffic_ops/traffic_ops_golang/types/types.go      |   6 +-
 traffic_ops/traffic_ops_golang/types/types_test.go |   3 +-
 45 files changed, 233 insertions(+), 242 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/api/change_log.go b/traffic_ops/traffic_ops_golang/api/change_log.go
index b8117ec..b9761e4 100644
--- a/traffic_ops/traffic_ops_golang/api/change_log.go
+++ b/traffic_ops/traffic_ops_golang/api/change_log.go
@@ -21,6 +21,7 @@ package api
 
 import (
 	"database/sql"
+	"errors"
 	"fmt"
 
 	"github.com/apache/trafficcontrol/lib/go-log"
@@ -28,7 +29,6 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 
 	"github.com/jmoiron/sqlx"
-	"errors"
 )
 
 type ChangeLog struct {
@@ -77,10 +77,10 @@ func CreateChangeLogBuildMsg(level string, action string, user *auth.CurrentUser
 }
 
 func CreateChangeLogRawErr(level string, msg string, user *auth.CurrentUser, tx *sql.Tx) error {
-	 if _, err := tx.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID); err != nil {
-		 return errors.New("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
-	 }
-	 return nil
+	if _, err := tx.Exec(`INSERT INTO log (level, message, tm_user) VALUES ($1, $2, $3)`, level, msg, user.ID); err != nil {
+		return errors.New("Inserting change log level '" + level + "' message '" + msg + "' user '" + user.UserName + "': " + err.Error())
+	}
+	return nil
 }
 
 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 4a10cdb..46f4b77 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -304,7 +304,7 @@ func UpdateHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 		//auditing here
 		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()))
+			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
 		*inf.CommitTx = true
@@ -343,7 +343,6 @@ func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 		}
 		defer inf.Close()
 
-
 		d := typeFactory(inf)
 
 		params, err := GetCombinedParams(r)
@@ -395,7 +394,7 @@ func DeleteHandler(typeFactory func(reqInfo *APIInfo) CRUDer) http.HandlerFunc {
 		log.Debugf("changelog for delete on object")
 		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()))
+			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
 		*inf.CommitTx = true
@@ -467,7 +466,7 @@ func CreateHandler(typeConstructor func(reqInfo *APIInfo) CRUDer) http.HandlerFu
 
 		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()))
+			HandleErr(w, r, http.StatusInternalServerError, tc.DBError, errors.New("inserting changelog: "+err.Error()))
 			return
 		}
 		*inf.CommitTx = true
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
index 56880d4..f234d8a 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
@@ -42,7 +42,7 @@ type tester struct {
 	errorType tc.ApiErrorType //only for testing
 }
 
-func GetTypeSingleton() func(apiInfo *APIInfo) CRUDer{
+func GetTypeSingleton() func(apiInfo *APIInfo) CRUDer {
 	return func(apiInfo *APIInfo) CRUDer {
 		tester := tester{}
 		return &tester
@@ -129,12 +129,12 @@ func TestCreateHandler(t *testing.T) {
 	ctx = context.WithValue(ctx, DBContextKey, db)
 	ctx = context.WithValue(ctx, ConfigContextKey, &cfg)
 	ctx = context.WithValue(ctx, ReqIDContextKey, uint64(0))
-	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id":"1"})
+	ctx = context.WithValue(ctx, PathParamsKey, map[string]string{"id": "1"})
 
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{ID:1}
+	typeRef := tester{ID: 1}
 
 	createFunc := CreateHandler(GetTypeSingleton())
 
@@ -221,7 +221,7 @@ func TestUpdateHandler(t *testing.T) {
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{ID:1}
+	typeRef := tester{ID: 1}
 	updateFunc := UpdateHandler(GetTypeSingleton())
 
 	//verifies we get the right changelog insertion
@@ -265,7 +265,7 @@ func TestDeleteHandler(t *testing.T) {
 	// Add our context to the request
 	r = r.WithContext(ctx)
 
-	typeRef := tester{ID:1}
+	typeRef := tester{ID: 1}
 	deleteFunc := DeleteHandler(GetTypeSingleton())
 
 	//verifies we get the right changelog insertion
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index 9306392..0886c6f 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -41,13 +41,13 @@ import (
 const ASNsPrivLevel = 10
 
 //we need a type alias to define functions on
-type TOASNV11 struct{
+type TOASNV11 struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.ASNNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOASNV11{reqInfo, tc.ASNNullable{}}
 		return &toReturn
 	}
@@ -157,7 +157,6 @@ func (asn *TOASNV11) Read(parameters map[string]string) ([]interface{}, []error,
 	return iasns, err, errType
 }
 
-
 // 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() http.HandlerFunc {
 	return func(w http.ResponseWriter, r *http.Request) {
diff --git a/traffic_ops/traffic_ops_golang/asn/asns_test.go b/traffic_ops/traffic_ops_golang/asn/asns_test.go
index 5ad859a..3c1c159 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns_test.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns_test.go
@@ -30,8 +30,8 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/test"
 	"github.com/jmoiron/sqlx"
 
-	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
 	"github.com/apache/trafficcontrol/lib/go-util"
+	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
 )
 
 func getTestASNs() []tc.ASNNullable {
@@ -80,7 +80,7 @@ func TestGetASNs(t *testing.T) {
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
 	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 	asns, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 
 	if len(errs) > 0 {
@@ -116,7 +116,7 @@ func TestInterfaces(t *testing.T) {
 
 func TestValidate(t *testing.T) {
 	i := -99
-	asn := TOASNV11{nil,tc.ASNNullable{ASN: &i, CachegroupID: &i}}
+	asn := TOASNV11{nil, tc.ASNNullable{ASN: &i, CachegroupID: &i}}
 
 	errs := test.SortErrors(asn.Validate())
 	expected := []error{
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index 784e15f..505352b 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -37,10 +37,10 @@ import (
 	"github.com/lib/pq"
 )
 
-type TOCacheGroup struct{
+type TOCacheGroup struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.CacheGroupNullable
-	}
+}
 
 func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
 	return func(reqInfo *api.APIInfo) api.CRUDer {
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
index 1c4eaff..1dade56 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups_test.go
@@ -79,7 +79,6 @@ func TestReadCacheGroups(t *testing.T) {
 	db := sqlx.NewDb(mockDB, "sqlmock")
 	defer db.Close()
 
-
 	testCGs := getTestCacheGroups()
 	cols := test.ColsFromStructByTag("db", v13.CacheGroup{})
 	rows := sqlmock.NewRows(cols)
@@ -103,7 +102,7 @@ func TestReadCacheGroups(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 	cachegroups, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("cdn.Read expected: no errors, actual: %v", errs)
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns.go b/traffic_ops/traffic_ops_golang/cdn/cdns.go
index 007468e..3412a4c 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns.go
@@ -38,13 +38,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOCDN struct{
+type TOCDN struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.CDNNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCDN{reqInfo, v13.CDNNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
index c09529e..2bc4288 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns_test.go
@@ -27,8 +27,8 @@ import (
 	"time"
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
-	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"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/test"
 	"github.com/jmoiron/sqlx"
@@ -83,7 +83,7 @@ func TestReadCDNs(t *testing.T) {
 	mock.ExpectCommit()
 
 	v := map[string]string{"dsId": "1"}
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 	servers, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("cdn.Read expected: no errors, actual: %v", errs)
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index 2707519..eaabede 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
@@ -38,13 +38,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOCoordinate struct{
+type TOCoordinate struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.CoordinateNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOCoordinate{reqInfo, v13.CoordinateNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
index 6d3275b..b8ba402 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
@@ -26,9 +26,9 @@ import (
 	"testing"
 	"time"
 
-	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"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/test"
 	"github.com/jmoiron/sqlx"
@@ -87,7 +87,7 @@ func TestReadCoordinates(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 	coordinates, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("coordinate.Read expected: no errors, actual: %v", errs)
@@ -141,10 +141,10 @@ func TestValidate(t *testing.T) {
 	la := -190.0
 	lo := -190.0
 	lu := tc.TimeNoMod{Time: time.Now()}
-	c := TOCoordinate{ CoordinateNullable: v13.CoordinateNullable{ID: &id,
-		Name: &nm,
-		Latitude: &la,
-		Longitude: &lo,
+	c := TOCoordinate{CoordinateNullable: v13.CoordinateNullable{ID: &id,
+		Name:        &nm,
+		Latitude:    &la,
+		Longitude:   &lo,
 		LastUpdated: &lu,
 	}}
 	errs := test.SortErrors(c.Validate())
@@ -163,10 +163,10 @@ func TestValidate(t *testing.T) {
 	nm = "This.is.2.a-Valid---Coordinate."
 	la = 90.0
 	lo = 90.0
-	c = TOCoordinate{ CoordinateNullable: v13.CoordinateNullable{ID: &id,
-		Name: &nm,
-		Latitude: &la,
-		Longitude: &lo,
+	c = TOCoordinate{CoordinateNullable: v13.CoordinateNullable{ID: &id,
+		Name:        &nm,
+		Latitude:    &la,
+		Longitude:   &lo,
 		LastUpdated: &lu,
 	}}
 	expectedErrs = []error{}
diff --git a/traffic_ops/traffic_ops_golang/crconfig/edgelocations.go b/traffic_ops/traffic_ops_golang/crconfig/edgelocations.go
index 5020caf..924afb6 100644
--- a/traffic_ops/traffic_ops_golang/crconfig/edgelocations.go
+++ b/traffic_ops/traffic_ops_golang/crconfig/edgelocations.go
@@ -70,7 +70,6 @@ and cachegroup_fallbacks.primary_cg = $1 order by cachegroup_fallbacks.set_order
 			}
 			defer dbRows.Close()
 
-
 			if fallbackToClosest == nil {
 				fallbackToClosest = new(bool)
 				*fallbackToClosest = true
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index dd2d3cc..5c5d1d9 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -36,15 +36,13 @@ import (
 	"github.com/lib/pq"
 )
 
-
 type TODeliveryServiceV12 struct {
 	ReqInfo *api.APIInfo
 	tc.DeliveryServiceNullableV12
 }
 
-
-func GetTypeV12Factory() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer{
+func GetTypeV12Factory() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceV12{reqInfo, tc.DeliveryServiceNullableV12{}}
 		return &toReturn
 	}
@@ -130,7 +128,6 @@ func (ds *TODeliveryServiceV12) IsTenantAuthorized(user *auth.CurrentUser) (bool
 	return isTenantAuthorized(user, ds.ReqInfo.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 {
@@ -173,13 +170,12 @@ func isTenantAuthorized(user *auth.CurrentUser, tx *sqlx.Tx, ds *tc.DeliveryServ
 	return true, nil
 }
 
-
 func (ds *TODeliveryServiceV12) Validate() []error {
 	return ds.DeliveryServiceNullableV12.Validate(ds.ReqInfo.Tx.Tx)
 }
 
 // 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) {
+func (ds *TODeliveryServiceV12) Create() (error, tc.ApiErrorType) {
 	return errors.New("The Create method is not implemented"), http.StatusNotImplemented
 }
 
@@ -214,122 +210,121 @@ func CreateV12() http.HandlerFunc {
 	}
 }
 
-	func(ds *TODeliveryServiceV12) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
+func (ds *TODeliveryServiceV12) Read(params map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
 	returnable := []interface{}{}
 	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
+//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
 
-		// 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 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
 
-		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
-		}
+	// 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
+	}
 
-		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
-		}
+	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
+	}
 
-		paramConfigFilePrefixes := []string{"hdr_rw_", "hdr_rw_mid_", "regex_remap_", "cacheurl_"}
-		configFiles := []string{}
-		for _, prefix := range paramConfigFilePrefixes {
-			configFiles = append(configFiles, prefix + *ds.XMLID+".config")
+	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
+	}
 
-		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
-		}
+	paramConfigFilePrefixes := []string{"hdr_rw_", "hdr_rw_mid_", "regex_remap_", "cacheurl_"}
+	configFiles := []string{}
+	for _, prefix := range paramConfigFilePrefixes {
+		configFiles = append(configFiles, prefix+*ds.XMLID+".config")
+	}
 
-		return nil, tc.NoError
+	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
+}
 
 // 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) {
+func (ds *TODeliveryServiceV12) Update() (error, tc.ApiErrorType) {
 	return errors.New("The Update method is not implemented"), http.StatusNotImplemented
 }
 
 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)
-		return
-	}
-	defer inf.Close()
+		inf, userErr, sysErr, errCode := api.NewInfo(r, []string{"id"}, []string{"id"})
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		defer inf.Close()
 
-	ds := tc.DeliveryServiceNullableV12{}
-	ds.ID = util.IntPtr(inf.IntParams["id"])
-	if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
-		api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
-		return
-	}
-	dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
-	if authorized, err := isTenantAuthorized(inf.User, inf.Tx, &ds); err != nil {
-		api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
-		return
-	} else if !authorized {
-		api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
-		return
-	}
-	dsv13, errCode, userErr, sysErr = update(inf.Tx.Tx, *inf.Config, inf.User, &dsv13)
-	if userErr != nil || sysErr != nil {
-		api.HandleErr(w, r, errCode, userErr, sysErr)
-		return
+		ds := tc.DeliveryServiceNullableV12{}
+		ds.ID = util.IntPtr(inf.IntParams["id"])
+		if err := api.Parse(r.Body, inf.Tx.Tx, &ds); err != nil {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("decoding: "+err.Error()), nil)
+			return
+		}
+		dsv13 := tc.NewDeliveryServiceNullableV13FromV12(ds)
+		if authorized, err := isTenantAuthorized(inf.User, inf.Tx, &ds); err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("checking tenant: "+err.Error()))
+			return
+		} else if !authorized {
+			api.HandleErr(w, r, http.StatusForbidden, errors.New("not authorized on this tenant"), nil)
+			return
+		}
+		dsv13, errCode, userErr, sysErr = update(inf.Tx.Tx, *inf.Config, inf.User, &dsv13)
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		*inf.CommitTx = true
+		api.WriteResp(w, r, []tc.DeliveryServiceNullableV12{dsv13.DeliveryServiceNullableV12})
 	}
-	*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 3eab671..fbd996e 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -21,8 +21,8 @@ package deliveryservice
 
 import (
 	"database/sql"
-	"errors"
 	"encoding/json"
+	"errors"
 	"fmt"
 	"net/http"
 	"strconv"
@@ -50,7 +50,7 @@ type TODeliveryServiceV13 struct {
 }
 
 func (ds *TODeliveryServiceV13) V12() *TODeliveryServiceV12 {
-	return &TODeliveryServiceV12{ReqInfo:ds.ReqInfo, DeliveryServiceNullableV12: ds.DeliveryServiceNullableV12}
+	return &TODeliveryServiceV12{ReqInfo: ds.ReqInfo, DeliveryServiceNullableV12: ds.DeliveryServiceNullableV12}
 }
 
 func (ds TODeliveryServiceV13) MarshalJSON() ([]byte, error) {
@@ -60,8 +60,8 @@ func (ds *TODeliveryServiceV13) UnmarshalJSON(data []byte) error {
 	return json.Unmarshal(data, ds.DeliveryServiceNullableV13)
 }
 
-func GetTypeV13Factory() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer{
+func GetTypeV13Factory() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceV13{reqInfo, tc.DeliveryServiceNullableV13{}}
 		return &toReturn
 	}
@@ -94,7 +94,7 @@ func (ds *TODeliveryServiceV13) Validate() []error {
 }
 
 // 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) {
+func (ds *TODeliveryServiceV13) Create() (error, tc.ApiErrorType) {
 	return errors.New("The Create method is not implemented"), http.StatusNotImplemented
 }
 
@@ -236,7 +236,7 @@ func create(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds tc.Deliver
 	}
 
 	ds.LastUpdated = &lastUpdated
-	if err := api.CreateChangeLogRawErr(api.ApiChange, "Created ds: "+*ds.XMLID+" id: "+strconv.Itoa(*ds.ID), user, tx); err!= nil {
+	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
@@ -378,7 +378,7 @@ func getTypeFromID(id int, tx *sql.Tx) (tc.DSType, error) {
 }
 
 // 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) {
+func (ds *TODeliveryServiceV13) Update() (error, tc.ApiErrorType) {
 	return errors.New("The Update method is not implemented"), http.StatusNotImplemented
 }
 
@@ -536,7 +536,6 @@ func update(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds *tc.Delive
 		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())
@@ -685,7 +684,6 @@ func readGetDeliveryServices(params map[string]string, tx *sqlx.Tx, user *auth.C
 	return dses, nil, tc.NoError
 }
 
-
 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!")
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 50ca589..5887b8f 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TODeliveryServiceRequestComment struct{
+type TODeliveryServiceRequestComment struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.DeliveryServiceRequestCommentNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceRequestComment{reqInfo, tc.DeliveryServiceRequestCommentNullable{}}
 		return &toReturn
 	}
@@ -222,7 +222,7 @@ func (comment *TODeliveryServiceRequestComment) Update() (error, tc.ApiErrorType
 func (comment *TODeliveryServiceRequestComment) Delete() (error, tc.ApiErrorType) {
 
 	var current TODeliveryServiceRequestComment
-	err :=comment.ReqInfo.Tx.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
+	err := comment.ReqInfo.Tx.QueryRowx(selectQuery() + `WHERE dsrc.id=` + strconv.Itoa(*comment.ID)).StructScan(&current)
 	if err != nil {
 		log.Errorf("Error querying DeliveryServiceRequestComments: %v", err)
 		return err, tc.SystemError
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
index 17b436c..fa1de48 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments_test.go
@@ -25,9 +25,9 @@ import (
 	"strings"
 	"testing"
 
+	"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/test"
-	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
 func TestFuncs(t *testing.T) {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index 8ac9f2b..e2bd836 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TODeliveryServiceRequest struct{
+type TODeliveryServiceRequest struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.DeliveryServiceRequestNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}
 		return &toReturn
 	}
@@ -447,14 +447,13 @@ WHERE id=:id`
 ////////////////////////////////////////////////////////////////
 // Assignment change
 
-func GetAssignmentTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetAssignmentTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := deliveryServiceRequestAssignment{TODeliveryServiceRequest{reqInfo, tc.DeliveryServiceRequestNullable{}}}
 		return &toReturn
 	}
 }
 
-
 type deliveryServiceRequestAssignment struct {
 	TODeliveryServiceRequest
 }
@@ -538,12 +537,13 @@ type deliveryServiceRequestStatus struct {
 	TODeliveryServiceRequest
 }
 
-func GetStatusTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+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() (error, tc.ApiErrorType) {
 	// req represents the state the deliveryservice_request is to transition to
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 b69b9b9..d22033a 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests_test.go
@@ -77,14 +77,14 @@ func TestGetDeliveryServiceRequest(t *testing.T) {
 	}}
 
 	expectedErrors := []string{
-		/*
-			`'regionalGeoBlocking' is required`,
-			`'xmlId' cannot contain spaces`,
-			`'dscp' is required`,
-			`'displayName' cannot be blank`,
-			`'geoProvider' is required`,
-			`'typeId' is required`,
-		*/
+	/*
+		`'regionalGeoBlocking' is required`,
+		`'xmlId' cannot contain spaces`,
+		`'dscp' is required`,
+		`'displayName' cannot be blank`,
+		`'geoProvider' is required`,
+		`'typeId' is required`,
+	*/
 	}
 
 	r.SetKeys(map[string]interface{}{"id": 10})
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
index 977a996..2249a43 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
@@ -63,6 +63,5 @@ func (req *TODeliveryServiceRequest) Validate() []error {
 
 	errs = append(errs, e...)
 
-
 	return errs
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index 48406af..177408c 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -713,13 +713,13 @@ func dssSelectQuery() string {
 	return selectStmt
 }
 
-type TODSSDeliveryService struct{
+type TODSSDeliveryService struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.DSSDeliveryService
 }
 
-func GetDSSDeliveryServiceReaderSingleton() func(reqInfo *api.APIInfo)api.Reader {
-	return func(reqInfo *api.APIInfo)api.Reader {
+func GetDSSDeliveryServiceReaderSingleton() func(reqInfo *api.APIInfo) api.Reader {
+	return func(reqInfo *api.APIInfo) api.Reader {
 		toReturn := TODSSDeliveryService{reqInfo, tc.DSSDeliveryService{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go b/traffic_ops/traffic_ops_golang/deliveryservicesregexes/deliveryservicesregexes.go
index b091ea9..7016943 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.CreateChangeLogRaw(api.ApiChange,fmt.Sprintf(`deleted deliveryservice_regex {"ds": %d, "regex": %d}`, dsID, regexID), user, dbx.DB)
+		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/division/divisions.go b/traffic_ops/traffic_ops_golang/division/divisions.go
index fa9d3a3..0e0e7e7 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TODivision struct{
+type TODivision struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.DivisionNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TODivision{reqInfo, tc.DivisionNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/division/divisions_test.go b/traffic_ops/traffic_ops_golang/division/divisions_test.go
index eedcbba..a8d93a8 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions_test.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions_test.go
@@ -75,7 +75,7 @@ func TestGetDivisions(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 	servers, errs, errType := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("getDivisions expected: no errors, actual: %v with error type: %s", errs, errType.String())
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index 73f4a4a..e760660 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -41,13 +41,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOOrigin struct{
+type TOOrigin struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.Origin
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOOrigin{reqInfo, v13.Origin{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/origin/origins_test.go b/traffic_ops/traffic_ops_golang/origin/origins_test.go
index ed5db33..56d1b87 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins_test.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins_test.go
@@ -204,7 +204,7 @@ func TestValidate(t *testing.T) {
 	port := 65535
 	pro := "http"
 	lu := tc.TimeNoMod{Time: time.Now()}
-	c = TOOrigin{Origin:v13.Origin{ID: &id,
+	c = TOOrigin{Origin: v13.Origin{ID: &id,
 		Name:              &nm,
 		DeliveryServiceID: &id,
 		FQDN:              &fqdn,
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters.go b/traffic_ops/traffic_ops_golang/parameter/parameters.go
index a948c01..c0363f9 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters.go
@@ -49,13 +49,13 @@ var (
 )
 
 //we need a type alias to define functions on
-type TOParameter struct{
+type TOParameter struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.ParameterNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOParameter{reqInfo, tc.ParameterNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters_test.go b/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
index 9b465f9..090e70b 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters_test.go
@@ -31,8 +31,8 @@ import (
 
 	"encoding/json"
 
-	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
 )
 
 func getTestParameters() []tc.ParameterNullable {
@@ -95,9 +95,9 @@ func TestGetParameters(t *testing.T) {
 	mock.ExpectQuery("SELECT").WillReturnRows(rows)
 	mock.ExpectCommit()
 	v := map[string]string{"name": "1"}
-	user := auth.CurrentUser{PrivLevel:30}
+	user := auth.CurrentUser{PrivLevel: 30}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false),User:&user}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false), User: &user}
 	pps, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
 		t.Errorf("parameter.Read expected: no errors, actual: %v", errs)
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
index b579a99..8528341 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOPhysLocation struct{
+type TOPhysLocation struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.PhysLocationNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOPhysLocation{reqInfo, tc.PhysLocationNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
index 3cad4ec..bb459c8 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations_test.go
@@ -97,7 +97,7 @@ func TestGetPhysLocations(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	physLocations, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles_test.go b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
index b00ea6c..5ad1b95 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles_test.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles_test.go
@@ -98,7 +98,7 @@ func TestGetProfiles(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"name": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	profiles, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
index 703d40c..e6e3366 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
@@ -42,13 +42,13 @@ const (
 )
 
 //we need a type alias to define functions on
-type TOProfileParameter struct{
+type TOProfileParameter struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.ProfileParameterNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOProfileParameter{reqInfo, v13.ProfileParameterNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
index 013ce3a..fea3c7d 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters_test.go
@@ -83,8 +83,7 @@ func TestGetProfileParameters(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"profile": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
-
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	pps, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
diff --git a/traffic_ops/traffic_ops_golang/region/regions.go b/traffic_ops/traffic_ops_golang/region/regions.go
index b54598c..f963923 100644
--- a/traffic_ops/traffic_ops_golang/region/regions.go
+++ b/traffic_ops/traffic_ops_golang/region/regions.go
@@ -32,13 +32,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TORegion struct{
+type TORegion struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.Region
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TORegion{reqInfo, tc.Region{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/region/regions_test.go b/traffic_ops/traffic_ops_golang/region/regions_test.go
index ad7cb6c..50d26de 100644
--- a/traffic_ops/traffic_ops_golang/region/regions_test.go
+++ b/traffic_ops/traffic_ops_golang/region/regions_test.go
@@ -76,7 +76,7 @@ func TestReadRegions(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"id": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	regions, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
diff --git a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
index 8140292..9a62735 100644
--- a/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
+++ b/traffic_ops/traffic_ops_golang/riaksvc/dsutil.go
@@ -63,7 +63,6 @@ func GetDeliveryServiceSSLKeysObj(xmlID string, version string, tx *sql.Tx, auth
 	return key, found, nil
 }
 
-
 func GetDeliveryServiceSSLKeysObjTx(xmlID string, version string, tx *sql.Tx, authOpts *riak.AuthOptions) (tc.DeliveryServiceSSLKeys, bool, error) {
 	key := tc.DeliveryServiceSSLKeys{}
 	if version == "" {
@@ -94,7 +93,7 @@ func GetDeliveryServiceSSLKeysObjTx(xmlID string, version string, tx *sql.Tx, au
 	return key, found, nil
 }
 
-	func PutDeliveryServiceSSLKeysObj(key tc.DeliveryServiceSSLKeys, tx *sql.Tx, authOpts *riak.AuthOptions) error {
+func PutDeliveryServiceSSLKeysObj(key tc.DeliveryServiceSSLKeys, tx *sql.Tx, authOpts *riak.AuthOptions) error {
 	keyJSON, err := json.Marshal(&key)
 	if err != nil {
 		return errors.New("marshalling key: " + err.Error())
diff --git a/traffic_ops/traffic_ops_golang/role/roles.go b/traffic_ops/traffic_ops_golang/role/roles.go
index 2cc8a37..763ab62 100644
--- a/traffic_ops/traffic_ops_golang/role/roles.go
+++ b/traffic_ops/traffic_ops_golang/role/roles.go
@@ -37,13 +37,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TORole struct{
+type TORole struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.Role
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TORole{reqInfo, v13.Role{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index c270424..6b3b215 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -175,12 +175,12 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 
 		// 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.ReadOnlyHandler(dsserver.GetDSSDeliveryServiceReaderSingleton()),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, `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.ReadOnlyHandler(dsserver.GetDSSDeliveryServiceReaderSingleton()), 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
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index 1ad8eea..7e51ab3 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -39,13 +39,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOServer struct{
+type TOServer struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	v13.ServerNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOServer{reqInfo, v13.ServerNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go b/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
index 2c06d35..7ebc076 100644
--- a/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
+++ b/traffic_ops/traffic_ops_golang/staticdnsentry/staticdnsentry.go
@@ -26,13 +26,13 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 )
 
-type TOStaticDNSEntry struct{
+type TOStaticDNSEntry struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.StaticDNSEntry
 }
 
-func GetReaderSingleton() func(reqInfo *api.APIInfo)api.Reader {
-	return func(reqInfo *api.APIInfo)api.Reader {
+func GetReaderSingleton() func(reqInfo *api.APIInfo) api.Reader {
+	return func(reqInfo *api.APIInfo) api.Reader {
 		toReturn := TOStaticDNSEntry{reqInfo, tc.StaticDNSEntry{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/status/statuses.go b/traffic_ops/traffic_ops_golang/status/statuses.go
index d00aed9..5821a35 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOStatus struct{
+type TOStatus struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.StatusNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOStatus{reqInfo, tc.StatusNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/status/statuses_test.go b/traffic_ops/traffic_ops_golang/status/statuses_test.go
index 7a684f1..d455787 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses_test.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses_test.go
@@ -77,7 +77,7 @@ func TestReadStatuses(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	statuses, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {
diff --git a/traffic_ops/traffic_ops_golang/systeminfo/system_info.go b/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
index 40da18a..22972ab 100644
--- a/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
+++ b/traffic_ops/traffic_ops_golang/systeminfo/system_info.go
@@ -20,16 +20,17 @@ package systeminfo
  */
 
 import (
+	"context"
 	"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"
+
+	"github.com/jmoiron/sqlx"
 )
 
 func Handler(db *sqlx.DB) http.HandlerFunc {
@@ -80,9 +81,9 @@ p.last_updated,
 p.value
 FROM parameter p
 WHERE p.config_file='global'`
-	ctx, cancel := context.WithTimeout(context.Background(), time.Second * 10)
+	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
 	defer cancel()
-	rows, err := db.QueryxContext(ctx,query)
+	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 ea6d5ce..cd8edd3 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -38,9 +38,6 @@ import (
 	"github.com/lib/pq"
 )
 
-// TOTenant provides a local type against which to define methods
-type TOTenant tc.TenantNullable
-
 // DeliveryServiceTenantInfo provides only deliveryservice info needed here
 type DeliveryServiceTenantInfo tc.DeliveryServiceNullable
 
@@ -295,10 +292,18 @@ func IsResourceAuthorizedToUserTx(resourceTenantID int, user *auth.CurrentUser,
 	}
 }
 
-// GetRefType allows shared handlers to decode JSON to the tenant type
-// Part of the Identifier interface
-func GetRefType() *TOTenant {
-	return &TOTenant{}
+
+// TOTenant provides a local type against which to define methods
+type TOTenant struct {
+	ReqInfo *api.APIInfo `json:"-"`
+	tc.TenantNullable
+}
+
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
+		toReturn := TOTenant{reqInfo, tc.TenantNullable{}}
+		return &toReturn
+	}
 }
 
 // GetID wraps the ID member with null checking
diff --git a/traffic_ops/traffic_ops_golang/types/types.go b/traffic_ops/traffic_ops_golang/types/types.go
index f160f8b..ab95214 100644
--- a/traffic_ops/traffic_ops_golang/types/types.go
+++ b/traffic_ops/traffic_ops_golang/types/types.go
@@ -36,13 +36,13 @@ import (
 )
 
 //we need a type alias to define functions on
-type TOType struct{
+type TOType struct {
 	ReqInfo *api.APIInfo `json:"-"`
 	tc.TypeNullable
 }
 
-func GetTypeSingleton() func(reqInfo *api.APIInfo)api.CRUDer {
-	return func(reqInfo *api.APIInfo)api.CRUDer {
+func GetTypeSingleton() func(reqInfo *api.APIInfo) api.CRUDer {
+	return func(reqInfo *api.APIInfo) api.CRUDer {
 		toReturn := TOType{reqInfo, tc.TypeNullable{}}
 		return &toReturn
 	}
diff --git a/traffic_ops/traffic_ops_golang/types/types_test.go b/traffic_ops/traffic_ops_golang/types/types_test.go
index 252bed2..780f893 100644
--- a/traffic_ops/traffic_ops_golang/types/types_test.go
+++ b/traffic_ops/traffic_ops_golang/types/types_test.go
@@ -86,8 +86,7 @@ func TestGetType(t *testing.T) {
 	mock.ExpectCommit()
 	v := map[string]string{"dsId": "1"}
 
-	reqInfo := api.APIInfo{Tx:db.MustBegin(),CommitTx:util.BoolPtr(false)}
-
+	reqInfo := api.APIInfo{Tx: db.MustBegin(), CommitTx: util.BoolPtr(false)}
 
 	types, errs, _ := GetTypeSingleton()(&reqInfo).Read(v)
 	if len(errs) > 0 {


[trafficcontrol] 06/15: fix bug with insert query introduced by conflict

Posted by da...@apache.org.
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 c679b4999ded10eca26e10d6005079376b023e6d
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Tue Jun 26 22:51:58 2018 -0600

    fix bug with insert query introduced by conflict
---
 .../deliveryservice/deliveryservicesv13.go                 | 14 +++++---------
 1 file changed, 5 insertions(+), 9 deletions(-)

diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index 10d3182..3eab671 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -50,7 +50,7 @@ type TODeliveryServiceV13 struct {
 }
 
 func (ds *TODeliveryServiceV13) V12() *TODeliveryServiceV12 {
-	return &TODeliveryServiceV12{DeliveryServiceNullableV12: ds.DeliveryServiceNullableV12}
+	return &TODeliveryServiceV12{ReqInfo:ds.ReqInfo, DeliveryServiceNullableV12: ds.DeliveryServiceNullableV12}
 }
 
 func (ds TODeliveryServiceV13) MarshalJSON() ([]byte, error) {
@@ -119,20 +119,18 @@ func CreateV13() http.HandlerFunc {
 		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)
+		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 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
@@ -140,7 +138,6 @@ func CreateV13() http.HandlerFunc {
 			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)
@@ -160,8 +157,7 @@ func create(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds tc.Deliver
 		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.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 [...]
-
+	resultRows, err := tx.Query(insertQuery(), &ds.Active, &ds.AnonymousBlockingEnabled, &ds.CacheURL, &ds.CCRDNSTTL, &ds.CDNID, &ds.CheckPath, &deepCachingType, &ds.DisplayName, &ds.DNSBypassCNAME, &ds.DNSBypassIP, &ds.DNSBypassIP6, &ds.DNSBypassTTL, &ds.DSCP, &ds.EdgeHeaderRewrite, &ds.GeoLimitRedirectURL, &ds.GeoLimit, &ds.GeoLimitCountries, &ds.GeoProvider, &ds.GlobalMaxMBPS, &ds.GlobalMaxTPS, &ds.FQPacingRate, &ds.HTTPBypassFQDN, &ds.InfoURL, &ds.InitialDispersion, &ds.IPV6RoutingEnabl [...]
 	if err != nil {
 		if pqerr, ok := err.(*pq.Error); ok {
 			err, _ := dbhelpers.ParsePQUniqueConstraintError(pqerr)
@@ -575,7 +571,7 @@ func update(tx *sql.Tx, cfg config.Config, user *auth.CurrentUser, ds *tc.Delive
 //The DeliveryService implementation of the Deleter interface
 //all implementations of Deleter should use transactions and return the proper errorType
 func (ds *TODeliveryServiceV13) Delete() (error, tc.ApiErrorType) {
-	return GetTypeV12Factory()(ds.ReqInfo).Delete()
+	return ds.V12().Delete()
 }
 
 // IsTenantAuthorized implements the Tenantable interface to ensure the user is authorized on the deliveryservice tenant


[trafficcontrol] 12/15: convert Validate functions to return a single error with all validation issues

Posted by da...@apache.org.
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 8fe0a128446c5303c2dba1f8787c4160ff3cb4b3
Author: Dylan Volz <Dy...@comcast.com>
AuthorDate: Mon Jul 2 07:13:18 2018 -0600

    convert Validate functions to return a single error with all validation issues
---
 lib/go-tc/deliveryservices.go                           | 12 ++++++------
 lib/go-tc/parameters.go                                 | 12 ++++++------
 traffic_ops/traffic_ops_golang/api/api.go               |  6 +++---
 traffic_ops/traffic_ops_golang/api/shared_handlers.go   | 16 ++++++++--------
 .../traffic_ops_golang/api/shared_handlers_test.go      |  2 +-
 traffic_ops/traffic_ops_golang/api/shared_interfaces.go |  2 +-
 traffic_ops/traffic_ops_golang/asn/asns.go              |  5 +++--
 .../traffic_ops_golang/cachegroup/cachegroups.go        |  5 +++--
 traffic_ops/traffic_ops_golang/cdn/cdns.go              |  5 +++--
 .../traffic_ops_golang/coordinate/coordinates.go        |  5 +++--
 .../deliveryservice/deliveryservicesv12.go              |  2 +-
 .../deliveryservice/deliveryservicesv13.go              | 10 +++++-----
 .../deliveryservice/request/comment/comments.go         |  5 +++--
 .../deliveryservice/request/requests.go                 |  4 ++--
 .../deliveryservice/request/validate.go                 |  9 +++++----
 .../deliveryservice/servers/servers.go                  |  5 +++--
 traffic_ops/traffic_ops_golang/division/divisions.go    |  5 +++--
 traffic_ops/traffic_ops_golang/origin/origins.go        |  5 +++--
 traffic_ops/traffic_ops_golang/parameter/parameters.go  |  5 +++--
 .../traffic_ops_golang/physlocation/phys_locations.go   |  5 +++--
 traffic_ops/traffic_ops_golang/profile/profiles.go      |  5 +++--
 .../profileparameter/profile_parameters.go              |  5 +++--
 traffic_ops/traffic_ops_golang/region/regions.go        |  7 +++----
 traffic_ops/traffic_ops_golang/role/roles.go            |  7 ++++---
 traffic_ops/traffic_ops_golang/server/servers.go        | 17 ++++++++---------
 traffic_ops/traffic_ops_golang/status/statuses.go       |  5 +++--
 traffic_ops/traffic_ops_golang/tenant/tenancy.go        |  4 ++--
 traffic_ops/traffic_ops_golang/types/types.go           |  5 +++--
 28 files changed, 97 insertions(+), 83 deletions(-)

diff --git a/lib/go-tc/deliveryservices.go b/lib/go-tc/deliveryservices.go
index ae34729..50983e8 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 toErrs
+		return util.JoinErrs(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 v12Errs := ds.DeliveryServiceNullableV12.Validate(tx); len(v12Errs) > 0 {
-		errs = append(errs, v12Errs...)
+	if v12Err := ds.DeliveryServiceNullableV12.Validate(tx); v12Err != nil {
+		errs = append(errs, v12Err)
 	}
 	if len(errs) == 0 {
 		return nil
 	}
-	return errs // don't add context, so versions chain well
+	return util.JoinErrs(errs) // don't add context, so versions chain well
 }
 
 // Value implements the driver.Valuer interface
diff --git a/lib/go-tc/parameters.go b/lib/go-tc/parameters.go
index 1151f30..afa6103 100644
--- a/lib/go-tc/parameters.go
+++ b/lib/go-tc/parameters.go
@@ -121,7 +121,7 @@ func (pp *ProfileParametersByNamePost) UnmarshalJSON(bts []byte) error {
 	return nil
 }
 
-func (pp *ProfileParametersByNamePost) Validate(tx *sql.Tx) []error {
+func (pp *ProfileParametersByNamePost) Validate(tx *sql.Tx) error {
 	errs := []error{}
 	ppArr := ([]ProfileParameterByNamePost)(*pp)
 	for i, profileParam := range ppArr {
@@ -132,7 +132,7 @@ func (pp *ProfileParametersByNamePost) Validate(tx *sql.Tx) []error {
 		}
 	}
 	if len(errs) > 0 {
-		return errs
+		return util.JoinErrs(errs)
 	}
 	return nil
 }
@@ -160,7 +160,7 @@ func (pp *PostProfileParam) Sanitize(tx *sql.Tx) {
 	}
 }
 
-func (pp *PostProfileParam) Validate(tx *sql.Tx) []error {
+func (pp *PostProfileParam) Validate(tx *sql.Tx) error {
 	pp.Sanitize(tx)
 	errs := []error{}
 	if pp.ProfileID == nil {
@@ -178,7 +178,7 @@ func (pp *PostProfileParam) Validate(tx *sql.Tx) []error {
 		errs = append(errs, errors.New(fmt.Sprintf("parameters with IDs %v don't all exist", *pp.ParamIDs)))
 	}
 	if len(errs) > 0 {
-		return errs
+		return util.JoinErrs(errs)
 	}
 	return nil
 }
@@ -195,7 +195,7 @@ func (pp *PostParamProfile) Sanitize(tx *sql.Tx) {
 	}
 }
 
-func (pp *PostParamProfile) Validate(tx *sql.Tx) []error {
+func (pp *PostParamProfile) Validate(tx *sql.Tx) error {
 	pp.Sanitize(tx)
 
 	errs := []error{}
@@ -214,7 +214,7 @@ func (pp *PostParamProfile) Validate(tx *sql.Tx) []error {
 		errs = append(errs, errors.New(fmt.Sprintf("profiles with IDs %v don't all exist", *pp.ProfileIDs)))
 	}
 	if len(errs) > 0 {
-		return errs
+		return util.JoinErrs(errs)
 	}
 	return nil
 }
diff --git a/traffic_ops/traffic_ops_golang/api/api.go b/traffic_ops/traffic_ops_golang/api/api.go
index ec3f9f9..d67d914 100644
--- a/traffic_ops/traffic_ops_golang/api/api.go
+++ b/traffic_ops/traffic_ops_golang/api/api.go
@@ -220,7 +220,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.
@@ -229,8 +229,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 errs := v.Validate(tx); len(errs) > 0 {
-		return errors.New("validating: " + util.JoinErrs(errs).Error())
+	if err := v.Validate(tx); err != nil {
+		return errors.New("validating: " + err.Error())
 	}
 	return nil
 }
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index 237cc60..b2b6465 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -106,11 +106,11 @@ 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) []error {
+func decodeAndValidateRequestBody(r *http.Request, v Validator) error {
 	defer r.Body.Close()
 
 	if err := json.NewDecoder(r.Body).Decode(v); err != nil {
-		return []error{err}
+		return err
 	}
 	return v.Validate()
 }
@@ -248,9 +248,9 @@ func UpdateHandler(typeFactory CRUDFactory) 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
-		errs := decodeAndValidateRequestBody(r, u)
-		if len(errs) > 0 {
-			handleErrs(http.StatusBadRequest, errs...)
+		err = decodeAndValidateRequestBody(r, u)
+		if err != nil {
+			handleErrs(http.StatusBadRequest, err)
 			return
 		}
 
@@ -435,10 +435,10 @@ func CreateHandler(typeConstructor CRUDFactory) http.HandlerFunc {
 
 		i := typeConstructor(inf)
 		//decode the body and validate the request struct
-		errs := decodeAndValidateRequestBody(r, i)
+		err := decodeAndValidateRequestBody(r, i)
 
-		if len(errs) > 0 {
-			handleErrs(http.StatusBadRequest, errs...)
+		if err != nil {
+			handleErrs(http.StatusBadRequest, err)
 			return
 		}
 
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
index f234d8a..abf122f 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers_test.go
@@ -74,7 +74,7 @@ func (i *tester) GetAuditName() string {
 }
 
 //Validator interface function
-func (v *tester) Validate() []error {
+func (v *tester) Validate() error {
 	if v.ID < 1 {
 		return []error{errors.New("ID is too low")}
 	}
diff --git a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
index a8df96f..b1e4635 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_interfaces.go
@@ -54,7 +54,7 @@ type Deleter interface {
 }
 
 type Validator interface {
-	Validate() []error
+	Validate() error
 }
 
 type Tenantable interface {
diff --git a/traffic_ops/traffic_ops_golang/asn/asns.go b/traffic_ops/traffic_ops_golang/asn/asns.go
index af59b18..3e790a7 100644
--- a/traffic_ops/traffic_ops_golang/asn/asns.go
+++ b/traffic_ops/traffic_ops_golang/asn/asns.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -86,12 +87,12 @@ func (asn TOASNV11) GetType() string {
 	return "asn"
 }
 
-func (asn TOASNV11) Validate() []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)),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TOASNV11 implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
index 1e12577..c84e329 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/cachegroups.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -146,7 +147,7 @@ func IsValidParentCachegroupID(id *int) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (cg TOCacheGroup) Validate() []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"
@@ -159,7 +160,7 @@ func (cg TOCacheGroup) Validate() []error {
 		"parentCacheGroupID":          validation.Validate(cg.ParentCachegroupID, validation.Min(1)),
 		"secondaryParentCachegroupID": validation.Validate(cg.SecondaryParentCachegroupID, validation.Min(1)),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 // looks up the parent_cachegroup_id and the secondary_cachegroup_id
diff --git a/traffic_ops/traffic_ops_golang/cdn/cdns.go b/traffic_ops/traffic_ops_golang/cdn/cdns.go
index 39e5492..ce67cef 100644
--- a/traffic_ops/traffic_ops_golang/cdn/cdns.go
+++ b/traffic_ops/traffic_ops_golang/cdn/cdns.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -104,14 +105,14 @@ func IsValidCDNName(str string) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (cdn TOCDN) Validate() []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{
 		"name":       validation.Validate(cdn.Name, validation.Required, validName),
 		"domainName": validation.Validate(cdn.DomainName, validation.Required, validDomainName),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TOCDN implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index 3b978ac..e21dea4 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -104,7 +105,7 @@ func IsValidCoordinateName(str string) bool {
 }
 
 // Validate fulfills the api.Validator interface
-func (coordinate TOCoordinate) Validate() []error {
+func (coordinate TOCoordinate) Validate() error {
 	validName := validation.NewStringRule(IsValidCoordinateName, "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"
@@ -113,7 +114,7 @@ func (coordinate TOCoordinate) Validate() []error {
 		"latitude":  validation.Validate(coordinate.Latitude, validation.Min(-90.0).Error(latitudeErr), validation.Max(90.0).Error(latitudeErr)),
 		"longitude": validation.Validate(coordinate.Longitude, validation.Min(-180.0).Error(longitudeErr), validation.Max(180.0).Error(longitudeErr)),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TOCoordinate implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
index cd16e7e..e1876bb 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv12.go
@@ -170,7 +170,7 @@ func isTenantAuthorized(user *auth.CurrentUser, tx *sqlx.Tx, ds *tc.DeliveryServ
 	return true, nil
 }
 
-func (ds *TODeliveryServiceV12) Validate() []error {
+func (ds *TODeliveryServiceV12) Validate() error {
 	return ds.DeliveryServiceNullableV12.Validate(ds.ReqInfo.Tx.Tx)
 }
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
index d6170da..d7b0f7a 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/deliveryservicesv13.go
@@ -89,7 +89,7 @@ func (ds *TODeliveryServiceV13) GetType() string {
 	return ds.V12().GetType()
 }
 
-func (ds *TODeliveryServiceV13) Validate() []error {
+func (ds *TODeliveryServiceV13) Validate() error {
 	return ds.DeliveryServiceNullableV13.Validate(ds.ReqInfo.Tx.Tx)
 }
 
@@ -127,8 +127,8 @@ func CreateV13() http.HandlerFunc {
 		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)
+		if err := ds.Validate(inf.Tx.Tx); err != nil {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+err.Error()), nil)
 			return
 		}
 		if authorized, err := isTenantAuthorized(inf.User, inf.Tx, &ds.DeliveryServiceNullableV12); err != nil {
@@ -401,8 +401,8 @@ func UpdateV13() http.HandlerFunc {
 		}
 		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)
+		if err := ds.Validate(inf.Tx.Tx); err != nil {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("invalid request: "+err.Error()), nil)
 			return
 		}
 
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 3818266..cd5972c 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/comment/comments.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -76,12 +77,12 @@ func (comment TODeliveryServiceRequestComment) GetType() string {
 	return "deliveryservice_request_comment"
 }
 
-func (comment TODeliveryServiceRequestComment) Validate() []error {
+func (comment TODeliveryServiceRequestComment) Validate() error {
 	errs := validation.Errors{
 		"deliveryServiceRequestId": validation.Validate(comment.DeliveryServiceRequestID, validation.NotNil),
 		"value":                    validation.Validate(comment.Value, validation.NotNil),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 func (comment *TODeliveryServiceRequestComment) Create() (error, tc.ApiErrorType) {
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
index 45c0ff9..9dccd08 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/requests.go
@@ -514,7 +514,7 @@ func (req *deliveryServiceRequestAssignment) Update() (error, tc.ApiErrorType) {
 	return nil, tc.NoError
 }
 
-func (req deliveryServiceRequestAssignment) Validate() []error {
+func (req deliveryServiceRequestAssignment) Validate() error {
 	return nil
 }
 
@@ -596,7 +596,7 @@ func (req *deliveryServiceRequestStatus) Update() (error, tc.ApiErrorType) {
 }
 
 // Validate is not needed when only Status is updated
-func (req deliveryServiceRequestStatus) Validate() []error {
+func (req deliveryServiceRequestStatus) Validate() error {
 	return nil
 }
 
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
index 2249a43..c9ce200 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/request/validate.go
@@ -26,18 +26,19 @@ import (
 
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 
 	"github.com/go-ozzo/ozzo-validation"
 )
 
 // Validate ensures all required fields are present and in correct form.  Also checks request JSON is complete and valid
-func (req *TODeliveryServiceRequest) Validate() []error {
+func (req *TODeliveryServiceRequest) Validate() error {
 	fromStatus := tc.RequestStatusDraft
 	if req.ID != nil && *req.ID > 0 {
 		err := req.ReqInfo.Tx.QueryRow(`SELECT status FROM deliveryservice_request WHERE id=` + strconv.Itoa(*req.ID)).Scan(&fromStatus)
 
 		if err != nil {
-			return []error{err}
+			return err
 		}
 	}
 
@@ -61,7 +62,7 @@ func (req *TODeliveryServiceRequest) Validate() []error {
 	// ensure the deliveryservice requested is valid
 	e := req.DeliveryService.Validate(req.ReqInfo.Tx.Tx)
 
-	errs = append(errs, e...)
+	errs = append(errs, e)
 
-	return errs
+	return util.JoinErrs(errs)
 }
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index 177408c..d1e3018 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-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/dbhelpers"
@@ -91,14 +92,14 @@ func (dss *TODeliveryServiceServer) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (dss *TODeliveryServiceServer) Validate(db *sqlx.DB) []error {
+func (dss *TODeliveryServiceServer) Validate(db *sqlx.DB) error {
 
 	errs := validation.Errors{
 		"deliveryservice": validation.Validate(dss.DeliveryService, validation.Required),
 		"server":          validation.Validate(dss.Server, validation.Required),
 	}
 
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 // ReadDSSHandler list all of the Deliveryservice Servers in response to requests to api/1.1/deliveryserviceserver$
diff --git a/traffic_ops/traffic_ops_golang/division/divisions.go b/traffic_ops/traffic_ops_golang/division/divisions.go
index 52a8e4c..810f6ce 100644
--- a/traffic_ops/traffic_ops_golang/division/divisions.go
+++ b/traffic_ops/traffic_ops_golang/division/divisions.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -79,11 +80,11 @@ func (division TODivision) GetType() string {
 	return "division"
 }
 
-func (division TODivision) Validate() []error {
+func (division TODivision) Validate() error {
 	errs := validation.Errors{
 		"name": validation.Validate(division.Name, validation.NotNil, validation.Required),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TODivision implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/origin/origins.go b/traffic_ops/traffic_ops_golang/origin/origins.go
index 82eb3ea..0693ff5 100644
--- a/traffic_ops/traffic_ops_golang/origin/origins.go
+++ b/traffic_ops/traffic_ops_golang/origin/origins.go
@@ -29,6 +29,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/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/dbhelpers"
@@ -88,7 +89,7 @@ func (origin *TOOrigin) GetType() string {
 	return "origin"
 }
 
-func (origin *TOOrigin) Validate() []error {
+func (origin *TOOrigin) Validate() error {
 
 	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
 	validProtocol := validation.NewStringRule(tovalidate.IsOneOfStringICase("http", "https"), "must be http or https")
@@ -107,7 +108,7 @@ func (origin *TOOrigin) Validate() []error {
 		"protocol":          validation.Validate(origin.Protocol, validation.Required, validProtocol),
 		"tenantId":          validation.Validate(origin.TenantID, validation.Min(1)),
 	}
-	return tovalidate.ToErrors(validateErrs)
+	return util.JoinErrs(tovalidate.ToErrors(validateErrs))
 }
 
 // GetTenantID returns a pointer to the Origin's tenant ID from the Tx and any error encountered
diff --git a/traffic_ops/traffic_ops_golang/parameter/parameters.go b/traffic_ops/traffic_ops_golang/parameter/parameters.go
index e21011d..3217d93 100644
--- a/traffic_ops/traffic_ops_golang/parameter/parameters.go
+++ b/traffic_ops/traffic_ops_golang/parameter/parameters.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-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/dbhelpers"
@@ -93,7 +94,7 @@ func (param *TOParameter) GetType() string {
 }
 
 // Validate fulfills the api.Validator interface
-func (param TOParameter) Validate() []error {
+func (param TOParameter) Validate() error {
 	// Test
 	// - Secure Flag is always set to either 1/0
 	// - Admin rights only
@@ -104,7 +105,7 @@ func (param TOParameter) Validate() []error {
 		ValueQueryParam:      validation.Validate(param.Value, validation.Required),
 	}
 
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TOParameter implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
index b9de171..c2c86ba 100644
--- a/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
+++ b/traffic_ops/traffic_ops_golang/physlocation/phys_locations.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -79,7 +80,7 @@ func (pl *TOPhysLocation) GetType() string {
 	return "physLocation"
 }
 
-func (pl *TOPhysLocation) Validate() []error {
+func (pl *TOPhysLocation) Validate() error {
 	errs := validation.Errors{
 		"address":   validation.Validate(pl.Address, validation.Required),
 		"city":      validation.Validate(pl.City, validation.Required),
@@ -90,7 +91,7 @@ func (pl *TOPhysLocation) Validate() []error {
 		"zip":       validation.Validate(pl.Zip, validation.Required),
 	}
 	if errs != nil {
-		return tovalidate.ToErrors(errs)
+		return util.JoinErrs(tovalidate.ToErrors(errs))
 	}
 	return nil
 }
diff --git a/traffic_ops/traffic_ops_golang/profile/profiles.go b/traffic_ops/traffic_ops_golang/profile/profiles.go
index b3cbaa2..a522eb5 100644
--- a/traffic_ops/traffic_ops_golang/profile/profiles.go
+++ b/traffic_ops/traffic_ops_golang/profile/profiles.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/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/dbhelpers"
@@ -90,7 +91,7 @@ func (prof *TOProfile) GetType() string {
 	return "profile"
 }
 
-func (prof *TOProfile) Validate() []error {
+func (prof *TOProfile) Validate() error {
 	errs := validation.Errors{
 		NameQueryParam:        validation.Validate(prof.Name, validation.Required),
 		DescriptionQueryParam: validation.Validate(prof.Description, validation.Required),
@@ -98,7 +99,7 @@ func (prof *TOProfile) Validate() []error {
 		TypeQueryParam:        validation.Validate(prof.Type, validation.Required),
 	}
 	if errs != nil {
-		return tovalidate.ToErrors(errs)
+		return util.JoinErrs(tovalidate.ToErrors(errs))
 	}
 	return nil
 }
diff --git a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
index 9d3c7b7..2eb33c6 100644
--- a/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
+++ b/traffic_ops/traffic_ops_golang/profileparameter/profile_parameters.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -95,14 +96,14 @@ func (pp *TOProfileParameter) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (pp *TOProfileParameter) Validate() []error {
+func (pp *TOProfileParameter) Validate() error {
 
 	errs := validation.Errors{
 		"profile":   validation.Validate(pp.ProfileID, validation.Required),
 		"parameter": validation.Validate(pp.ParameterID, validation.Required),
 	}
 
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 //The TOProfileParameter implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/region/regions.go b/traffic_ops/traffic_ops_golang/region/regions.go
index a8d074b..abeec54 100644
--- a/traffic_ops/traffic_ops_golang/region/regions.go
+++ b/traffic_ops/traffic_ops_golang/region/regions.go
@@ -66,12 +66,11 @@ func (region *TORegion) GetType() string {
 	return "region"
 }
 
-func (region *TORegion) Validate() []error {
-	errs := []error{}
+func (region *TORegion) Validate() error {
 	if len(region.Name) < 1 {
-		errs = append(errs, errors.New(`Region 'name' is required.`))
+		return errors.New(`Region 'name' is required.`)
 	}
-	return errs
+	return nil
 }
 
 func (region *TORegion) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
diff --git a/traffic_ops/traffic_ops_golang/role/roles.go b/traffic_ops/traffic_ops_golang/role/roles.go
index ba5916f..72c2904 100644
--- a/traffic_ops/traffic_ops_golang/role/roles.go
+++ b/traffic_ops/traffic_ops_golang/role/roles.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -81,7 +82,7 @@ func (role *TORole) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (role TORole) Validate() []error {
+func (role TORole) Validate() error {
 	errs := validation.Errors{
 		"name":        validation.Validate(role.Name, validation.Required),
 		"description": validation.Validate(role.Description, validation.Required),
@@ -94,13 +95,13 @@ func (role TORole) Validate() []error {
 		err := role.ReqInfo.Tx.Select(&badCaps, checkCaps, pq.Array(role.Capabilities))
 		if err != nil {
 			log.Errorf("got error from selecting bad capabilities: %v", err)
-			return []error{tc.DBError}
+			return tc.DBError
 		}
 		if len(badCaps) > 0 {
 			errsToReturn = append(errsToReturn, fmt.Errorf("can not add non-existent capabilities: %v", badCaps))
 		}
 	}
-	return errsToReturn
+	return util.JoinErrs(errsToReturn)
 }
 
 //The TORole implementation of the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/server/servers.go b/traffic_ops/traffic_ops_golang/server/servers.go
index 879d165..91ef619 100644
--- a/traffic_ops/traffic_ops_golang/server/servers.go
+++ b/traffic_ops/traffic_ops_golang/server/servers.go
@@ -28,6 +28,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
 	"github.com/apache/trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/trafficcontrol/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/dbhelpers"
@@ -82,7 +83,7 @@ func (server *TOServer) GetType() string {
 	return "server"
 }
 
-func (server *TOServer) Validate() []error {
+func (server *TOServer) Validate() error {
 
 	noSpaces := validation.NewStringRule(tovalidate.NoSpaces, "cannot contain spaces")
 
@@ -104,22 +105,20 @@ func (server *TOServer) Validate() []error {
 	}
 	errs := tovalidate.ToErrors(validateErrs)
 	if len(errs) > 0 {
-		return errs
+		return util.JoinErrs(errs)
 	}
 
 	rows, err := server.ReqInfo.Tx.Query("select use_in_table from type where id=$1", server.TypeID)
 	if err != nil {
 		log.Error.Printf("could not execute select use_in_table from type: %s\n", err)
-		errs = append(errs, tc.DBError)
-		return errs
+		return tc.DBError
 	}
 	defer rows.Close()
 	var useInTable string
 	for rows.Next() {
 		if err := rows.Scan(&useInTable); err != nil {
 			log.Error.Printf("could not scan use_in_table from type: %s\n", err)
-			errs = append(errs, tc.DBError)
-			return errs
+			return tc.DBError
 		}
 	}
 	if useInTable != "server" {
@@ -130,7 +129,7 @@ func (server *TOServer) Validate() []error {
 	if err != nil {
 		log.Error.Printf("could not execute select cdnID from profile: %s\n", err)
 		errs = append(errs, tc.DBError)
-		return errs
+		return util.JoinErrs(errs)
 	}
 	defer rows.Close()
 	var cdnID int
@@ -138,14 +137,14 @@ func (server *TOServer) Validate() []error {
 		if err := rows.Scan(&cdnID); err != nil {
 			log.Error.Printf("could not scan cdnID from profile: %s\n", err)
 			errs = append(errs, tc.DBError)
-			return errs
+			return util.JoinErrs(errs)
 		}
 	}
 	log.Infof("got cdn id: %d from profile and cdn id: %d from server", cdnID, *server.CDNID)
 	if cdnID != *server.CDNID {
 		errs = append(errs, errors.New(fmt.Sprintf("CDN id '%d' for profile '%d' does not match Server CDN '%d'", cdnID, *server.ProfileID, *server.CDNID)))
 	}
-	return errs
+	return util.JoinErrs(errs)
 }
 
 // ChangeLogMessage implements the api.ChangeLogger interface for a custom log message
diff --git a/traffic_ops/traffic_ops_golang/status/statuses.go b/traffic_ops/traffic_ops_golang/status/statuses.go
index 9c405af..fb36b8f 100644
--- a/traffic_ops/traffic_ops_golang/status/statuses.go
+++ b/traffic_ops/traffic_ops_golang/status/statuses.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -79,11 +80,11 @@ func (status TOStatus) GetType() string {
 	return "status"
 }
 
-func (status TOStatus) Validate() []error {
+func (status TOStatus) Validate() error {
 	errs := validation.Errors{
 		"name": validation.Validate(status.Name, validation.NotNil, validation.Required),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 func (status *TOStatus) Read(parameters map[string]string) ([]interface{}, []error, tc.ApiErrorType) {
diff --git a/traffic_ops/traffic_ops_golang/tenant/tenancy.go b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
index 202dd70..51f41ad 100644
--- a/traffic_ops/traffic_ops_golang/tenant/tenancy.go
+++ b/traffic_ops/traffic_ops_golang/tenant/tenancy.go
@@ -352,13 +352,13 @@ func (ten *TOTenant) SetKeys(keys map[string]interface{}) {
 }
 
 // Validate fulfills the api.Validator interface
-func (ten TOTenant) Validate() []error {
+func (ten TOTenant) Validate() error {
 	errs := validation.Errors{
 		"name":     validation.Validate(ten.Name, validation.Required),
 		"active":   validation.Validate(ten.Active), // only validate it's boolean
 		"parentId": validation.Validate(ten.ParentID, validation.Required, validation.Min(1)),
 	}
-	return tovalidate.ToErrors(errs)
+	return util.JoinErrs(tovalidate.ToErrors(errs))
 }
 
 // Create implements the Creator interface
diff --git a/traffic_ops/traffic_ops_golang/types/types.go b/traffic_ops/traffic_ops_golang/types/types.go
index a8c809f..4493ab7 100644
--- a/traffic_ops/traffic_ops_golang/types/types.go
+++ b/traffic_ops/traffic_ops_golang/types/types.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-log"
 	"github.com/apache/trafficcontrol/lib/go-tc"
 	"github.com/apache/trafficcontrol/lib/go-tc/tovalidate"
+	"github.com/apache/trafficcontrol/lib/go-util"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 
@@ -79,14 +80,14 @@ func (typ *TOType) GetType() string {
 	return "type"
 }
 
-func (typ *TOType) Validate() []error {
+func (typ *TOType) Validate() error {
 	errs := validation.Errors{
 		"name":         validation.Validate(typ.Name, validation.Required),
 		"description":  validation.Validate(typ.Description, validation.Required),
 		"use_in_table": validation.Validate(typ.UseInTable, validation.Required),
 	}
 	if errs != nil {
-		return tovalidate.ToErrors(errs)
+		return util.JoinErrs(tovalidate.ToErrors(errs))
 	}
 	return nil
 }