You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by ro...@apache.org on 2018/05/07 18:59:12 UTC

[incubator-trafficcontrol] 02/03: added api to DeliveryserviceServer Get in Go

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

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

commit b50539c49e61b7ec189f9b71511166f822e73625
Author: ASchmidt <An...@comcast.com>
AuthorDate: Fri May 4 15:03:15 2018 -0600

    added api to DeliveryserviceServer Get in Go
---
 lib/go-tc/deliveryservice_servers.go               | 240 +++++++--------------
 lib/go-tc/deliveryservices.go                      |  13 --
 .../deliveryservice/servers/servers.go             | 239 ++++++++++++--------
 .../deliveryservice/servers/servers_test.go        |  91 +-------
 traffic_ops/traffic_ops_golang/routes.go           |   6 +
 5 files changed, 230 insertions(+), 359 deletions(-)

diff --git a/lib/go-tc/deliveryservice_servers.go b/lib/go-tc/deliveryservice_servers.go
index 9c7e6ae..bf6a3f2 100644
--- a/lib/go-tc/deliveryservice_servers.go
+++ b/lib/go-tc/deliveryservice_servers.go
@@ -1,5 +1,7 @@
 package tc
 
+import "time"
+
 /*
 
    Licensed under the Apache License, Version 2.0 (the "License");
@@ -15,179 +17,85 @@ package tc
    limitations under the License.
 */
 
-import (
-	"database/sql/driver"
-	"encoding/json"
-	"errors"
-	"fmt"
-	"strconv"
-	"strings"
-
-	log "github.com/apache/incubator-trafficcontrol/lib/go-log"
-)
-
-// IDNoMod type is used to suppress JSON unmarshalling
-type IDNoMod int
-
-// DeliveryServiceRequest is used as part of the workflow to create,
-// modify, or delete a delivery service.
-type DeliveryServiceRequest struct {
-	AssigneeID      int             `json:"assigneeId,omitempty"`
-	Assignee        string          `json:"assignee,omitempty"`
-	AuthorID        IDNoMod         `json:"authorId"`
-	Author          string          `json:"author"`
-	ChangeType      string          `json:"changeType"`
-	CreatedAt       *TimeNoMod      `json:"createdAt"`
-	ID              int             `json:"id"`
-	LastEditedBy    string          `json:"lastEditedBy,omitempty"`
-	LastEditedByID  IDNoMod         `json:"lastEditedById,omitempty"`
-	LastUpdated     *TimeNoMod      `json:"lastUpdated"`
-	DeliveryService DeliveryService `json:"deliveryService"`
-	Status          RequestStatus   `json:"status"`
-	XMLID           string          `json:"-" db:"xml_id"`
-}
-
-// DeliveryServiceRequestNullable is used as part of the workflow to create,
-// modify, or delete a delivery service.
-type DeliveryServiceRequestNullable struct {
-	AssigneeID      *int                     `json:"assigneeId,omitempty" db:"assignee_id"`
-	Assignee        *string                  `json:"assignee,omitempty"`
-	AuthorID        *IDNoMod                 `json:"authorId" db:"author_id"`
-	Author          *string                  `json:"author"`
-	ChangeType      *string                  `json:"changeType" db:"change_type"`
-	CreatedAt       *TimeNoMod               `json:"createdAt" db:"created_at"`
-	ID              *int                     `json:"id" db:"id"`
-	LastEditedBy    *string                  `json:"lastEditedBy"`
-	LastEditedByID  *IDNoMod                 `json:"lastEditedById" db:"last_edited_by_id"`
-	LastUpdated     *TimeNoMod               `json:"lastUpdated" db:"last_updated"`
-	DeliveryService *DeliveryServiceNullable `json:"deliveryService" db:"deliveryservice"`
-	Status          *RequestStatus           `json:"status" db:"status"`
-	XMLID           *string                  `json:"-" db:"xml_id"`
-}
-
-// UnmarshalJSON implements the json.Unmarshaller interface to suppress unmarshalling for IDNoMod
-func (a *IDNoMod) UnmarshalJSON([]byte) error {
-	return nil
-}
-
-// RequestStatus captures where in the workflow this request is
-type RequestStatus string
-
-const (
-	// RequestStatusInvalid -- invalid state
-	RequestStatusInvalid = RequestStatus("invalid")
-	// RequestStatusDraft -- newly created; not ready to be reviewed
-	RequestStatusDraft = RequestStatus("draft")
-	// RequestStatusSubmitted -- newly created; ready to be reviewed
-	RequestStatusSubmitted = RequestStatus("submitted")
-	// RequestStatusRejected -- reviewed, but problems found
-	RequestStatusRejected = RequestStatus("rejected")
-	// RequestStatusPending -- reviewed and locked; ready to be implemented
-	RequestStatusPending = RequestStatus("pending")
-	// RequestStatusComplete -- implemented and locked
-	RequestStatusComplete = RequestStatus("complete")
-)
-
-// RequestStatuses -- user-visible string associated with each of the above
-var RequestStatuses = []RequestStatus{
-	// "invalid" -- don't list here..
-	"draft",
-	"submitted",
-	"rejected",
-	"pending",
-	"complete",
-}
-
-// UnmarshalJSON implements json.Unmarshaller
-func (r *RequestStatus) UnmarshalJSON(b []byte) error {
-	u, err := strconv.Unquote(string(b))
-	if err != nil {
-		return err
-	}
+/*
+# get all delivery services associated with a server (from deliveryservice_server table)
+$r->get( "/api/$version/servers/:id/deliveryservices" => [ id => qr/\d+/ ] )->over( authenticated => 1, not_ldap => 1 )->to( 'Deliveryservice#get_deliveryservices_by_serverId', namespace => $namespace );
+
+# delivery service / server assignments
+$r->post("/api/$version/deliveryservices/:xml_id/servers")->over( authenticated => 1, not_ldap => 1 )
+->to( 'Deliveryservice#assign_servers', namespace => $namespace );
+$r->delete("/api/$version/deliveryservice_server/:dsId/:serverId" => [ dsId => qr/\d+/, serverId => qr/\d+/ ] )->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#remove_server_from_ds', namespace => $namespace );
+	# -- DELIVERYSERVICES: SERVERS
+	# Supports ?orderby=key
+	$r->get("/api/$version/deliveryserviceserver")->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#index', namespace => $namespace );
+	$r->post("/api/$version/deliveryserviceserver")->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#assign_servers_to_ds', namespace => $namespace );
 
-	// just check to see if the string represents a valid requeststatus
-	_, err = RequestStatusFromString(u)
-	if err != nil {
-		return err
-	}
-	return json.Unmarshal(b, (*string)(r))
-}
+*/
 
-// MarshalJSON implements json.Marshaller
-func (r RequestStatus) MarshalJSON() ([]byte, error) {
-	return json.Marshal(string(r))
+// DeliveryServiceServerResponse ...
+type DeliveryServiceServerResponse struct {
+	Response []DeliveryServiceServer `json:"response"`
+	Size     int                     `json:"size"`
+	OrderBy  string                  `json:"orderby"`
+	Limit    int                     `json:"limit"`
 }
 
-// Value implements driver.Valuer
-func (r *RequestStatus) Value() (driver.Value, error) {
-	v, err := json.Marshal(r)
-	log.Debugf("value is %v; err is %v", v, err)
-	v = []byte(strings.Trim(string(v), `"`))
-	return v, err
+// DeliveryServiceServer ...
+type DeliveryServiceServer struct {
+	Server          *int             `json:"server"`
+	DeliveryService *int             `json:"deliveryService"`
+	LastUpdated     *TimeNoMod       `json:"lastUpdated" db:"last_updated"`
 }
 
-// Scan implements sql.Scanner
-func (r *RequestStatus) Scan(src interface{}) error {
-	b, ok := src.([]byte)
-	if !ok {
-		return fmt.Errorf("expected requeststatus in byte array form; got %T", src)
-	}
-	b = []byte(`"` + string(b) + `"`)
-	return json.Unmarshal(b, r)
-}
 
-// RequestStatusFromString gets the status enumeration from a string
-func RequestStatusFromString(rs string) (RequestStatus, error) {
-	if rs == "" {
-		return RequestStatusDraft, nil
-	}
-	for _, s := range RequestStatuses {
-		if string(s) == rs {
-			return s, nil
-		}
-	}
-	return RequestStatusInvalid, errors.New(rs + " is not a valid RequestStatus name")
+type DssServer struct {
+	Cachegroup       *string              `json:"cachegroup" db:"cachegroup"`
+	CachegroupID     *int                 `json:"cachegroupId" db:"cachegroup_id"`
+	CDNID            *int                 `json:"cdnId" db:"cdn_id"`
+	CDNName          *string              `json:"cdnName" db:"cdn_name"`
+	DeliveryServices *map[string][]string `json:"deliveryServices,omitempty"`
+	DomainName       *string              `json:"domainName" db:"domain_name"`
+	FQDN             *string              `json:"fqdn,omitempty"`
+	FqdnTime         time.Time            `json:"-"`
+	GUID             *string              `json:"guid" db:"guid"`
+	HostName         *string              `json:"hostName" db:"host_name"`
+	HTTPSPort        *int                 `json:"httpsPort" db:"https_port"`
+	ID               *int                 `json:"id" db:"id"`
+	ILOIPAddress     *string              `json:"iloIpAddress" db:"ilo_ip_address"`
+	ILOIPGateway     *string              `json:"iloIpGateway" db:"ilo_ip_gateway"`
+	ILOIPNetmask     *string              `json:"iloIpNetmask" db:"ilo_ip_netmask"`
+	ILOPassword      *string              `json:"iloPassword" db:"ilo_password"`
+	ILOUsername      *string              `json:"iloUsername" db:"ilo_username"`
+	InterfaceMtu     *int                 `json:"interfaceMtu" db:"interface_mtu"`
+	InterfaceName    *string              `json:"interfaceName" db:"interface_name"`
+	IP6Address       *string              `json:"ip6Address" db:"ip6_address"`
+	IP6Gateway       *string              `json:"ip6Gateway" db:"ip6_gateway"`
+	IPAddress        *string              `json:"ipAddress" db:"ip_address"`
+	IPGateway        *string              `json:"ipGateway" db:"ip_gateway"`
+	IPNetmask        *string              `json:"ipNetmask" db:"ip_netmask"`
+	LastUpdated      *TimeNoMod           `json:"lastUpdated" db:"last_updated"`
+	MgmtIPAddress    *string              `json:"mgmtIpAddress" db:"mgmt_ip_address"`
+	MgmtIPGateway    *string              `json:"mgmtIpGateway" db:"mgmt_ip_gateway"`
+	MgmtIPNetmask    *string              `json:"mgmtIpNetmask" db:"mgmt_ip_netmask"`
+	OfflineReason    *string              `json:"offlineReason" db:"offline_reason"`
+	PhysLocation     *string              `json:"physLocation" db:"phys_location"`
+	PhysLocationID   *int                 `json:"physLocationId" db:"phys_location_id"`
+	Profile          *string              `json:"profile" db:"profile"`
+	ProfileDesc      *string              `json:"profileDesc" db:"profile_desc"`
+	ProfileID        *int                 `json:"profileId" db:"profile_id"`
+	Rack             *string              `json:"rack" db:"rack"`
+	RouterHostName   *string              `json:"routerHostName" db:"router_host_name"`
+	RouterPortName   *string              `json:"routerPortName" db:"router_port_name"`
+	Status           *string              `json:"status" db:"status"`
+	StatusID         *int                 `json:"statusId" db:"status_id"`
+	TCPPort          *int                 `json:"tcpPort" db:"tcp_port"`
+	Type             string               `json:"type" db:"server_type"`
+	TypeID           *int                 `json:"typeId" db:"server_type_id"`
+	UpdPending       *bool                `json:"updPending" db:"upd_pending"`
 }
 
-// ValidTransition returns nil if the transition is allowed for the workflow, an error if not
-func (r RequestStatus) ValidTransition(to RequestStatus) error {
-	if r == RequestStatusRejected || r == RequestStatusComplete {
-		// once rejected or completed,  no changes allowed
-		return errors.New(string(r) + " request cannot be changed")
-	}
-
-	if r == to {
-		// no change -- always allowed
-		return nil
-	}
+// UnmarshalJSON implements the json.Unmarshaller interface to suppress unmarshalling for IDNoMod
+//func (a *IDNoMod) UnmarshalJSON([]byte) error {
+	//return nil
+//}
 
-	// indicate if valid transitioning to this RequestStatus
-	switch to {
-	case RequestStatusDraft:
-		// can go back to draft if submitted or rejected
-		if r == RequestStatusSubmitted {
-			return nil
-		}
-	case RequestStatusSubmitted:
-		// can go be submitted if draft or rejected
-		if r == RequestStatusDraft {
-			return nil
-		}
-	case RequestStatusRejected:
-		// only submitted can be rejected
-		if r == RequestStatusSubmitted {
-			return nil
-		}
-	case RequestStatusPending:
-		// only submitted can move to pending
-		if r == RequestStatusSubmitted {
-			return nil
-		}
-	case RequestStatusComplete:
-		// only pending can be completed.  Completed can never change.
-		if r == RequestStatusPending {
-			return nil
-		}
-	}
-	return errors.New("invalid transition from " + string(r) + " to " + string(to))
-}
diff --git a/lib/go-tc/deliveryservices.go b/lib/go-tc/deliveryservices.go
index c352882..a01b921 100644
--- a/lib/go-tc/deliveryservices.go
+++ b/lib/go-tc/deliveryservices.go
@@ -281,17 +281,4 @@ type DeliveryServiceRouting struct {
 	RegionalDenied    int     `json:"regionalDenied"`
 }
 
-// DeliveryServiceServerResponse ...
-type DeliveryServiceServerResponse struct {
-	Response []DeliveryServiceServer `json:"response"`
-	Size     int                     `json:"size"`
-	OrderBy  string                  `json:"orderby"`
-	Limit    int                     `json:"limit"`
-}
 
-// DeliveryServiceServer ...
-type DeliveryServiceServer struct {
-	LastUpdated     string `json:"lastUpdated"`
-	Server          int    `json:"server"`
-	DeliveryService int    `json:"deliveryService"`
-}
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
index 463ff06..768dab0 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers.go
@@ -1,4 +1,4 @@
-package profileparameter
+package servers
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -25,92 +25,105 @@ import (
 	"strconv"
 
 	"github.com/apache/incubator-trafficcontrol/lib/go-log"
-	tc "github.com/apache/incubator-trafficcontrol/lib/go-tc"
-	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/auth"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/dbhelpers"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/tovalidate"
-	validation "github.com/go-ozzo/ozzo-validation"
+	"github.com/go-ozzo/ozzo-validation"
 
 	"github.com/jmoiron/sqlx"
 	"github.com/lib/pq"
 )
 
-const (
-	ProfileIDQueryParam   = "profileId"
-	ParameterIDQueryParam = "parameterId"
-)
-
-//we need a type alias to define functions on
-type TOProfileParameter v13.ProfileParameterNullable
+// TODeliveryServiceRequest provides a type alias to define functions on
+type TODeliveryServiceServer tc.DeliveryServiceServer
 
 //the refType is passed into the handlers where a copy of its type is used to decode the json.
-var refType = TOProfileParameter(v13.ProfileParameterNullable{})
+var refType = TODeliveryServiceServer(tc.DeliveryServiceServer{})
 
-func GetRefType() *TOProfileParameter {
+func GetRefType() *TODeliveryServiceServer {
 	return &refType
 }
 
-func (pp TOProfileParameter) GetKeyFieldsInfo() []api.KeyFieldInfo {
-	return []api.KeyFieldInfo{{ProfileIDQueryParam, api.GetIntKey}, {ParameterIDQueryParam, api.GetIntKey}}
+/*
+# get all delivery services associated with a server (from deliveryservice_server table)
+$r->get( "/api/$version/servers/:id/deliveryservices" => [ id => qr/\d+/ ] )->over( authenticated => 1, not_ldap => 1 )->to( 'Deliveryservice#get_deliveryservices_by_serverId', namespace => $namespace );
+
+# delivery service / server assignments
+$r->post("/api/$version/deliveryservices/:xml_id/servers")->over( authenticated => 1, not_ldap => 1 )
+->to( 'Deliveryservice#assign_servers', namespace => $namespace );
+$r->delete("/api/$version/deliveryservice_server/:dsId/:serverId" => [ dsId => qr/\d+/, serverId => qr/\d+/ ] )->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#remove_server_from_ds', namespace => $namespace );
+	# -- DELIVERYSERVICES: SERVERS
+	# Supports ?orderby=key
+	$r->get("/api/$version/deliveryserviceserver")->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#index', namespace => $namespace );
+	$r->post("/api/$version/deliveryserviceserver")->over( authenticated => 1, not_ldap => 1 )->to( 'DeliveryServiceServer#assign_servers_to_ds', namespace => $namespace );
+
+		{1.2, http.MethodGet, `deliveryservices/{id}/servers$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.2, http.MethodGet, `deliveryservices/{id}/unassigned_servers$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.2, http.MethodGet, `deliveryservices/{id}/servers/eligible$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+
+*/
+
+func (dss TODeliveryServiceServer) GetKeyFieldsInfo() []api.KeyFieldInfo {
+	return []api.KeyFieldInfo{{"deliveryservice", api.GetIntKey}, {"server", api.GetIntKey}}
 }
 
 //Implementation of the Identifier, Validator interface functions
-func (pp TOProfileParameter) GetKeys() (map[string]interface{}, bool) {
-	if pp.ProfileID == nil {
-		return map[string]interface{}{ProfileIDQueryParam: 0}, false
+func (dss TODeliveryServiceServer) GetKeys() (map[string]interface{}, bool) {
+	if dss.DeliveryService == nil {
+		return map[string]interface{}{"deliveryservice": 0}, false
 	}
-	if pp.ParameterID == nil {
-		return map[string]interface{}{ParameterIDQueryParam: 0}, false
+	if dss.Server == nil {
+		return map[string]interface{}{"server": 0}, false
 	}
 	keys := make(map[string]interface{})
-	profileID := *pp.ProfileID
-	parameterID := *pp.ParameterID
+	ds_id := *dss.DeliveryService
+	server_id := *dss.Server
 
-	keys[ProfileIDQueryParam] = profileID
-	keys[ParameterIDQueryParam] = parameterID
+	keys["deliveryservice"] = ds_id
+	keys["server"] = server_id
 	return keys, true
 }
 
-func (pp *TOProfileParameter) GetAuditName() string {
-	if pp.ProfileID != nil {
-		return strconv.Itoa(*pp.ProfileID) + "-" + strconv.Itoa(*pp.ParameterID)
+func (dss *TODeliveryServiceServer) GetAuditName() string {
+	if dss.DeliveryService != nil {
+		return strconv.Itoa(*dss.DeliveryService) + "-" + strconv.Itoa(*dss.Server)
 	}
 	return "unknown"
 }
 
-func (pp *TOProfileParameter) GetType() string {
-	return "profileParameter"
+func (dss *TODeliveryServiceServer) GetType() string {
+	return "deliveryserviceServers"
 }
 
-func (pp *TOProfileParameter) SetKeys(keys map[string]interface{}) {
-	profId, _ := keys[ProfileIDQueryParam].(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.
-	pp.ProfileID = &profId
+func (dss *TODeliveryServiceServer) SetKeys(keys map[string]interface{}) {
+	ds_id, _ := keys["deliveryservice"].(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.
+	dss.DeliveryService = &ds_id
 
-	paramId, _ := keys[ParameterIDQueryParam].(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.
-	pp.ParameterID = &paramId
+	server_id, _ := keys["server"].(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.
+	dss.Server = &server_id
 }
 
 // Validate fulfills the api.Validator interface
-func (pp *TOProfileParameter) Validate(db *sqlx.DB) []error {
+func (dss *TODeliveryServiceServer) Validate(db *sqlx.DB) []error {
 
 	errs := validation.Errors{
-		"profile":   validation.Validate(pp.ProfileID, validation.Required),
-		"parameter": validation.Validate(pp.ParameterID, validation.Required),
+		"deliveryservice": validation.Validate(dss.DeliveryService, validation.Required),
+		"server":          validation.Validate(dss.Server, validation.Required),
 	}
 
 	return tovalidate.ToErrors(errs)
 }
 
-//The TOProfileParameter implementation of the Creator interface
+//The TODeliveryServiceServer implementation of the Creator interface
 //all implementations of Creator should use transactions and return the proper errorType
 //ParsePQUniqueConstraintError is used to determine if a profileparameter 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 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) {
+func (dss *TODeliveryServiceServer) Create(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
 	rollbackTransaction := true
 	tx, err := db.Beginx()
 	defer func() {
@@ -127,7 +140,7 @@ func (pp *TOProfileParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error,
 		log.Error.Printf("could not begin transaction: %v", err)
 		return tc.DBError, tc.SystemError
 	}
-	resultRows, err := tx.NamedQuery(insertQuery(), pp)
+	resultRows, err := tx.NamedQuery(insertQuery(), dss)
 	if err != nil {
 		if pqErr, ok := err.(*pq.Error); ok {
 			err, eType := dbhelpers.ParsePQUniqueConstraintError(pqErr)
@@ -141,19 +154,19 @@ func (pp *TOProfileParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error,
 	}
 	defer resultRows.Close()
 
-	var profile int
-	var parameter int
+	var ds_id int
+	var server_id int
 	var lastUpdated tc.TimeNoMod
 	rowsAffected := 0
 	for resultRows.Next() {
 		rowsAffected++
-		if err := resultRows.Scan(&profile, &parameter, &lastUpdated); err != nil {
-			log.Error.Printf("could not scan profile from insert: %s\n", err)
+		if err := resultRows.Scan(&ds_id, &server_id, &lastUpdated); err != nil {
+			log.Error.Printf("could not scan dss from insert: %s\n", err)
 			return tc.DBError, tc.SystemError
 		}
 	}
 	if rowsAffected == 0 {
-		err = errors.New("no profile_parameter was inserted, no profile+parameter was returned")
+		err = errors.New("no deliveryServiceServer was inserted, nothing to return")
 		log.Errorln(err)
 		return tc.DBError, tc.SystemError
 	}
@@ -163,8 +176,8 @@ func (pp *TOProfileParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error,
 		return tc.DBError, tc.SystemError
 	}
 
-	pp.SetKeys(map[string]interface{}{ProfileIDQueryParam: profile, ParameterIDQueryParam: parameter})
-	pp.LastUpdated = &lastUpdated
+	dss.SetKeys(map[string]interface{}{"deliveryservice": ds_id, "server": server_id})
+	dss.LastUpdated = &lastUpdated
 	err = tx.Commit()
 	if err != nil {
 		log.Errorln("Could not commit transaction: ", err)
@@ -175,57 +188,59 @@ func (pp *TOProfileParameter) Create(db *sqlx.DB, user auth.CurrentUser) (error,
 }
 
 func insertQuery() string {
-	query := `INSERT INTO profile_parameter (
-profile,
-parameter) VALUES (
-:profile_id,
-:parameter_id) RETURNING profile, parameter, last_updated`
+	query := `INSERT INTO deliveryservice_server (
+deliveryservice,
+server) VALUES (
+:ds_id,
+:server_id) RETURNING deliveryservice, server, last_updated`
 	return query
 }
 
-func (pp *TOProfileParameter) Read(db *sqlx.DB, parameters map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
-	var rows *sqlx.Rows
+func (dss *TODeliveryServiceServer) Read(db *sqlx.DB, params map[string]string, user auth.CurrentUser) ([]interface{}, []error, tc.ApiErrorType) {
+	idstr, ok := params["id"]
 
-	// Query Parameters to Database Query column mappings
-	// see the fields mapped in the SQL query
-	queryParamsToQueryCols := map[string]dbhelpers.WhereColumnInfo{
-		"profileId":   dbhelpers.WhereColumnInfo{"pp.profile", nil},
-		"parameterId": dbhelpers.WhereColumnInfo{"pp.parameter", nil},
-		"lastUpdated": dbhelpers.WhereColumnInfo{"pp.last_updated", nil},
+	if !ok {
+		log.Errorf("Deliveryservice Server Id missing")
+		return nil, []error{errors.New("Deliverservice id is required.")}, tc.DataMissingError
 	}
+	id, err := strconv.Atoi(idstr)
 
-	where, orderBy, queryValues, errs := dbhelpers.BuildWhereAndOrderBy(parameters, queryParamsToQueryCols)
-	if len(errs) > 0 {
-		return nil, errs, tc.DataConflictError
+	if err != nil {
+		log.Errorf("Deliveryservice Server Id is not an integer")
+		return nil, []error{errors.New("Deliverservice id is not an integer.")}, tc.SystemError
 	}
 
-	query := selectQuery() + where + orderBy
+	query := selectQuery()
 	log.Debugln("Query is ", query)
 
-	rows, err := db.NamedQuery(query, queryValues)
+	rows, err := db.Queryx(query, id)
 	if err != nil {
-		log.Errorf("Error querying Parameters: %v", err)
+		log.Errorf("Error querying DeliveryserviceServers: %v", err)
 		return nil, []error{tc.DBError}, tc.SystemError
 	}
 	defer rows.Close()
 
-	params := []interface{}{}
+	servers := []interface{}{}
 	for rows.Next() {
-		var p v13.ProfileParameterNullable
-		if err = rows.StructScan(&p); err != nil {
-			log.Errorf("error parsing pp rows: %v", err)
+		var s tc.DssServer
+		if err = rows.StructScan(&s); err != nil {
+			log.Errorf("error parsing dss rows: %v", err)
 			return nil, []error{tc.DBError}, tc.SystemError
 		}
-		params = append(params, p)
+		hiddenField := ""
+		if user.PrivLevel < auth.PrivLevelAdmin {
+			s.ILOPassword = &hiddenField
+		}
+		servers = append(servers, s)
 	}
 
-	return params, []error{}, tc.NoError
+	return servers, []error{}, tc.NoError
 
 }
 
 //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) {
+func (dss *TODeliveryServiceServer) Delete(db *sqlx.DB, user auth.CurrentUser) (error, tc.ApiErrorType) {
 	rollbackTransaction := true
 	tx, err := db.Beginx()
 	defer func() {
@@ -242,8 +257,8 @@ func (pp *TOProfileParameter) Delete(db *sqlx.DB, user auth.CurrentUser) (error,
 		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(), pp)
-	result, err := tx.NamedExec(deleteQuery(), pp)
+	log.Debugf("about to run exec query: %s with parameter: %++v", deleteQuery(), dss)
+	result, err := tx.NamedExec(deleteQuery(), dss)
 	if err != nil {
 		log.Errorf("received error: %++v from delete execution", err)
 		return tc.DBError, tc.SystemError
@@ -267,27 +282,71 @@ func (pp *TOProfileParameter) Delete(db *sqlx.DB, user auth.CurrentUser) (error,
 	rollbackTransaction = false
 	return nil, tc.NoError
 }
-
 func selectQuery() string {
 
-	query := `SELECT
-pp.last_updated,
-pp.profile profile_id,
-pp.parameter parameter_id,
-prof.name profile,
-param.name parameter
-FROM profile_parameter pp
-JOIN profile prof ON prof.id = pp.profile
-JOIN parameter param ON param.id = pp.parameter`
-	return query
+	const JumboFrameBPS = 9000
+
+	// COALESCE is needed to default values that are nil in the database
+	// because Go does not allow that to marshal into the struct
+	selectStmt := `SELECT
+	cg.name as cachegroup,
+	s.cachegroup as cachegroup_id,
+	s.cdn_id,
+	cdn.name as cdn_name,
+	s.domain_name,
+	s.guid,
+	s.host_name,
+	s.https_port,
+	s.id,
+	s.ilo_ip_address,
+	s.ilo_ip_gateway,
+	s.ilo_ip_netmask,
+	s.ilo_password,
+	s.ilo_username,
+	COALESCE(s.interface_mtu, ` + strconv.Itoa(JumboFrameBPS) + `) as interface_mtu,
+	s.interface_name,
+	s.ip6_address,
+	s.ip6_gateway,
+	s.ip_address,
+	s.ip_gateway,
+	s.ip_netmask,
+	s.last_updated,
+	s.mgmt_ip_address,
+	s.mgmt_ip_gateway,
+	s.mgmt_ip_netmask,
+	s.offline_reason,
+	pl.name as phys_location,
+	s.phys_location as phys_location_id,
+	p.name as profile,
+	p.description as profile_desc,
+	s.profile as profile_id,
+	s.rack,
+	s.router_host_name,
+	s.router_port_name,
+	st.name as status,
+	s.status as status_id,
+	s.tcp_port,
+	t.name as server_type,
+	s.type as server_type_id,
+	s.upd_pending as upd_pending
+	FROM server s
+	JOIN cachegroup cg ON s.cachegroup = cg.id
+	JOIN cdn cdn ON s.cdn_id = cdn.id
+	JOIN phys_location pl ON s.phys_location = pl.id
+	JOIN profile p ON s.profile = p.id
+	JOIN status st ON s.status = st.id
+	JOIN type t ON s.type = t.id
+	WHERE s.id in (select server from deliveryservice_server where deliveryservice = $1)`
+
+	return selectStmt
 }
 
 func updateQuery() string {
 	query := `UPDATE
-profile_parameter SET
-profile=:profile_id,
-parameter=:parameter_id
-WHERE profile=:profile_id AND 
+	profile_parameter SET
+	profile=:profile_id,
+	parameter=:parameter_id
+	WHERE profile=:profile_id AND 
       parameter = :parameter_id 
       RETURNING last_updated`
 	return query
diff --git a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers_test.go b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers_test.go
index 9e7ce20..58d556e 100644
--- a/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers_test.go
+++ b/traffic_ops/traffic_ops_golang/deliveryservice/servers/servers_test.go
@@ -1,4 +1,4 @@
-package profileparameter
+package servers
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -19,93 +19,4 @@ package profileparameter
  * under the License.
  */
 
-import (
-	"testing"
-	"time"
 
-	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
-	"github.com/apache/incubator-trafficcontrol/lib/go-tc/v13"
-	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
-	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/auth"
-	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/test"
-	"github.com/jmoiron/sqlx"
-
-	sqlmock "gopkg.in/DATA-DOG/go-sqlmock.v1"
-)
-
-func getTestProfileParameters() []v13.ProfileParameterNullable {
-	pps := []v13.ProfileParameterNullable{}
-	lastUpdated := tc.TimeNoMod{}
-	lastUpdated.Scan(time.Now())
-	profileID := 1
-	parameterID := 1
-
-	pp := v13.ProfileParameterNullable{
-		LastUpdated: &lastUpdated,
-		ProfileID:   &profileID,
-		ParameterID: &parameterID,
-	}
-	pps = append(pps, pp)
-
-	pp2 := pp
-	pp2.ProfileID = &profileID
-	pp2.ParameterID = &parameterID
-	pps = append(pps, pp2)
-
-	return pps
-}
-
-func TestGetProfileParameters(t *testing.T) {
-	mockDB, mock, 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()
-
-	testPPs := getTestProfileParameters()
-	cols := test.ColsFromStructByTag("db", v13.ProfileParameterNullable{})
-	rows := sqlmock.NewRows(cols)
-
-	for _, ts := range testPPs {
-		rows = rows.AddRow(
-			ts.LastUpdated,
-			ts.Profile,
-			ts.ProfileID,
-			ts.Parameter,
-			ts.ParameterID,
-		)
-	}
-	mock.ExpectQuery("SELECT").WillReturnRows(rows)
-	v := map[string]string{"profile": "1"}
-
-	pps, errs, _ := refType.Read(db, v, auth.CurrentUser{})
-	if len(errs) > 0 {
-		t.Errorf("profileparameter.Read expected: no errors, actual: %v", errs)
-	}
-
-	if len(pps) != 2 {
-		t.Errorf("profileparameter.Read expected: len(pps) == 2, actual: %v", len(pps))
-	}
-
-}
-
-func TestInterfaces(t *testing.T) {
-	var i interface{}
-	i = &TOProfileParameter{}
-
-	if _, ok := i.(api.Creator); !ok {
-		t.Errorf("ProfileParameter must be Creator")
-	}
-	if _, ok := i.(api.Reader); !ok {
-		t.Errorf("ProfileParameter must be Reader")
-	}
-	if _, ok := i.(api.Deleter); !ok {
-		t.Errorf("ProfileParameter must be Deleter")
-	}
-	if _, ok := i.(api.Identifier); !ok {
-		t.Errorf("ProfileParameter must be Identifier")
-	}
-}
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index b82fa59..125f7ef 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -38,6 +38,7 @@ import (
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/coordinate"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/crconfig"
 	dsrequest "github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request"
+	dsserver "github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/servers"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservice/request/comment"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/division"
 	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/hwinfo"
@@ -148,6 +149,11 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{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},
 
+		// get all edge servers associated with a delivery service (from deliveryservice_server table)
+		{1.2, http.MethodGet, `deliveryservices/{id}/servers$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.2, http.MethodGet, `deliveryservices/{id}/unassigned_servers$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.2, http.MethodGet, `deliveryservices/{id}/servers/eligible$`, api.ReadHandler(dsserver.GetRefType(), d.DB),auth.PrivLevelReadOnly, Authenticated, nil},
+
 		//Server
 		{1.1, http.MethodGet, `servers/checks$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
 		{1.1, http.MethodGet, `servers/details$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},

-- 
To stop receiving notification emails like this one, please contact
rob@apache.org.