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

[incubator-trafficcontrol] branch master updated: Add TO Go Servers Details endpoint

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 30c960f  Add TO Go Servers Details endpoint
30c960f is described below

commit 30c960ff3e117fa3080567c9f185d3e4c65a8436
Author: Robert Butts <ro...@apache.org>
AuthorDate: Mon May 14 15:48:56 2018 -0600

    Add TO Go Servers Details endpoint
---
 lib/go-tc/servers.go                               |  39 ++++
 lib/go-util/join.go                                |   6 +
 .../traffic_ops_golang/api/shared_handlers.go      |  26 +++
 traffic_ops/traffic_ops_golang/routes.go           |   5 +-
 traffic_ops/traffic_ops_golang/server/detail.go    | 255 +++++++++++++++++++++
 5 files changed, 330 insertions(+), 1 deletion(-)

diff --git a/lib/go-tc/servers.go b/lib/go-tc/servers.go
index 6d9a7a0..c70b177 100644
--- a/lib/go-tc/servers.go
+++ b/lib/go-tc/servers.go
@@ -138,3 +138,42 @@ type ServerUpdateStatus struct {
 	ParentPending      bool   `json:"parent_pending"`
 	ParentRevalPending bool   `json:"parent_reval_pending"`
 }
+
+type ServerDetail struct {
+	CacheGroup         *string           `json:"cachegroup" db:"cachegroup"`
+	CDNName            *string           `json:"cdnName" db:"cdn_name"`
+	DeliveryServiceIDs []int64           `json:"deliveryservices,omitempty"`
+	DomainName         *string           `json:"domainName" db:"domain_name"`
+	GUID               *string           `json:"guid" db:"guid"`
+	HardwareInfo       map[string]string `json:"hardwareInfo"`
+	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"`
+	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"`
+	Profile            *string           `json:"profile" db:"profile"`
+	ProfileDesc        *string           `json:"profileDesc" db:"profile_desc"`
+	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"`
+	TCPPort            *int              `json:"tcpPort" db:"tcp_port"`
+	Type               string            `json:"type" db:"server_type"`
+	XMPPID             *string           `json:"xmppId" db:"xmpp_id"`
+	XMPPPasswd         *string           `json:"xmppPasswd" db:"xmpp_passwd"`
+}
diff --git a/lib/go-util/join.go b/lib/go-util/join.go
index e3f10d8..933d67d 100644
--- a/lib/go-util/join.go
+++ b/lib/go-util/join.go
@@ -21,6 +21,8 @@ package util
 
 import (
 	"fmt"
+	"regexp"
+	"strings"
 )
 
 func JoinErrsStr(errs []error) string {
@@ -58,3 +60,7 @@ func JoinErrsSep(errs []error, separator string) error {
 
 	return fmt.Errorf("%s", joinedErrors)
 }
+
+func CamelToSnakeCase(s string) string {
+	return strings.ToLower(regexp.MustCompile("([a-z0-9])([A-Z])").ReplaceAllString(s, "${1}_${2}"))
+}
diff --git a/traffic_ops/traffic_ops_golang/api/shared_handlers.go b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
index c4b520f..408bab8 100644
--- a/traffic_ops/traffic_ops_golang/api/shared_handlers.go
+++ b/traffic_ops/traffic_ops_golang/api/shared_handlers.go
@@ -432,6 +432,20 @@ func WriteResp(w http.ResponseWriter, r *http.Request, v interface{}) {
 	w.Write(respBts)
 }
 
+// WriteRespVals is like WriteResp, but also takes a map of root-level values to write. The API most commonly needs these for meta-parameters, like size, limit, and orderby.
+// This is a helper for the common case; not using this in unusual cases is perfectly acceptable.
+func WriteRespVals(w http.ResponseWriter, r *http.Request, v interface{}, vals map[string]interface{}) {
+	vals["response"] = v
+	respBts, err := json.Marshal(vals)
+	if err != nil {
+		log.Errorf("marshalling JSON for %T: %v", v, err)
+		tc.GetHandleErrorsFunc(w, r)(http.StatusInternalServerError, errors.New(http.StatusText(http.StatusInternalServerError)))
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(respBts)
+}
+
 // HandleErr handles an API error, writing the given statusCode and userErr to the user, and logging the sysErr. If userErr is nil, the text of the HTTP statusCode is written.
 // This is a helper for the common case; not using this in unusual cases is perfectly acceptable.
 func HandleErr(w http.ResponseWriter, r *http.Request, statusCode int, userErr error, sysErr error) {
@@ -465,6 +479,18 @@ func RespWriter(w http.ResponseWriter, r *http.Request) func(v interface{}, err
 	}
 }
 
+// RespWriterVals is like RespWriter, but also takes a map of root-level values to write. The API most commonly needs these for meta-parameters, like size, limit, and orderby.
+// This is a helper for the common case; not using this in unusual cases is perfectly acceptable.
+func RespWriterVals(w http.ResponseWriter, r *http.Request, vals map[string]interface{}) func(v interface{}, err error) {
+	return func(v interface{}, err error) {
+		if err != nil {
+			HandleErr(w, r, http.StatusInternalServerError, nil, err)
+			return
+		}
+		WriteRespVals(w, r, v, vals)
+	}
+}
+
 // WriteRespAlert creates an alert, 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 WriteRespAlert(w http.ResponseWriter, r *http.Request, level tc.AlertLevel, msg string) {
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 9a9f43a..6976ca4 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -165,10 +165,13 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 
 		//Server
 		{1.1, http.MethodGet, `servers/checks$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
-		{1.1, http.MethodGet, `servers/details$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
 		{1.1, http.MethodGet, `servers/status$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
 		{1.1, http.MethodGet, `servers/totals$`, handlerToFunc(proxyHandler), 0, NoAuth, []Middleware{}},
 
+		//Server Details
+		{1.2, http.MethodGet, `servers/details/?(\.json)?$`, server.GetDetailParamHandler(d.DB.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		{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},
diff --git a/traffic_ops/traffic_ops_golang/server/detail.go b/traffic_ops/traffic_ops_golang/server/detail.go
new file mode 100644
index 0000000..978dc7b
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/server/detail.go
@@ -0,0 +1,255 @@
+package server
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import (
+	"database/sql"
+	"errors"
+	"net/http"
+	"strconv"
+
+	"github.com/apache/incubator-trafficcontrol/lib/go-tc"
+	"github.com/apache/incubator-trafficcontrol/lib/go-util"
+	"github.com/apache/incubator-trafficcontrol/traffic_ops/traffic_ops_golang/api"
+
+	"github.com/lib/pq"
+)
+
+func GetDetailHandler(db *sql.DB) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		params, err := api.GetCombinedParams(r)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting combined params: "+err.Error()))
+			return
+		}
+		servers, err := getDetailServers(db, params["hostName"], -1, "", 0)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting detail servers: "+err.Error()))
+			return
+		}
+		if len(servers) == 0 {
+			api.HandleErr(w, r, http.StatusNotFound, nil, nil)
+			return
+		}
+		server := servers[0]
+		api.WriteResp(w, r, server)
+	}
+}
+
+func GetDetailParamHandler(db *sql.DB) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		params, err := api.GetCombinedParams(r)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting combined params: "+err.Error()))
+			return
+		}
+		hostName := params["hostName"]
+		physLocationIDStr := params["physLocationID"]
+		physLocationID := -1
+		if physLocationIDStr != "" {
+			physLocationID, err = strconv.Atoi(physLocationIDStr)
+			if err != nil {
+				api.HandleErr(w, r, http.StatusBadRequest, errors.New("physLocationID parameter is not an integer"), nil)
+				return
+			}
+		}
+		if hostName == "" && physLocationIDStr == "" {
+			api.HandleErr(w, r, http.StatusBadRequest, errors.New("Missing required fields: 'hostname' or 'physLocationID'"), nil)
+			return
+		}
+		orderBy := "hostName"
+		if _, ok := params["orderby"]; ok {
+			orderBy = params["orderby"]
+		}
+		limit := 1000
+		if limitStr, ok := params["limit"]; ok {
+			limit, err = strconv.Atoi(limitStr)
+			if err != nil {
+				api.HandleErr(w, r, http.StatusBadRequest, errors.New("limit parameter is not an integer"), nil)
+				return
+			}
+		}
+		servers, err := getDetailServers(db, hostName, physLocationID, util.CamelToSnakeCase(orderBy), limit)
+		respVals := map[string]interface{}{
+			"orderby": orderBy,
+			"limit":   limit,
+			"size":    len(servers),
+		}
+		api.RespWriterVals(w, r, respVals)(servers, err)
+	}
+}
+
+func getDetailServers(db *sql.DB, hostName string, physLocationID int, orderBy string, limit int) ([]tc.ServerDetail, error) {
+	allowedOrderByCols := map[string]string{
+		"":                 "",
+		"cachegroup":       "s.cachegroup",
+		"cdn_name":         "cdn.name",
+		"domain_name":      "s.domain_name",
+		"guid":             "s.guid",
+		"host_name":        "s.host_name",
+		"https_port":       "s.https_port",
+		"id":               "s.id",
+		"ilo_ip_address":   "s.ilo_ip_address",
+		"ilo_ip_gateway":   "s.ilo_ip_gateway",
+		"ilo_ip_netmask":   "s.ilo_ip_netmask",
+		"ilo_password":     "s.ilo_password",
+		"ilo_username":     "s.ilo_username",
+		"interface_mtu":    "interface_mtu",
+		"interface_name":   "s.interface_name",
+		"ip6_address":      "s.ip6_address",
+		"ip6_gateway":      "s.ip6_gateway",
+		"ip_address":       "s.ip_address",
+		"ip_gateway":       "s.ip_gateway",
+		"ip_netmask":       "s.ip_netmask",
+		"mgmt_ip_address":  "s.mgmt_ip_address",
+		"mgmt_ip_gateway":  "s.mgmt_ip_gateway",
+		"mgmt_ip_netmask":  "s.mgmt_ip_netmask",
+		"offline_reason":   "s.offline_reason",
+		"phys_location":    "pl.name",
+		"profile":          "p.name",
+		"profile_desc":     "p.description",
+		"rack":             "s.rack",
+		"router_host_name": "s.router_host_name",
+		"router_port_name": "s.router_port_name",
+		"status":           "st.name",
+		"tcp_port":         "s.tcp_port",
+		"server_type":      "t.name",
+		"xmpp_id":          "s.xmpp_id",
+		"xmpp_passwd":      "s.xmpp_passwd",
+	}
+	orderBy, ok := allowedOrderByCols[orderBy]
+	if !ok {
+		return nil, errors.New("orderBy '" + orderBy + "' not permitted")
+	}
+	const JumboFrameBPS = 9000
+	q := `
+SELECT
+cg.name as cachegroup,
+cdn.name as cdn_name,
+ARRAY(select deliveryservice from deliveryservice_server where server = s.id),
+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.mgmt_ip_address,
+s.mgmt_ip_gateway,
+s.mgmt_ip_netmask,
+s.offline_reason,
+pl.name as phys_location,
+p.name as profile,
+p.description as profile_desc,
+s.rack,
+s.router_host_name,
+s.router_port_name,
+st.name as status,
+s.tcp_port,
+t.name as server_type,
+s.xmpp_id,
+s.xmpp_passwd
+FROM server as s
+JOIN cachegroup cg ON s.cachegroup = cg.id
+JOIN 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
+`
+	limitStr := ""
+	if limit != 0 {
+		limitStr = " LIMIT " + strconv.Itoa(limit)
+	}
+	orderByStr := ""
+	if orderBy != "" {
+		orderByStr = " ORDER BY " + orderBy
+	}
+	rows := (*sql.Rows)(nil)
+	err := error(nil)
+	if hostName != "" && physLocationID != -1 {
+		q += ` WHERE s.host_name = $1::text AND s.phys_location = $2::bigint` + orderByStr + limitStr
+		rows, err = db.Query(q, hostName, physLocationID)
+	} else if hostName != "" {
+		q += ` WHERE s.host_name = $1::text` + orderByStr + limitStr
+		rows, err = db.Query(q, hostName)
+	} else if physLocationID != -1 {
+		q += ` WHERE s.phys_location = $1::int` + orderByStr + limitStr
+		rows, err = db.Query(q, physLocationID)
+	} else {
+		q += orderByStr + limitStr
+		rows, err = db.Query(q) // Should never happen for API <1.3, which don't allow querying without hostName or physLocation
+	}
+	if err != nil {
+		return nil, errors.New("Error querying detail servers: " + err.Error())
+	}
+	defer rows.Close()
+	sIDs := []int{}
+	servers := []tc.ServerDetail{}
+	for rows.Next() {
+		s := tc.ServerDetail{}
+		if err := rows.Scan(&s.CacheGroup, &s.CDNName, pq.Array(&s.DeliveryServiceIDs), &s.DomainName, &s.GUID, &s.HostName, &s.HTTPSPort, &s.ID, &s.ILOIPAddress, &s.ILOIPGateway, &s.ILOIPNetmask, &s.ILOPassword, &s.ILOUsername, &s.InterfaceMTU, &s.InterfaceName, &s.IP6Address, &s.IP6Gateway, &s.IPAddress, &s.IPGateway, &s.IPNetmask, &s.MgmtIPAddress, &s.MgmtIPGateway, &s.MgmtIPNetmask, &s.OfflineReason, &s.PhysLocation, &s.Profile, &s.ProfileDesc, &s.Rack, &s.RouterHostName, &s.RouterPortName [...]
+			return nil, errors.New("Error scanning detail server: " + err.Error())
+		}
+		servers = append(servers, s)
+		sIDs = append(sIDs, *s.ID)
+	}
+
+	rows, err = db.Query(`SELECT serverid, description, val from hwinfo where serverid = ANY($1);`, pq.Array(sIDs))
+	if err != nil {
+		return nil, errors.New("Error querying detail servers hardware info: " + err.Error())
+	}
+	defer rows.Close()
+	hwInfos := map[int]map[string]string{}
+	for rows.Next() {
+		serverID := 0
+		desc := ""
+		val := ""
+		if err := rows.Scan(&serverID, &desc, &val); err != nil {
+			return nil, errors.New("Error scanning detail server hardware info: " + err.Error())
+		}
+
+		hwInfo, ok := hwInfos[serverID]
+		if !ok {
+			hwInfo = map[string]string{}
+		}
+		hwInfo[desc] = val
+		hwInfos[serverID] = hwInfo
+	}
+	for i, server := range servers {
+		hw, ok := hwInfos[*server.ID]
+		if !ok {
+			continue
+		}
+		server.HardwareInfo = hw
+		servers[i] = server
+	}
+	return servers, nil
+}

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