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

[trafficcontrol] 01/02: Add TO Go users/id/deliveryservices

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

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

commit 4cbd06bf04ada5dcfb5c74fb2c71bc93095e58c2
Author: Robert Butts <ro...@apache.org>
AuthorDate: Sun May 27 12:19:51 2018 -0600

    Add TO Go users/id/deliveryservices
---
 lib/go-tc/deliveryservices.go                      |  30 +++
 traffic_ops/traffic_ops_golang/routes.go           |   5 +-
 .../traffic_ops_golang/user/deliveryservices.go    | 252 +++++++++++++++++++++
 3 files changed, 286 insertions(+), 1 deletion(-)

diff --git a/lib/go-tc/deliveryservices.go b/lib/go-tc/deliveryservices.go
index 50983e8..7cd1324 100644
--- a/lib/go-tc/deliveryservices.go
+++ b/lib/go-tc/deliveryservices.go
@@ -544,6 +544,36 @@ type DeliveryServiceRouting struct {
 	RegionalDenied    int     `json:"regionalDenied"`
 }
 
+type UserAvailableDS struct {
+	ID          *int    `json:"id" db:"id"`
+	DisplayName *string `json:"displayName" db:"display_name"`
+	XMLID       *string `json:"xmlId" db:"xml_id"`
+	TenantID    *int    `json:"-"` // tenant is necessary to check authorization, but not serialized
+}
+
+type DeliveryServiceUserPost struct {
+	UserID           *int   `json:"userId"`
+	DeliveryServices *[]int `json:"deliveryServices"`
+	Replace          *bool  `json:"replace"`
+}
+
+type UserDeliveryServicePostResponse struct {
+	Alerts   []Alert                 `json:"alerts"`
+	Response DeliveryServiceUserPost `json:"response"`
+}
+
+type UserDeliveryServicesResponseV13 struct {
+	Response []DeliveryServiceV13 `json:"response"`
+}
+
+type UserDeliveryServicesResponseV12 struct {
+	Response []DeliveryServiceV13 `json:"response"`
+}
+
+type UserDeliveryServicesResponse struct {
+	Response []DeliveryServiceNullableV13 `json:"response"`
+}
+
 type DSServerIDs struct {
 	DeliveryServiceID *int  `json:"dsId", db:"deliveryservice"`
 	ServerIDs         []int `json:"servers"`
diff --git a/traffic_ops/traffic_ops_golang/routes.go b/traffic_ops/traffic_ops_golang/routes.go
index 33acff1..60670a3 100644
--- a/traffic_ops/traffic_ops_golang/routes.go
+++ b/traffic_ops/traffic_ops_golang/routes.go
@@ -48,7 +48,6 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/deliveryservicesregexes"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/division"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/hwinfo"
-	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/origin"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/parameter"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/physlocation"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/ping"
@@ -62,7 +61,9 @@ import (
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/systeminfo"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
 	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/types"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/user"
 
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/origin"
 	"github.com/basho/riak-go-client"
 	"github.com/jmoiron/sqlx"
 )
@@ -140,6 +141,8 @@ func Routes(d ServerData) ([]Route, []RawRoute, http.Handler, error) {
 		{1.1, http.MethodGet, `hwinfo-wip/?(\.json)?$`, hwinfo.HWInfoHandler(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 
 		//Login
+		{1.1, http.MethodGet, `users/{id}/deliveryservices/?(\.json)?$`, user.GetDSes(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
+		{1.1, http.MethodGet, `user/{id}/deliveryservices/available/?(\.json)?$`, user.GetAvailableDSes(d.DB), auth.PrivLevelReadOnly, Authenticated, nil},
 		{1.2, http.MethodPost, `user/login/?$`, wrapAccessLog(d.Secrets[0], auth.LoginHandler(d.DB, d.Config)), 0, NoAuth, nil}, {1.3, http.MethodPost, `user/login/?$`, auth.LoginHandler(d.DB, d.Config), 0, NoAuth, nil},
 
 		//Parameter: CRUD
diff --git a/traffic_ops/traffic_ops_golang/user/deliveryservices.go b/traffic_ops/traffic_ops_golang/user/deliveryservices.go
new file mode 100644
index 0000000..509bd57
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/user/deliveryservices.go
@@ -0,0 +1,252 @@
+package user
+
+/*
+ * 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"
+
+	"github.com/apache/trafficcontrol/lib/go-tc"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/api"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/auth"
+	"github.com/apache/trafficcontrol/traffic_ops/traffic_ops_golang/tenant"
+
+	"github.com/jmoiron/sqlx"
+)
+
+func GetDSes(db *sqlx.DB) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		user, err := auth.GetCurrentUser(r.Context())
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting current user: "+err.Error()))
+			return
+		}
+		_, intParams, userErr, sysErr, errCode := api.AllParams(r, []string{"id"}, []string{"id"})
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		dsUserID := intParams["id"]
+		dses, err := getUserDSes(db.DB, dsUserID)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user delivery services: "+err.Error()))
+			return
+		}
+
+		dses, err = filterAuthorized(db, dses, user)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("filtering user-authorized delivery services: "+err.Error()))
+			return
+		}
+		api.WriteResp(w, r, dses)
+	}
+}
+
+func GetAvailableDSes(db *sqlx.DB) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		user, err := auth.GetCurrentUser(r.Context())
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting current user: "+err.Error()))
+			return
+		}
+		_, intParams, userErr, sysErr, errCode := api.AllParams(r, []string{"id"}, []string{"id"})
+		if userErr != nil || sysErr != nil {
+			api.HandleErr(w, r, errCode, userErr, sysErr)
+			return
+		}
+		dsUserID := intParams["id"]
+		dses, err := getUserAvailableDSes(db.DB, dsUserID)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("getting user delivery services: "+err.Error()))
+			return
+		}
+
+		dses, err = filterAvailableAuthorized(db, dses, user)
+		if err != nil {
+			api.HandleErr(w, r, http.StatusInternalServerError, nil, errors.New("filtering user-authorized delivery services: "+err.Error()))
+			return
+		}
+		api.WriteResp(w, r, dses)
+	}
+}
+
+func filterAuthorized(db *sqlx.DB, dses []tc.DeliveryServiceNullableV13, user *auth.CurrentUser) ([]tc.DeliveryServiceNullableV13, error) {
+	authorizedDSes := []tc.DeliveryServiceNullableV13{}
+	for _, ds := range dses {
+		if ds.TenantID == nil {
+			continue
+		}
+		authorized, err := tenant.IsResourceAuthorizedToUser(*ds.TenantID, *user, db)
+		if err != nil {
+			return nil, errors.New("checking delivery service tenancy authorization: " + err.Error())
+		}
+		if !authorized {
+			continue // TODO determine if this is correct - Perl appears to return an error if any DS on the user is unauthorized to the current user
+		}
+		authorizedDSes = append(authorizedDSes, ds)
+	}
+	return authorizedDSes, nil
+}
+
+func filterAvailableAuthorized(db *sqlx.DB, dses []tc.UserAvailableDS, user *auth.CurrentUser) ([]tc.UserAvailableDS, error) {
+	authorizedDSes := []tc.UserAvailableDS{}
+	for _, ds := range dses {
+		if ds.TenantID == nil {
+			continue
+		}
+		authorized, err := tenant.IsResourceAuthorizedToUser(*ds.TenantID, *user, db)
+		if err != nil {
+			return nil, errors.New("checking delivery service tenancy authorization: " + err.Error())
+		}
+		if !authorized {
+			continue // TODO determine if this is correct - Perl appears to return an error if any DS on the user is unauthorized to the current user
+		}
+		authorizedDSes = append(authorizedDSes, ds)
+	}
+	return authorizedDSes, nil
+}
+
+func getUserDSes(db *sql.DB, userID int) ([]tc.DeliveryServiceNullableV13, error) {
+	q := `
+SELECT
+ds.active,
+ds.cacheurl,
+ds.ccr_dns_ttl,
+ds.cdn_id,
+cdn.name as cdnName,
+ds.check_path,
+ds.deep_caching_type,
+ds.display_name,
+ds.dns_bypass_cname,
+ds.dns_bypass_ip,
+ds.dns_bypass_ip6,
+ds.dns_bypass_ttl,
+ds.dscp,
+ds.edge_header_rewrite,
+ds.geolimit_redirect_url,
+ds.geo_limit,
+ds.geo_limit_countries,
+ds.geo_provider,
+ds.global_max_mbps,
+ds.global_max_tps,
+ds.fq_pacing_rate,
+ds.http_bypass_fqdn,
+ds.id,
+ds.info_url,
+ds.initial_dispersion,
+ds.ipv6_routing_enabled,
+ds.last_updated,
+ds.logs_enabled,
+ds.long_desc,
+ds.long_desc_1,
+ds.long_desc_2,
+ds.max_dns_answers,
+ds.mid_header_rewrite,
+COALESCE(ds.miss_lat, 0.0),
+COALESCE(ds.miss_long, 0.0),
+ds.multi_site_origin,
+ds.org_server_fqdn,
+ds.origin_shield,
+ds.profile as profileID,
+profile.name as profile_name,
+profile.description  as profile_description,
+ds.protocol,
+ds.qstring_ignore,
+ds.range_request_handling,
+ds.regex_remap,
+ds.regional_geo_blocking,
+ds.remap_text,
+ds.routing_name,
+ds.signing_algorithm,
+ds.ssl_key_version,
+ds.tenant_id,
+tenant.name,
+ds.tr_request_headers,
+ds.tr_response_headers,
+type.name,
+ds.type as type_id,
+ds.xml_id
+FROM deliveryservice as ds
+JOIN type ON ds.type = type.id
+JOIN cdn ON ds.cdn_id = cdn.id
+JOIN deliveryservice_tmuser dsu ON ds.id = dsu.deliveryservice
+LEFT JOIN profile ON ds.profile = profile.id
+LEFT JOIN tenant ON ds.tenant_id = tenant.id
+WHERE dsu.tm_user_id = $1
+`
+	rows, err := db.Query(q, userID)
+	if err != nil {
+		return nil, errors.New("querying user delivery services: " + err.Error())
+	}
+	defer rows.Close()
+	dses := []tc.DeliveryServiceNullableV13{}
+	for rows.Next() {
+		ds := tc.DeliveryServiceNullableV13{}
+		deepCachingTypeStr := ""
+		err := rows.Scan(&ds.Active, &ds.CacheURL, &ds.CCRDNSTTL, &ds.CDNID, &ds.CDNName, &ds.CheckPath, &deepCachingTypeStr, &ds.DisplayName, &ds.DNSBypassCNAME, &ds.DNSBypassIP, &ds.DNSBypassIP6, &ds.DNSBypassTTL, &ds.DSCP, &ds.EdgeHeaderRewrite, &ds.GeoLimitRedirectURL, &ds.GeoLimit, &ds.GeoLimitCountries, &ds.GeoProvider, &ds.GlobalMaxMBPS, &ds.GlobalMaxTPS, &ds.FQPacingRate, &ds.HTTPBypassFQDN, &ds.ID, &ds.InfoURL, &ds.InitialDispersion, &ds.IPV6RoutingEnabled, &ds.LastUpdated, &ds.LogsEn [...]
+		if err != nil {
+			return nil, errors.New("scanning user delivery services : " + err.Error())
+		}
+		deepCachingType := tc.DeepCachingTypeFromString(deepCachingTypeStr)
+		ds.DeepCachingType = &deepCachingType
+		dses = append(dses, ds)
+	}
+	return dses, nil
+}
+
+func getUserAvailableDSes(db *sql.DB, userID int) ([]tc.UserAvailableDS, error) {
+	q := `
+SELECT
+ds.id,
+ds.display_name,
+ds.xml_id,
+ds.tenant_id
+FROM deliveryservice as ds
+JOIN deliveryservice_tmuser dsu ON ds.id = dsu.deliveryservice
+WHERE dsu.tm_user_id = $1
+`
+	rows, err := db.Query(q, userID)
+	if err != nil {
+		return nil, errors.New("querying user available delivery services: " + err.Error())
+	}
+	defer rows.Close()
+	dses := []tc.UserAvailableDS{}
+	for rows.Next() {
+		ds := tc.UserAvailableDS{}
+		err := rows.Scan(&ds.ID, &ds.DisplayName, &ds.XMLID, &ds.TenantID)
+		if err != nil {
+			return nil, errors.New("scanning user available delivery services : " + err.Error())
+		}
+		dses = append(dses, ds)
+	}
+	return dses, nil
+}
+
+func getUserTenantIDByID(db *sql.DB, id int) (*int, bool, error) {
+	tenantID := (*int)(nil)
+	if err := db.QueryRow(`SELECT tenant_id FROM tm_user WHERE id = $1`, id).Scan(&tenantID); err != nil {
+		if err == sql.ErrNoRows {
+			return nil, false, nil
+		}
+		return nil, false, errors.New("querying user: " + err.Error())
+	}
+	return tenantID, true, nil
+}