You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by zr...@apache.org on 2023/06/08 15:04:56 UTC

[trafficcontrol] branch master updated: Change /coordinates timestamps to RFC3339 (#7563)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e52aaa2258 Change /coordinates timestamps to RFC3339 (#7563)
e52aaa2258 is described below

commit e52aaa22580112358d1fbf393f913c56726aa15f
Author: ocket8888 <oc...@apache.org>
AuthorDate: Thu Jun 8 09:04:48 2023 -0600

    Change /coordinates timestamps to RFC3339 (#7563)
    
    * Add/update/correct GoDoc comments
    
    * Add APIv5 representations of Coordinates and related structures
    
    Also added deprecation notices to older types.
    
    * Remove redundant types from map composite literal
    
    * Make receiver names consistent
    
    ...and also remove a super-long copypasta comment noting features of the
    language in which this program is written.
    
    * Add v5 /coordinates GET handler and routing
    
    Also put a deprecation notice on the old, "CRUDer" method
    
    * Add a Stringer method for API version
    
    * Add v5 /coordinates POST handler and routing
    
    Also put a deprecation notice on the old, "CRUDer" method(s)
    
    * Add v5 /coordinates PUT handler and routing
    
    Also put a deprecation notice on the old, "CRUDer" method(s)
    
    * Add v5 /coordinates DELETE handler and routing
    
    * Add a bunch of deprecation notices.
    
    * fix tests
    
    * update alert messages for consistency
    
    * add changelogs
    
    * Update docs
    
    * update client
    
    * fix broken v5 client
    
    * Fix response codes when requesting non-existent Coordinate
    
    * add a utility function for writing a not modified response
    
    * add IMS functionality to GET handler
    
    * add IUS/ETag support to PUT handler
    
    * fix GET IMS not supported when using ordering/pagination
---
 docs/source/api/v5/coordinates.rst                 |  49 ++-
 lib/go-tc/coordinates.go                           |  42 +++
 traffic_ops/testing/api/v5/coordinates_test.go     |  47 +--
 traffic_ops/testing/api/v5/traffic_control_test.go |   2 +-
 traffic_ops/traffic_ops_golang/api/api.go          |  18 +
 traffic_ops/traffic_ops_golang/api/api_test.go     |  12 +
 .../traffic_ops_golang/coordinate/coordinates.go   | 386 +++++++++++++++++++--
 .../coordinate/coordinates_test.go                 |  10 +-
 traffic_ops/traffic_ops_golang/routing/routes.go   |   8 +-
 traffic_ops/v5-client/coordinate.go                |  28 +-
 traffic_ops/v5-client/origin.go                    |   2 +-
 11 files changed, 512 insertions(+), 92 deletions(-)

diff --git a/docs/source/api/v5/coordinates.rst b/docs/source/api/v5/coordinates.rst
index 6f624cad50..dc966fbeab 100644
--- a/docs/source/api/v5/coordinates.rst
+++ b/docs/source/api/v5/coordinates.rst
@@ -56,7 +56,7 @@ Request Structure
 Response Structure
 ------------------
 :id:          Integral, unique, identifier for this coordinate pair
-:lastUpdated: The time and date at which this entry was last updated, in a ``ctime``-like format
+:lastUpdated: The time and date at which this entry was last updated, in :rfc:3339 format
 :latitude:    Latitude of the coordinate
 :longitude:   Longitude of the coordinate
 :name:        The name of the coordinate - typically this just reflects the name of the Cache Group for which the coordinate was created
@@ -82,55 +82,55 @@ Response Structure
 			"name": "from_cachegroup_TRAFFIC_ANALYTICS",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 2,
 			"name": "from_cachegroup_TRAFFIC_OPS",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 3,
 			"name": "from_cachegroup_TRAFFIC_OPS_DB",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 4,
 			"name": "from_cachegroup_TRAFFIC_PORTAL",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 5,
 			"name": "from_cachegroup_TRAFFIC_STATS",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 6,
 			"name": "from_cachegroup_CDN_in_a_Box_Mid",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:04+00"
+			"lastUpdated": "2018-10-24T16:07:04.596321Z"
 		},
 		{
 			"id": 7,
 			"name": "from_cachegroup_CDN_in_a_Box_Edge",
 			"latitude": 38.897663,
 			"longitude": -77.036574,
-			"lastUpdated": "2018-10-24 16:07:05+00"
+			"lastUpdated": "2018-10-24T16:07:05.596321Z"
 		}
 	]}
 
 ``POST``
 ========
-Creates a new coordinate pair
+Creates a new coordinate pair.
 
 :Auth. Required: Yes
 :Roles Required: "admin" or "operations"
@@ -159,7 +159,7 @@ Request Structure
 Response Structure
 ------------------
 :id:          Integral, unique, identifier for the newly created coordinate pair
-:lastUpdated: The time and date at which this entry was last updated, in a ``ctime``-like format
+:lastUpdated: The time and date at which this entry was last updated, in :rfc:3339 format
 :latitude:    Latitude of the newly created coordinate
 :longitude:   Longitude of the newly created coordinate
 :name:        The name of the coordinate
@@ -181,7 +181,7 @@ Response Structure
 
 	{ "alerts": [
 		{
-			"text": "coordinate was created.",
+			"text": "Coordinate 'test' (#9) created",
 			"level": "success"
 		}
 	],
@@ -190,7 +190,7 @@ Response Structure
 		"name": "test",
 		"latitude": 0,
 		"longitude": 0,
-		"lastUpdated": "2018-11-15 17:48:55+00"
+		"lastUpdated": "2018-11-15T17:48:55.596321Z"
 	}}
 
 
@@ -233,7 +233,7 @@ Request Structure
 Response Structure
 ------------------
 :id:          Integral, unique, identifier for the coordinate pair
-:lastUpdated: The time and date at which this entry was last updated, in a ``ctime``-like format
+:lastUpdated: The time and date at which this entry was last updated, in :rfc:3339 format
 :latitude:    Latitude of the coordinate
 :longitude:   Longitude of the coordinate
 :name:        The name of the coordinate
@@ -255,7 +255,7 @@ Response Structure
 
 	{ "alerts": [
 		{
-			"text": "coordinate was updated.",
+			"text": "Coordinate 'quest' (#9) updated",
 			"level": "success"
 		}
 	],
@@ -264,7 +264,7 @@ Response Structure
 		"name": "quest",
 		"latitude": 0,
 		"longitude": 0,
-		"lastUpdated": "2018-11-15 17:54:30+00"
+		"lastUpdated": "2018-11-15T17:54:30.596321Z"
 	}}
 
 ``DELETE``
@@ -274,10 +274,16 @@ Deletes a coordinate
 :Auth. Required: Yes
 :Roles Required: "admin" or "operations"
 :Permissions Required: COORDINATE:DELETE, COORDINATE:READ
-:Response Type:  ``undefined``
+:Response Type:  Object
 
 Request Structure
 -----------------
+:id:          Integral, unique, identifier for the coordinate pair
+:lastUpdated: The time and date at which this entry was last updated, in :rfc:3339 format
+:latitude:    Latitude of the coordinate
+:longitude:   Longitude of the coordinate
+:name:        The name of the coordinate
+
 .. table:: Request Query Parameters
 
 	+------+----------+-------------------------------------------------------------+
@@ -305,7 +311,14 @@ Response Structure
 
 	{ "alerts": [
 		{
-			"text": "coordinate was deleted.",
+			"text": "Coordinate 'quest' (#9) deleted",
 			"level": "success"
 		}
-	]}
+	],
+		"response": {
+		"id": 9,
+		"name": "quest",
+		"latitude": 0,
+		"longitude": 0,
+		"lastUpdated": "2018-11-15T17:54:30.596321Z"
+	}}
diff --git a/lib/go-tc/coordinates.go b/lib/go-tc/coordinates.go
index 0c86b67eb4..be39d2618e 100644
--- a/lib/go-tc/coordinates.go
+++ b/lib/go-tc/coordinates.go
@@ -19,7 +19,11 @@ package tc
  * under the License.
  */
 
+import "time"
+
 // CoordinatesResponse is a list of Coordinates as a response.
+// Deprecated: In newer API versions, coordinates are represented by the
+// CoordinatesResponseV5 structures.
 // swagger:response CoordinatesResponse
 // in: body
 type CoordinatesResponse struct {
@@ -30,6 +34,8 @@ type CoordinatesResponse struct {
 
 // CoordinateResponse is a single Coordinate response for Update and Create to
 // depict what changed.
+// Deprecated: In newer API versions, coordinates are represented by the
+// CoordinateResponseV5 structures.
 // swagger:response CoordinateResponse
 // in: body
 type CoordinateResponse struct {
@@ -40,6 +46,8 @@ type CoordinateResponse struct {
 
 // Coordinate is a representation of a Coordinate as it relates to the Traffic
 // Ops data model.
+// Deprecated: In newer API versions, coordinates are represented by the
+// CoordinateV5 structures.
 type Coordinate struct {
 
 	// The Coordinate to retrieve
@@ -71,6 +79,8 @@ type Coordinate struct {
 
 // CoordinateNullable is identical to Coordinate except that its fields are
 // reference values, which allows them to be nil.
+// Deprecated: In newer API versions, coordinates are represented by the
+// CoordinateV5 structures.
 type CoordinateNullable struct {
 
 	// The Coordinate to retrieve
@@ -99,3 +109,35 @@ type CoordinateNullable struct {
 	//
 	LastUpdated *TimeNoMod `json:"lastUpdated" db:"last_updated"`
 }
+
+// CoordinateV5 is the representation of a Coordinate used in the latest minor
+// version of APIv5.
+type CoordinateV5 = CoordinateV50
+
+// CoordinateV50 is the representation of a Coordinate used in API v5.0.
+type CoordinateV50 struct {
+	// The integral, unique identifier of a Coordinate.
+	ID *int `json:"id" db:"id"`
+	// The Coordinate's name.
+	Name string `json:"name" db:"name"`
+	// The latitude of the Coordinate.
+	Latitude float64 `json:"latitude" db:"latitude"`
+	// The longitude of the Coordinate.
+	Longitude float64 `json:"longitude" db:"longitude"`
+	// The time and date at which the Coordinate was last modified.
+	LastUpdated time.Time `json:"lastUpdated" db:"last_updated"`
+}
+
+// CoordinateResponseV5 is the type of a response from the /coordinates endpoint
+// in the latest minor version of APIv5.
+type CoordinateResponseV5 struct {
+	Alerts
+	Response CoordinateV5
+}
+
+// CoordinatesResponseV5 is the type of a response from the /coordinates
+// endpoint in the latest minor version of APIv5.
+type CoordinatesResponseV5 struct {
+	Alerts
+	Response []CoordinateV5
+}
diff --git a/traffic_ops/testing/api/v5/coordinates_test.go b/traffic_ops/testing/api/v5/coordinates_test.go
index 2a02a0c428..a73696d5b9 100644
--- a/traffic_ops/testing/api/v5/coordinates_test.go
+++ b/traffic_ops/testing/api/v5/coordinates_test.go
@@ -38,7 +38,7 @@ func TestCoordinates(t *testing.T) {
 		currentTimeRFC := currentTime.Format(time.RFC1123)
 		tomorrow := currentTime.AddDate(0, 0, 1).Format(time.RFC1123)
 
-		methodTests := utils.TestCase[client.Session, client.RequestOptions, tc.Coordinate]{
+		methodTests := utils.TestCase[client.Session, client.RequestOptions, tc.CoordinateV5]{
 			"GET": {
 				"NOT MODIFIED when NO CHANGES made": {
 					ClientSession: TOSession,
@@ -109,7 +109,7 @@ func TestCoordinates(t *testing.T) {
 			"POST": {
 				"BAD REQUEST when INVALID NAME": {
 					ClientSession: TOSession,
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  1.1,
 						Longitude: 2.2,
 						Name:      "",
@@ -118,7 +118,7 @@ func TestCoordinates(t *testing.T) {
 				},
 				"BAD REQUEST when INVALID LATITUDE": {
 					ClientSession: TOSession,
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  20000,
 						Longitude: 2.2,
 						Name:      "testlatitude",
@@ -127,7 +127,7 @@ func TestCoordinates(t *testing.T) {
 				},
 				"BAD REQUEST when INVALID LONGITUDE": {
 					ClientSession: TOSession,
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  1.1,
 						Longitude: 20000,
 						Name:      "testlongitude",
@@ -139,7 +139,7 @@ func TestCoordinates(t *testing.T) {
 				"OK when VALID request": {
 					EndpointID:    GetCoordinateID(t, "coordinate2"),
 					ClientSession: TOSession,
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  7.7,
 						Longitude: 8.8,
 						Name:      "coordinate2",
@@ -149,7 +149,7 @@ func TestCoordinates(t *testing.T) {
 				},
 				"NOT FOUND when INVALID ID parameter": {
 					EndpointID: func() int { return 111111 },
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  1.1,
 						Longitude: 2.2,
 						Name:      "coordinate1",
@@ -161,7 +161,7 @@ func TestCoordinates(t *testing.T) {
 					EndpointID:    GetCoordinateID(t, "coordinate1"),
 					ClientSession: TOSession,
 					RequestOpts:   client.RequestOptions{Header: http.Header{rfc.IfUnmodifiedSince: {currentTimeRFC}}},
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  1.1,
 						Longitude: 2.2,
 						Name:      "coordinate1",
@@ -171,7 +171,7 @@ func TestCoordinates(t *testing.T) {
 				"PRECONDITION FAILED when updating with IFMATCH ETAG Header": {
 					EndpointID:    GetCoordinateID(t, "coordinate1"),
 					ClientSession: TOSession,
-					RequestBody: tc.Coordinate{
+					RequestBody: tc.CoordinateV5{
 						Latitude:  1.1,
 						Longitude: 2.2,
 						Name:      "coordinate1",
@@ -202,23 +202,23 @@ func TestCoordinates(t *testing.T) {
 						})
 					case "POST":
 						t.Run(name, func(t *testing.T) {
-							alerts, reqInf, err := testCase.ClientSession.CreateCoordinate(testCase.RequestBody, testCase.RequestOpts)
+							resp, reqInf, err := testCase.ClientSession.CreateCoordinate(testCase.RequestBody, testCase.RequestOpts)
 							for _, check := range testCase.Expectations {
-								check(t, reqInf, nil, alerts, err)
+								check(t, reqInf, nil, resp.Alerts, err)
 							}
 						})
 					case "PUT":
 						t.Run(name, func(t *testing.T) {
-							alerts, reqInf, err := testCase.ClientSession.UpdateCoordinate(testCase.EndpointID(), testCase.RequestBody, testCase.RequestOpts)
+							resp, reqInf, err := testCase.ClientSession.UpdateCoordinate(testCase.EndpointID(), testCase.RequestBody, testCase.RequestOpts)
 							for _, check := range testCase.Expectations {
-								check(t, reqInf, nil, alerts, err)
+								check(t, reqInf, nil, resp.Alerts, err)
 							}
 						})
 					case "DELETE":
 						t.Run(name, func(t *testing.T) {
-							alerts, reqInf, err := testCase.ClientSession.DeleteCoordinate(testCase.EndpointID(), testCase.RequestOpts)
+							resp, reqInf, err := testCase.ClientSession.DeleteCoordinate(testCase.EndpointID(), testCase.RequestOpts)
 							for _, check := range testCase.Expectations {
-								check(t, reqInf, nil, alerts, err)
+								check(t, reqInf, nil, resp.Alerts, err)
 							}
 						})
 					}
@@ -231,7 +231,7 @@ func TestCoordinates(t *testing.T) {
 func validateCoordinateFields(expectedResp map[string]interface{}) utils.CkReqFunc {
 	return func(t *testing.T, _ toclientlib.ReqInf, resp interface{}, _ tc.Alerts, _ error) {
 		assert.RequireNotNil(t, resp, "Expected Coordinate response to not be nil.")
-		coordinateResp := resp.([]tc.Coordinate)
+		coordinateResp := resp.([]tc.CoordinateV5)
 		for field, expected := range expectedResp {
 			for _, coordinate := range coordinateResp {
 				switch field {
@@ -262,7 +262,7 @@ func validateCoordinateUpdateCreateFields(name string, expectedResp map[string]i
 
 func validateCoordinatePagination(paginationParam string) utils.CkReqFunc {
 	return func(t *testing.T, _ toclientlib.ReqInf, resp interface{}, _ tc.Alerts, _ error) {
-		paginationResp := resp.([]tc.Coordinate)
+		paginationResp := resp.([]tc.CoordinateV5)
 		opts := client.NewRequestOptions()
 		opts.QueryParameters.Set("orderby", "id")
 		respBase, _, err := TOSession.GetCoordinates(opts)
@@ -285,7 +285,7 @@ func validateCoordinateSort() utils.CkReqFunc {
 	return func(t *testing.T, _ toclientlib.ReqInf, resp interface{}, alerts tc.Alerts, _ error) {
 		assert.RequireNotNil(t, resp, "Expected Coordinate response to not be nil.")
 		var coordinateNames []string
-		coordinateResp := resp.([]tc.Coordinate)
+		coordinateResp := resp.([]tc.CoordinateV5)
 		for _, coordinate := range coordinateResp {
 			coordinateNames = append(coordinateNames, coordinate.Name)
 		}
@@ -296,7 +296,7 @@ func validateCoordinateSort() utils.CkReqFunc {
 func validateCoordinateDescSort() utils.CkReqFunc {
 	return func(t *testing.T, _ toclientlib.ReqInf, resp interface{}, alerts tc.Alerts, _ error) {
 		assert.RequireNotNil(t, resp, "Expected Coordinate response to not be nil.")
-		coordinateDescResp := resp.([]tc.Coordinate)
+		coordinateDescResp := resp.([]tc.CoordinateV5)
 		var descSortedList []string
 		var ascSortedList []string
 		assert.RequireGreaterOrEqual(t, len(coordinateDescResp), 2, "Need at least 2 Coordinates in Traffic Ops to test desc sort, found: %d", len(coordinateDescResp))
@@ -324,7 +324,9 @@ func GetCoordinateID(t *testing.T, coordinateName string) func() int {
 		coordinatesResp, _, err := TOSession.GetCoordinates(opts)
 		assert.RequireNoError(t, err, "Get Coordinate Request failed with error:", err)
 		assert.RequireEqual(t, 1, len(coordinatesResp.Response), "Expected response object length 1, but got %d", len(coordinatesResp.Response))
-		return coordinatesResp.Response[0].ID
+		id := coordinatesResp.Response[0].ID
+		assert.RequireNotNil(t, id, "Traffic Ops responded with nil Coordinate ID")
+		return *id
 	}
 }
 
@@ -339,11 +341,14 @@ func DeleteTestCoordinates(t *testing.T) {
 	coordinates, _, err := TOSession.GetCoordinates(client.RequestOptions{})
 	assert.NoError(t, err, "Cannot get Coordinates: %v - alerts: %+v", err, coordinates.Alerts)
 	for _, coordinate := range coordinates.Response {
-		alerts, _, err := TOSession.DeleteCoordinate(coordinate.ID, client.RequestOptions{})
+		id := coordinate.ID
+		assert.RequireNotNil(t, id, "Traffic Ops responded with nil Coordinate ID")
+
+		alerts, _, err := TOSession.DeleteCoordinate(*id, client.RequestOptions{})
 		assert.NoError(t, err, "Unexpected error deleting Coordinate '%s' (#%d): %v - alerts: %+v", coordinate.Name, coordinate.ID, err, alerts.Alerts)
 		// Retrieve the Coordinate to see if it got deleted
 		opts := client.NewRequestOptions()
-		opts.QueryParameters.Set("id", strconv.Itoa(coordinate.ID))
+		opts.QueryParameters.Set("id", strconv.Itoa(*id))
 		getCoordinate, _, err := TOSession.GetCoordinates(opts)
 		assert.NoError(t, err, "Error getting Coordinate '%s' after deletion: %v - alerts: %+v", coordinate.Name, err, getCoordinate.Alerts)
 		assert.Equal(t, 0, len(getCoordinate.Response), "Expected Coordinate '%s' to be deleted, but it was found in Traffic Ops", coordinate.Name)
diff --git a/traffic_ops/testing/api/v5/traffic_control_test.go b/traffic_ops/testing/api/v5/traffic_control_test.go
index bd18b2b446..5fd0eea97f 100644
--- a/traffic_ops/testing/api/v5/traffic_control_test.go
+++ b/traffic_ops/testing/api/v5/traffic_control_test.go
@@ -26,7 +26,7 @@ type TrafficControl struct {
 	CDNLocks                                          []tc.CDNLock                            `json:"cdnlocks"`
 	CacheGroups                                       []tc.CacheGroupNullable                 `json:"cachegroups"`
 	Capabilities                                      []tc.Capability                         `json:"capability"`
-	Coordinates                                       []tc.Coordinate                         `json:"coordinates"`
+	Coordinates                                       []tc.CoordinateV5                       `json:"coordinates"`
 	DeliveryServicesRegexes                           []tc.DeliveryServiceRegexesTest         `json:"deliveryServicesRegexes"`
 	DeliveryServiceRequests                           []tc.DeliveryServiceRequestV5           `json:"deliveryServiceRequests"`
 	DeliveryServiceRequestComments                    []tc.DeliveryServiceRequestComment      `json:"deliveryServiceRequestComments"`
diff --git a/traffic_ops/traffic_ops_golang/api/api.go b/traffic_ops/traffic_ops_golang/api/api.go
index 52c1674434..f8b95f5e65 100644
--- a/traffic_ops/traffic_ops_golang/api/api.go
+++ b/traffic_ops/traffic_ops_golang/api/api.go
@@ -625,6 +625,15 @@ func (inf APIInfo) UseIMS() bool {
 	return inf.Config.UseIMS && inf.request.Header.Get(rfc.IfModifiedSince) != ""
 }
 
+// WriteNotModifiedResponse writes a 304 Not Modified response with the given
+// last modification time to the provided response writer. The request must be
+// provided as well, so that it can be marked as handled.
+func WriteNotModifiedResponse(t time.Time, w http.ResponseWriter, r *http.Request) {
+	AddLastModifiedHdr(w, t)
+	w.WriteHeader(http.StatusNotModified)
+	WriteResp(w, r, nil)
+}
+
 // CheckPrecondition checks a request's "preconditions" - its If-Match and
 // If-Unmodified-Since headers versus the last updated time of the requested
 // object(s), and returns (in order), an HTTP response code appropriate for the
@@ -794,11 +803,20 @@ func (val APIInfoImpl) APIInfo() *APIInfo {
 	return val.ReqInfo
 }
 
+// Version represents an API version.
 type Version struct {
 	Major uint64
 	Minor uint64
 }
 
+// String implements the fmt.Stringer interface.
+func (v *Version) String() string {
+	if v == nil {
+		return "{{null}}"
+	}
+	return strconv.FormatUint(v.Major, 10) + "." + strconv.FormatUint(v.Minor, 10)
+}
+
 func (v *Version) LessThan(otherVersion *Version) bool {
 	return v.Major < otherVersion.Major || (v.Major == otherVersion.Major && v.Minor < otherVersion.Minor)
 }
diff --git a/traffic_ops/traffic_ops_golang/api/api_test.go b/traffic_ops/traffic_ops_golang/api/api_test.go
index 2a08ee8e28..3118e6d6b5 100644
--- a/traffic_ops/traffic_ops_golang/api/api_test.go
+++ b/traffic_ops/traffic_ops_golang/api/api_test.go
@@ -24,6 +24,7 @@ import (
 	"database/sql"
 	"encoding/json"
 	"errors"
+	"fmt"
 	"net/http"
 	"net/url"
 	"testing"
@@ -33,6 +34,17 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-tc"
 )
 
+func ExampleVersion_String() {
+	// Because api.Info objects use pointers to Versions, this handles nil
+	// without needing the caller to do it - because that's annoying.
+	var v *Version
+	fmt.Println(v)
+	v = &Version{Major: 4, Minor: 20}
+	fmt.Println(v.String())
+	// Output: {{null}}
+	// 4.20
+}
+
 func TestCamelCase(t *testing.T) {
 	testStrings := []string{"hello_world", "trailing_underscore_", "w_h_a_t____"}
 	expected := []string{"helloWorld", "trailingUnderscore", "wHAT"}
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
index 5990fad907..06078b5596 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates.go
@@ -1,3 +1,5 @@
+// Package coordinate contains API handlers and associated logic for servicing
+// the `/coordinates` API endpoint.
 package coordinate
 
 /*
@@ -20,48 +22,92 @@ package coordinate
  */
 
 import (
+	"database/sql"
+	"encoding/json"
+	"errors"
+	"fmt"
 	"net/http"
 	"strconv"
 	"strings"
 	"time"
 
+	"github.com/apache/trafficcontrol/lib/go-log"
+	"github.com/apache/trafficcontrol/lib/go-rfc"
 	"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"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/util/ims"
 
 	validation "github.com/go-ozzo/ozzo-validation"
 )
 
-// we need a type alias to define functions on
+// TOCoordinate is a "CRUDer"-based API wrapper for Coordinate objects.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 type TOCoordinate struct {
 	api.APIInfoImpl `json:"-"`
 	tc.CoordinateNullable
 }
 
-func (v *TOCoordinate) SetLastUpdated(t tc.TimeNoMod) { v.LastUpdated = &t }
-func (v *TOCoordinate) InsertQuery() string           { return insertQuery() }
-func (v *TOCoordinate) NewReadObj() interface{}       { return &tc.CoordinateNullable{} }
-func (v *TOCoordinate) SelectQuery() string           { return selectQuery() }
-func (v *TOCoordinate) ParamColumns() map[string]dbhelpers.WhereColumnInfo {
+// SetLastUpdated implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (coordinate *TOCoordinate) SetLastUpdated(t tc.TimeNoMod) { coordinate.LastUpdated = &t }
+
+// InsertQuery implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) InsertQuery() string { return insertQuery() }
+
+// NewReadObj implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) NewReadObj() interface{} { return &tc.CoordinateNullable{} }
+
+// SelectQuery implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) SelectQuery() string { return selectQuery() }
+
+// ParamColumns implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) ParamColumns() map[string]dbhelpers.WhereColumnInfo {
 	return map[string]dbhelpers.WhereColumnInfo{
-		"id":   dbhelpers.WhereColumnInfo{Column: "id", Checker: api.IsInt},
-		"name": dbhelpers.WhereColumnInfo{Column: "name"},
+		"id":   {Column: "id", Checker: api.IsInt},
+		"name": {Column: "name"},
 	}
 }
 
-func (v *TOCoordinate) GetLastUpdated() (*time.Time, bool, error) {
-	return api.GetLastUpdated(v.APIInfo().Tx, *v.ID, "coordinate")
+// GetLastUpdated implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (coordinate *TOCoordinate) GetLastUpdated() (*time.Time, bool, error) {
+	return api.GetLastUpdated(coordinate.APIInfo().Tx, *coordinate.ID, "coordinate")
 }
 
-func (v *TOCoordinate) UpdateQuery() string { return updateQuery() }
-func (v *TOCoordinate) DeleteQuery() string { return deleteQuery() }
+// UpdateQuery implements a "CRUD"er interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) UpdateQuery() string { return updateQuery() }
+
+// DeleteQuery implements a "CRUD"er interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) DeleteQuery() string { return deleteQuery() }
+
+// GetKeyFieldsInfo implements a "CRUD"er interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func (coordinate TOCoordinate) GetKeyFieldsInfo() []api.KeyFieldInfo {
 	return []api.KeyFieldInfo{{Field: "id", Func: api.GetIntKey}}
 }
 
-// Implementation of the Identifier, Validator interface functions
+// GetKeys implements the Identifier and Validator interfaces.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func (coordinate TOCoordinate) GetKeys() (map[string]interface{}, bool) {
 	if coordinate.ID == nil {
 		return map[string]interface{}{"id": 0}, false
@@ -69,6 +115,9 @@ func (coordinate TOCoordinate) GetKeys() (map[string]interface{}, bool) {
 	return map[string]interface{}{"id": *coordinate.ID}, true
 }
 
+// GetAuditName implements a "CRUD"er interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func (coordinate TOCoordinate) GetAuditName() string {
 	if coordinate.Name != nil {
 		return *coordinate.Name
@@ -79,12 +128,18 @@ func (coordinate TOCoordinate) GetAuditName() string {
 	return "0"
 }
 
+// GetType implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func (coordinate TOCoordinate) GetType() string {
 	return "coordinate"
 }
 
+// SetKeys implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func (coordinate *TOCoordinate) 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.
+	i, _ := keys["id"].(int)
 	coordinate.ID = &i
 }
 
@@ -104,13 +159,18 @@ func isValidCoordinateChar(r rune) bool {
 	return false
 }
 
-// IsValidCoordinateName returns true if the name contains only characters valid for a Coordinate name
+// IsValidCoordinateName returns true if the name contains only characters valid
+// for a Coordinate name.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
 func IsValidCoordinateName(str string) bool {
 	i := strings.IndexFunc(str, func(r rune) bool { return !isValidCoordinateChar(r) })
 	return i == -1
 }
 
 // Validate fulfills the api.Validator interface.
+// Deprecated: All future Coordinate versions should use non-"CRUDer"
+// validation.
 func (coordinate TOCoordinate) Validate() (error, error) {
 	validName := validation.NewStringRule(IsValidCoordinateName, "invalid characters found - Use alphanumeric . or - or _ .")
 	latitudeErr := "Must be a floating point number within the range +-90"
@@ -123,35 +183,77 @@ func (coordinate TOCoordinate) Validate() (error, error) {
 	return util.JoinErrs(tovalidate.ToErrors(errs)), nil
 }
 
+// Create implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer" Create
+// function.
 func (coord *TOCoordinate) Create() (error, error, int) { return api.GenericCreate(coord) }
+
+// Read implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer" Read
+// function.
 func (coord *TOCoordinate) Read(h http.Header, useIMS bool) ([]interface{}, error, error, int, *time.Time) {
 	api.DefaultSort(coord.APIInfo(), "name")
 	return api.GenericRead(h, coord, useIMS)
 }
-func (v *TOCoordinate) SelectMaxLastUpdatedQuery(where, orderBy, pagination, tableName string) string {
-	return `SELECT max(t) from (
-		SELECT max(last_updated) as t from ` + tableName + ` c ` + where + orderBy + pagination +
-		` UNION ALL
-	select max(last_updated) as t from last_deleted l where l.table_name='` + tableName + `') as res`
+
+func selectMaxLastUpdatedQuery(where, orderBy, pagination string) string {
+	return `
+SELECT max(t) FROM (
+	SELECT max(last_updated) AS t
+	FROM (
+		SELECT *
+		FROM coordinate c
+		` + where + orderBy + pagination +
+		`	) AS coords
+	UNION ALL
+	SELECT max(last_updated) AS t
+	FROM last_deleted l
+	WHERE l.table_name='coordinate'
+) AS res`
+}
+
+// SelectMaxLastUpdatedQuery implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (*TOCoordinate) SelectMaxLastUpdatedQuery(where, orderBy, pagination, _ string) string {
+	return selectMaxLastUpdatedQuery(where, orderBy, pagination)
 }
 
+// Update implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer" Update
+// function.
 func (coord *TOCoordinate) Update(h http.Header) (error, error, int) {
 	return api.GenericUpdate(h, coord)
 }
-func (coord *TOCoordinate) Delete() (error, error, int) { return api.GenericDelete(coord) }
 
-func selectQuery() string {
-	query := `SELECT
-id,
-latitude,
-longitude,
-last_updated,
-name
+// Delete implements a "CRUDer" interface.
+// Deprecated: All future Coordinate versions should use the non-"CRUDer"
+// methodology.
+func (coord *TOCoordinate) Delete() (error, error, int) { return api.GenericDelete(coord) }
 
+const readQuery = `
+SELECT
+	id,
+	latitude,
+	longitude,
+	last_updated,
+	name
 FROM coordinate c`
-	return query
+
+func selectQuery() string {
+	return readQuery
 }
 
+const putQuery = `
+UPDATE coordinate
+SET
+	latitude=$1,
+	longitude=$2,
+	name=$3
+WHERE id=$4
+RETURNING
+	last_updated`
+
 func updateQuery() string {
 	query := `UPDATE
 coordinate SET
@@ -162,6 +264,19 @@ WHERE id=:id RETURNING last_updated`
 	return query
 }
 
+const createQuery = `
+INSERT INTO coordinate (
+	latitude,
+	longitude,
+	name
+) VALUES (
+	$1,
+	$2,
+	$3
+) RETURNING
+	id,
+	last_updated`
+
 func insertQuery() string {
 	query := `INSERT INTO coordinate (
 latitude,
@@ -173,6 +288,219 @@ name) VALUES (
 	return query
 }
 
+const delQuery = `
+DELETE FROM coordinate
+WHERE id = $1
+RETURNING
+	latitude,
+	longitude,
+	name,
+	last_updated
+`
+
 func deleteQuery() string {
 	return `DELETE FROM coordinate WHERE id = :id`
 }
+
+// Read is the handler for GET requests made to the /coordinates API (in APIv5
+// and later).
+func Read(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, nil, nil)
+	tx := inf.Tx.Tx
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+
+	cols := map[string]dbhelpers.WhereColumnInfo{
+		"id":   {Column: "c.id", Checker: api.IsInt},
+		"name": {Column: "c.name", Checker: nil},
+	}
+	api.DefaultSort(inf, "name")
+
+	where, orderBy, pagination, queryValues, errs := dbhelpers.BuildWhereAndOrderByAndPagination(inf.Params, cols)
+	if len(errs) > 0 {
+		errCode = http.StatusBadRequest
+		userErr = util.JoinErrs(errs)
+		api.HandleErr(w, r, tx, errCode, userErr, nil)
+		return
+	}
+
+	var maxTime time.Time
+	if inf.UseIMS() {
+		var runSecond bool
+		runSecond, maxTime = ims.TryIfModifiedSinceQuery(inf.Tx, r.Header, queryValues, selectMaxLastUpdatedQuery(where, orderBy, pagination))
+		if !runSecond {
+			log.Debugln("IMS HIT")
+			api.WriteNotModifiedResponse(maxTime, w, r)
+			return
+		}
+		log.Debugln("IMS MISS")
+	} else {
+		log.Debugln("Non IMS request")
+	}
+
+	query := readQuery + where + orderBy + pagination
+	rows, err := inf.Tx.NamedQuery(query, queryValues)
+	if err != nil {
+		api.HandleErr(w, r, tx, http.StatusInternalServerError, nil, fmt.Errorf("querying coordinates: %w", err))
+		return
+	}
+	defer log.Close(rows, "closing coordinate query rows")
+
+	cs := []tc.CoordinateV5{}
+	for rows.Next() {
+		var c tc.CoordinateV5
+		err := rows.Scan(&c.ID, &c.Latitude, &c.Longitude, &c.LastUpdated, &c.Name)
+		if err != nil {
+			api.HandleErr(w, r, tx, http.StatusInternalServerError, nil, fmt.Errorf("scanning a coordinate: %w", err))
+			return
+		}
+		cs = append(cs, c)
+	}
+
+	api.WriteResp(w, r, cs)
+}
+
+// isValid returns an error describing why c isn't a valid Coordinate, or nil if
+// it's actually valid.
+func isValid(c tc.CoordinateV5) 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"
+	errs := validation.Errors{
+		"name":      validation.Validate(c.Name, validation.Required, validName),
+		"latitude":  validation.Validate(c.Latitude, validation.Min(-90.0).Error(latitudeErr), validation.Max(90.0).Error(latitudeErr)),
+		"longitude": validation.Validate(c.Longitude, validation.Min(-180.0).Error(longitudeErr), validation.Max(180.0).Error(longitudeErr)),
+	}
+	return util.JoinErrs(tovalidate.ToErrors(errs))
+}
+
+// Create is the handler for POST requests made to the /coordinates API (in
+// APIv5 and later).
+func Create(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, nil, nil)
+	tx := inf.Tx.Tx
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+
+	var c tc.CoordinateV5
+	err := json.NewDecoder(r.Body).Decode(&c)
+	if err != nil {
+		api.HandleErr(w, r, tx, http.StatusBadRequest, err, nil)
+		return
+	}
+
+	if err = isValid(c); err != nil {
+		api.HandleErr(w, r, tx, http.StatusBadRequest, err, nil)
+		return
+	}
+
+	if err = tx.QueryRow(createQuery, c.Latitude, c.Longitude, c.Name).Scan(&c.ID, &c.LastUpdated); err != nil {
+		userErr, sysErr, errCode = api.ParseDBError(err)
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+
+	w.Header().Set(rfc.Location, fmt.Sprintf("/api/%s/coordinates?id=%d", inf.Version, *c.ID))
+	w.WriteHeader(http.StatusCreated)
+	api.WriteRespAlertObj(w, r, tc.SuccessLevel, fmt.Sprintf("Coordinate '%s' (#%d) created", c.Name, *c.ID), c)
+
+	changeLogMsg := fmt.Sprintf("USER: %s, COORDINATE: %s (#%d), ACTION: %s", inf.User.UserName, c.Name, *c.ID, api.Created)
+	api.CreateChangeLogRawTx(api.ApiChange, changeLogMsg, inf.User, tx)
+}
+
+// Update is the handler for PUT requests made to the /coordinates API (in API
+// v5 and later).
+func Update(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, []string{"id"}, []string{"id"})
+	tx := inf.Tx.Tx
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+
+	var c tc.CoordinateV5
+	err := json.NewDecoder(r.Body).Decode(&c)
+	if err != nil {
+		api.HandleErr(w, r, tx, http.StatusBadRequest, err, nil)
+		return
+	}
+
+	id := inf.IntParams["id"]
+	if c.ID != nil {
+		if *c.ID != id {
+			api.HandleErr(w, r, tx, http.StatusBadRequest, fmt.Errorf("ID mismatch; URI specifies %d but payload is for Coordinate #%d", id, *c.ID), nil)
+			return
+		}
+	} else {
+		c.ID = util.Ptr(id)
+	}
+
+	userErr, sysErr, statusCode := api.CheckIfUnModified(r.Header, inf.Tx, id, "coordinate")
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, tx, statusCode, userErr, sysErr)
+		return
+	}
+
+	if err = isValid(c); err != nil {
+		api.HandleErr(w, r, tx, http.StatusBadRequest, err, nil)
+		return
+	}
+
+	if err = tx.QueryRow(putQuery, c.Latitude, c.Longitude, c.Name, id).Scan(&c.LastUpdated); err != nil {
+		if errors.Is(err, sql.ErrNoRows) {
+			userErr = fmt.Errorf("no such Coordinate: #%d", id)
+			errCode = http.StatusNotFound
+			sysErr = nil
+		} else {
+			userErr, sysErr, errCode = api.ParseDBError(err)
+		}
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+
+	api.WriteRespAlertObj(w, r, tc.SuccessLevel, fmt.Sprintf("Coordinate '%s' (#%d) updated", c.Name, id), c)
+
+	changeLogMsg := fmt.Sprintf("USER: %s, COORDINATE: %s (#%d), ACTION: %s", inf.User.UserName, c.Name, id, api.Updated)
+	api.CreateChangeLogRawTx(api.ApiChange, changeLogMsg, inf.User, tx)
+}
+
+// Delete is the handler for PUT requests made to the /coordinates API (in API
+// v5 and later).
+func Delete(w http.ResponseWriter, r *http.Request) {
+	inf, userErr, sysErr, errCode := api.NewInfo(r, []string{"id"}, []string{"id"})
+	tx := inf.Tx.Tx
+	if userErr != nil || sysErr != nil {
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+	defer inf.Close()
+
+	id := inf.IntParams["id"]
+
+	c := tc.CoordinateV5{
+		ID: util.Ptr(id),
+	}
+	if err := tx.QueryRow(delQuery, id).Scan(&c.Latitude, &c.Longitude, &c.Name, &c.LastUpdated); err != nil {
+		if errors.Is(err, sql.ErrNoRows) {
+			userErr = fmt.Errorf("no such Coordinate: #%d", id)
+			errCode = http.StatusNotFound
+			sysErr = nil
+		} else {
+			userErr, sysErr, errCode = api.ParseDBError(err)
+		}
+		api.HandleErr(w, r, tx, errCode, userErr, sysErr)
+		return
+	}
+
+	api.WriteRespAlertObj(w, r, tc.SuccessLevel, fmt.Sprintf("Coordinate '%s' (#%d) deleted", c.Name, id), c)
+
+	changeLogMsg := fmt.Sprintf("USER: %s, COORDINATE: %s (#%d), ACTION: %s", inf.User.UserName, c.Name, id, api.Deleted)
+	api.CreateChangeLogRawTx(api.ApiChange, changeLogMsg, inf.User, tx)
+}
diff --git a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
index 0fa8bb5971..1a468a13af 100644
--- a/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
+++ b/traffic_ops/traffic_ops_golang/coordinate/coordinates_test.go
@@ -100,16 +100,18 @@ func TestReadCoordinates(t *testing.T) {
 }
 
 func TestFuncs(t *testing.T) {
-	if strings.Index(selectQuery(), "SELECT") != 0 {
+	trim := func(s string) string { return strings.TrimSpace((s)) }
+
+	if !strings.HasPrefix(trim(selectQuery()), "SELECT") {
 		t.Errorf("expected selectQuery to start with SELECT")
 	}
-	if strings.Index(insertQuery(), "INSERT") != 0 {
+	if !strings.HasPrefix(trim(insertQuery()), "INSERT") {
 		t.Errorf("expected insertQuery to start with INSERT")
 	}
-	if strings.Index(updateQuery(), "UPDATE") != 0 {
+	if !strings.HasPrefix(trim(updateQuery()), "UPDATE") {
 		t.Errorf("expected updateQuery to start with UPDATE")
 	}
-	if strings.Index(deleteQuery(), "DELETE") != 0 {
+	if !strings.HasPrefix(trim(deleteQuery()), "DELETE") {
 		t.Errorf("expected deleteQuery to start with DELETE")
 	}
 }
diff --git a/traffic_ops/traffic_ops_golang/routing/routes.go b/traffic_ops/traffic_ops_golang/routing/routes.go
index 8a993a4752..618002e4a6 100644
--- a/traffic_ops/traffic_ops_golang/routing/routes.go
+++ b/traffic_ops/traffic_ops_golang/routing/routes.go
@@ -356,10 +356,10 @@ func Routes(d ServerData) ([]Route, http.Handler, error) {
 		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodGet, Path: `about/?$`, Handler: about.Handler(), RequiredPrivLevel: auth.PrivLevelReadOnly, RequiredPermissions: nil, Authenticated: Authenticated, Middlewares: nil, ID: 431750116631},
 
 		//Coordinates
-		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodGet, Path: `coordinates/?$`, Handler: api.ReadHandler(&coordinate.TOCoordinate{}), RequiredPrivLevel: auth.PrivLevelReadOnly, RequiredPermissions: []string{"COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 49670074531},
-		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodPut, Path: `coordinates/?$`, Handler: api.UpdateHandler(&coordinate.TOCoordinate{}), RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:UPDATE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 46892617431},
-		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodPost, Path: `coordinates/?$`, Handler: api.CreateHandler(&coordinate.TOCoordinate{}), RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:CREATE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 442811215731},
-		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodDelete, Path: `coordinates/?$`, Handler: api.DeleteHandler(&coordinate.TOCoordinate{}), RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:DELETE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 430384988931},
+		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodGet, Path: `coordinates/?$`, Handler: coordinate.Read, RequiredPrivLevel: auth.PrivLevelReadOnly, RequiredPermissions: []string{"COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 49670074531},
+		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodPut, Path: `coordinates/?$`, Handler: coordinate.Update, RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:UPDATE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 46892617431},
+		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodPost, Path: `coordinates/?$`, Handler: coordinate.Create, RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:CREATE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 442811215731},
+		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodDelete, Path: `coordinates/?$`, Handler: coordinate.Delete, RequiredPrivLevel: auth.PrivLevelOperations, RequiredPermissions: []string{"COORDINATE:DELETE", "COORDINATE:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 430384988931},
 
 		//CDN notification
 		{Version: api.Version{Major: 5, Minor: 0}, Method: http.MethodGet, Path: `cdn_notifications/?$`, Handler: cdnnotification.Read, RequiredPrivLevel: auth.PrivLevelReadOnly, RequiredPermissions: []string{"CDN:READ"}, Authenticated: Authenticated, Middlewares: nil, ID: 22212245141},
diff --git a/traffic_ops/v5-client/coordinate.go b/traffic_ops/v5-client/coordinate.go
index 9397938b69..f5c9346ea7 100644
--- a/traffic_ops/v5-client/coordinate.go
+++ b/traffic_ops/v5-client/coordinate.go
@@ -27,38 +27,38 @@ import (
 const apiCoordinates = "/coordinates"
 
 // CreateCoordinate creates the given Coordinate.
-func (to *Session) CreateCoordinate(coordinate tc.Coordinate, opts RequestOptions) (tc.Alerts, toclientlib.ReqInf, error) {
-	var alerts tc.Alerts
-	reqInf, err := to.post(apiCoordinates, opts, coordinate, &alerts)
-	return alerts, reqInf, err
+func (to *Session) CreateCoordinate(coordinate tc.CoordinateV5, opts RequestOptions) (tc.CoordinateResponseV5, toclientlib.ReqInf, error) {
+	var resp tc.CoordinateResponseV5
+	reqInf, err := to.post(apiCoordinates, opts, coordinate, &resp)
+	return resp, reqInf, err
 }
 
 // UpdateCoordinate replaces the Coordinate with the given ID with the one
 // provided.
-func (to *Session) UpdateCoordinate(id int, coordinate tc.Coordinate, opts RequestOptions) (tc.Alerts, toclientlib.ReqInf, error) {
+func (to *Session) UpdateCoordinate(id int, coordinate tc.CoordinateV5, opts RequestOptions) (tc.CoordinateResponseV5, toclientlib.ReqInf, error) {
 	if opts.QueryParameters == nil {
 		opts.QueryParameters = url.Values{}
 	}
 	opts.QueryParameters.Set("id", strconv.Itoa(id))
-	var alerts tc.Alerts
-	reqInf, err := to.put(apiCoordinates, opts, coordinate, &alerts)
-	return alerts, reqInf, err
+	var resp tc.CoordinateResponseV5
+	reqInf, err := to.put(apiCoordinates, opts, coordinate, &resp)
+	return resp, reqInf, err
 }
 
 // GetCoordinates returns all Coordinates in Traffic Ops.
-func (to *Session) GetCoordinates(opts RequestOptions) (tc.CoordinatesResponse, toclientlib.ReqInf, error) {
-	var data tc.CoordinatesResponse
+func (to *Session) GetCoordinates(opts RequestOptions) (tc.CoordinatesResponseV5, toclientlib.ReqInf, error) {
+	var data tc.CoordinatesResponseV5
 	reqInf, err := to.get(apiCoordinates, opts, &data)
 	return data, reqInf, err
 }
 
 // DeleteCoordinate deletes the Coordinate with the given ID.
-func (to *Session) DeleteCoordinate(id int, opts RequestOptions) (tc.Alerts, toclientlib.ReqInf, error) {
+func (to *Session) DeleteCoordinate(id int, opts RequestOptions) (tc.CoordinateResponseV5, toclientlib.ReqInf, error) {
 	if opts.QueryParameters == nil {
 		opts.QueryParameters = url.Values{}
 	}
 	opts.QueryParameters.Set("id", strconv.Itoa(id))
-	var alerts tc.Alerts
-	reqInf, err := to.del(apiCoordinates, opts, &alerts)
-	return alerts, reqInf, err
+	var resp tc.CoordinateResponseV5
+	reqInf, err := to.del(apiCoordinates, opts, &resp)
+	return resp, reqInf, err
 }
diff --git a/traffic_ops/v5-client/origin.go b/traffic_ops/v5-client/origin.go
index b735527780..c5f050ed88 100644
--- a/traffic_ops/v5-client/origin.go
+++ b/traffic_ops/v5-client/origin.go
@@ -82,7 +82,7 @@ func (to *Session) originIDs(origin *tc.Origin) error {
 		if len(coordinates.Response) == 0 {
 			return fmt.Errorf("no coordinate with name '%s'", *origin.Coordinate)
 		}
-		origin.CoordinateID = &coordinates.Response[0].ID
+		origin.CoordinateID = coordinates.Response[0].ID
 	}
 
 	if origin.TenantID == nil && origin.Tenant != nil {