You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@trafficcontrol.apache.org by GitBox <gi...@apache.org> on 2022/04/13 22:07:14 UTC

[GitHub] [trafficcontrol] srijeet0406 opened a new pull request, #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

srijeet0406 opened a new pull request, #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754

   <!--
   Thank you for contributing! Please be sure to read our contribution guidelines: https://github.com/apache/trafficcontrol/blob/master/CONTRIBUTING.md
   If this closes or relates to an existing issue, please reference it using one of the following:
   
   Closes: #ISSUE
   Related: #ISSUE
   
   If this PR fixes a security vulnerability, DO NOT submit! Instead, contact
   the Apache Traffic Control Security Team at security@trafficcontrol.apache.org and follow the
   guidelines at https://apache.org/security regarding vulnerability disclosure.
   -->
   This PR is not related to any issue.
   It adds the ability in Traffic Ops, so that it can be run in conjunction with other backend services, wherein TO would just act as a reverse proxy and forward the requests coming in on the specified paths to the specified backends.
   <!-- **^ Add meaningful description above** --><hr/>
   
   ## Which Traffic Control components are affected by this PR?
   <!-- Please delete all components from this list that are NOT affected by this PR.
   Feel free to add the name of a tool or script that is affected but not on the list.
   -->
   - Documentation
   - Traffic Ops
   
   
   ## What is the best way to verify this PR?
   <!-- Please include here ALL the steps necessary to test your PR.
   If your PR has tests (and most should), provide the steps needed to run the tests.
   If not, please provide step-by-step instructions to test the PR manually and explain why your PR does not need tests. -->
    Set up a simple backend service that servers a couple of endpoints (say `foo` and `foos`) and run it on a couple of different ports(say, 8444 and 8445). Now, run Traffic Ops with the new config option and make sure that any request to Traffic Ops on the specified endpoints (in the new config) gets forwarded to the backend service. Also make sure that the user sees the response from the backend service.
   Make sure all the tests pass.
   
   ## If this is a bugfix, which Traffic Control versions contained the bug?
   <!-- Delete this section if the PR is not a bugfix, or if the bug is only in the master branch.
   Examples:
   - 5.1.2
   - 5.1.3 (RC1)
    -->
   - master
   
   ## PR submission checklist
   - [ ] This PR has tests <!-- If not, please delete this text and explain why this PR does not need tests. -->
   - [x] This PR has documentation <!-- If not, please delete this text and explain why this PR does not need documentation. -->
   - [x] This PR has a CHANGELOG.md entry <!-- A fix for a bug from an ATC release, an improvement, or a new feature should have a changelog entry. -->
   - [x] This PR **DOES NOT FIX A SERIOUS SECURITY VULNERABILITY** (see [the Apache Software Foundation's security guidelines](https://apache.org/security) for details)
   
   <!--
   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.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] rawlinp commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
rawlinp commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r861198495


##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -46,6 +51,28 @@ import (
 // RoutePrefix is a prefix that all API routes must match.
 const RoutePrefix = "^api" // TODO config?
 
+type backendConfigSynced struct {
+	cfg config.BackendConfig
+	*sync.RWMutex
+}
+
+// backendCfg stores the current backend config supplied to traffic ops.
+var backendCfg = backendConfigSynced{RWMutex: &sync.RWMutex{}}
+
+// GetBackendConfig returns the current BackendConfig.
+func GetBackendConfig() config.BackendConfig {
+	backendCfg.RLock()
+	defer backendCfg.RUnlock()
+	return backendCfg.cfg
+}
+
+// SetBackendConfig sets the BackendConfig to the value supplied.
+func SetBackendConfig(backendConfig config.BackendConfig) {
+	backendCfg.RLock()
+	defer backendCfg.RUnlock()

Review Comment:
   I believe these are meant to be `Lock()` and `Unlock()`, because we are writing to the shared value here rather than reading.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] rawlinp commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
rawlinp commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r856599358


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,29 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.

Review Comment:
   Rather than this being a `host:port` string combo, it might be better to make it an object:
   ```json
   {"hosts": {
     "protocol": "https",
     "hostname": "cdn-foo-backend-service-host",
     "port": 9090
   }}
   ```
   This way, we could add more attributes to hosts in the future, such as weights, filters, etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] srijeet0406 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
srijeet0406 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r860026758


##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}
+				rp.ErrorHandler = func(w http.ResponseWriter, r *http.Request, err error) {
+					api.HandleErr(w, r, nil, http.StatusInternalServerError, nil, err)
+					return
+				}
+				routeCtx := context.WithValue(ctx, api.DBContextKey, db)
+				routeCtx = context.WithValue(routeCtx, api.PathParamsKey, map[string]string{})
+				r = r.WithContext(routeCtx)
+				r.Header.Add(middleware.RouteID, strconv.Itoa(backendRoute.ID))

Review Comment:
   We need it so that the middleware can log the route ID in the access logs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] rawlinp commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
rawlinp commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r856593738


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,29 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.
+	:insecure:          A boolean specifying whether or not to enable `InsecureSkipVerify`. This is an optional parameter, defaulting to `false` when not present.
+	:permissions:       An array of permissions(strings) specifying the permissions required by the user to use this API route.

Review Comment:
   space needed between `permissions` and `(strings)`
   
   If there are no permissions specified, can the route be used by any user?



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,29 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.
+	:insecure:          A boolean specifying whether or not to enable `InsecureSkipVerify`. This is an optional parameter, defaulting to `false` when not present.

Review Comment:
   Rather than saying this is for the `InsecureSkipVerify` struct field, we should elaborate on what that actually means (controls whether TO verifies the backend server's certificate chain and host name -- not recommended for production use).



##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}
+				rp.ErrorHandler = func(w http.ResponseWriter, r *http.Request, err error) {
+					api.HandleErr(w, r, nil, http.StatusInternalServerError, nil, err)
+					return
+				}
+				routeCtx := context.WithValue(ctx, api.DBContextKey, db)
+				routeCtx = context.WithValue(routeCtx, api.PathParamsKey, map[string]string{})
+				r = r.WithContext(routeCtx)
+				r.Header.Add(middleware.RouteID, strconv.Itoa(backendRoute.ID))

Review Comment:
   why do we need to add the routeID as a request header?



##########
traffic_ops/traffic_ops_golang/config/config.go:
##########
@@ -286,6 +305,20 @@ func (c Config) EventLog() log.LogLocation {
 const BlockStartup = true
 const AllowStartup = false
 
+func LoadBackendConfig(backendConfigPath string) (BackendConfig, error) {
+	confBytes, err := ioutil.ReadFile(backendConfigPath)
+	if err != nil {
+		return BackendConfig{}, fmt.Errorf("reading backend conf '%s': %v", backendConfigPath, err)
+	}
+
+	cfg := BackendConfig{}
+	err = json.Unmarshal(confBytes, &cfg)
+	if err != nil {
+		return BackendConfig{}, fmt.Errorf("unmarshalling '%s': %v", backendConfigPath, err)
+	}
+	return cfg, nil

Review Comment:
   after unmarshalling, we should validate the provided config options (e.g. valid paths, hostnames, ports, algorithm, etc)



##########
CHANGELOG.md:
##########
@@ -16,6 +16,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
 - Replaces all Traffic Portal Tenant select boxes with a novel tree select box [#6427](https://github.com/apache/trafficcontrol/issues/6427).
 - Traffic Monitor: Add support for `access.log` to TM.
 - Added functionality for login to provide a Bearer token and for that token to be later used for authorization.
+- [Traffic Ops] Added support for backend configurations so that Traffic Ops can be run as a Service Oriented architecture(SOA) product [#6754](https://github.com/apache/trafficcontrol/pull/6754).

Review Comment:
   > can be run as a Service Oriented architecture(SOA) product
   
   maybe we should say "can act as a reverse proxy for other backend services" -- it's a bit more specific without prescribing SOA



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,29 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.

Review Comment:
   Rather than this being a `host:port` string combo, it might be better to make it an object:
   ```
   "hosts": [
     "protocol": "https",
     "hostname": "cdn-foo-backend-service-host",
     "port": 9090
   ]
   ```
   This way, we could add more attributes to hosts in the future, such as weights, filters, etc.



##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -46,6 +51,26 @@ import (
 // RoutePrefix is a prefix that all API routes must match.
 const RoutePrefix = "^api" // TODO config?
 
+// BackendConfig stores the current backend config supplied to traffic ops.
+var BackendConfig config.BackendConfig
+
+// Mutex is a mutex for safely reading/ writing to BackendConfig.
+var Mutex = sync.RWMutex{}

Review Comment:
   I think it's a good practice to embed the mutex into a struct that contains the locked data. For example:
   ```
   type backendConfigSynced struct {
       cfg config.BackendConfig
       *sync.RWMutex // note that this is a pointer to reduce the chances of accidentally copying the lock value (which is invalid)
   }
   var backendCfg backendConfigSynced{RWMutex: &sync.RWMutex{}} // note that it's lowercase so that it's private to this package
   
   func GetBackendConfig() config.BackendConfig {
       backendCfg.RLock()
       defer backendCfg.RUnlock()
       return backendCfg.cfg
   }
   ```



##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -214,7 +226,7 @@ func main() {
 			file.Close()
 		}
 
-		if err := server.ListenAndServeTLS(cfg.CertPath, cfg.KeyPath); err != nil {
+		if err := http.ListenAndServeTLS(server.Addr, cfg.CertPath, cfg.KeyPath, mux); err != nil {

Review Comment:
   So, if we're no longer using the `server` struct that was created above, does this still configure the TLSConfig, timeouts, etc?



##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -350,6 +381,12 @@ func reloadProfilingInfo(configFileName string) (bool, string, error) {
 	return cfg.ProfilingEnabled, profilingLocation, nil
 }
 
+func reloadBackendConfig(backendConfigFileName string) (config.BackendConfig, error) {

Review Comment:
   this function doesn't seem like it's doing enough to deserve its own function -- should we just add the logging to `setNewBackendConfig` and get rid of this function?



##########
traffic_ops/app/conf/backends.conf:
##########
@@ -0,0 +1,38 @@
+{

Review Comment:
   Typically the default config file should be usable (essentially, empty/all defaults). We wouldn't want fresh, default installs to include these routes, but this is a useful example. We should copy this into the documentation as an example, then make `production/backends.conf` basically empty.



##########
traffic_ops/app/conf/backends.conf:
##########
@@ -0,0 +1,38 @@
+{
+  "routes": [
+    {
+      "path": "/api/4.0/foo",
+      "method": "GET",
+      "hosts": [
+        "localhost:8444",
+        "localhost:8445"
+      ],
+      "insecure": true,
+      "permissions": [
+        "CDN:READ",
+        "CDN:WRITE"
+      ],
+      "routeId": 123456,
+      "opts": {
+        "alg": "roundrobin"
+      }
+    },
+    {
+      "path": "/api/4.0/foos",
+      "method": "GET",
+      "hosts": [
+        "localhost:8444",
+        "localhost:8445"
+      ],
+      "insecure": true,
+      "permissions": [
+        "CDN:READ",
+        "CDN:WRITE"
+      ],
+      "routeId": 123457,
+      "opts": {
+        "alg": "roundrobin"
+      }
+    }
+  ]
+}

Review Comment:
   missing newline



##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}
+				rp.ErrorHandler = func(w http.ResponseWriter, r *http.Request, err error) {
+					api.HandleErr(w, r, nil, http.StatusInternalServerError, nil, err)
+					return
+				}
+				routeCtx := context.WithValue(ctx, api.DBContextKey, db)
+				routeCtx = context.WithValue(routeCtx, api.PathParamsKey, map[string]string{})
+				r = r.WithContext(routeCtx)
+				r.Header.Add(middleware.RouteID, strconv.Itoa(backendRoute.ID))
+				userErr, sysErr, code := HandleBackendRoute(cfg, backendRoute, w, r)
+				if userErr != nil || sysErr != nil {
+					h2 := middleware.WrapAccessLog(cfg.Secrets[0], middleware.BackendErrorHandler(code, userErr, sysErr))
+					h2.ServeHTTP(w, r)
+					return
+				}
+				backendHandler := middleware.WrapAccessLog(cfg.Secrets[0], rp)
+				backendHandler.ServeHTTP(w, r)
+				return
+			} else {
+				h2 := middleware.WrapAccessLog(cfg.Secrets[0], middleware.BackendErrorHandler(http.StatusBadRequest, errors.New("only an algorithm of roundrobin is supported by the backend options currently"), nil))
+				h2.ServeHTTP(w, r)
+				return
+			}
+		}
+	}
+	if !backendRouteHandled {
+		catchall.ServeHTTP(w, r)
+	}
+}
+
+// HandleBackendRoute does all the pre processing for the backend routes.
+func HandleBackendRoute(cfg *config.Config, route config.BackendRoute, w http.ResponseWriter, r *http.Request) (error, error, int) {
+	var userErr, sysErr error
+	var errCode int
+	var user auth.CurrentUser
+	var inf *api.APIInfo
 
-	catchall.ServeHTTP(w, r)
+	user, userErr, sysErr, errCode = api.GetUserFromReq(w, r, cfg.Secrets[0])
+	if userErr != nil || sysErr != nil {
+		return userErr, sysErr, errCode
+	}
+	v := api.GetRequestedAPIVersion(r.URL.Path)

Review Comment:
   Why do we need to check the API version?



##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {

Review Comment:
   I think we'll probably want more than just exact string and method match -- what if the route is `/foo/{id}/bar`?



##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -234,6 +246,13 @@ func main() {
 
 	reloadProfilingConfig := func() {
 		setNewProfilingInfo(*configFileName, &profiling, &profilingLocation, cfg.Version)
+		backendConfig, err = setNewBackendConfig(backendConfigFileName)
+		if err != nil {
+			log.Errorf("could not reload backend config: %v", err)
+		} else {
+			d.BackendConfig = backendConfig

Review Comment:
   There is another thread that would be reading `d.BackendConfig`, right? Does this assignment need to be synchronized? Going through the rest of the PR, I guess I can't tell why `d.BackendConfig` is even needed or if it's even used? Wouldn't readers just call `routing.GetBackendConfig()`?



##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -234,6 +246,13 @@ func main() {
 
 	reloadProfilingConfig := func() {

Review Comment:
   this name no longer makes sense since it's reloading more than just the profiling config now



##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -293,6 +312,18 @@ func setupTrafficVault(riakConfigFileName string, cfg *config.Config) trafficvau
 	return &disabled.Disabled{}
 }
 
+func setNewBackendConfig(backendConfigFileName *string) (config.BackendConfig, error) {

Review Comment:
   this doesn't seem to `set`, so maybe `get` would be a better?



##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}

Review Comment:
   This can be simplified to just 
   ```
   rp.Transport = &http.Transport{
       TLSClientConfig: &tls.Config{InsecureSkipVerify: insecure}
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] ocket8888 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r852481059


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -600,9 +600,11 @@ This file deals with the configuration parameters of running Traffic Ops as a re
 
 	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
 	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
-	:route_id:          The integral identifier for the new route being added.
+	:routeId:           The integral identifier for the new route being added.
 	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.
-	:insecure:          A boolean specifying whether or not to enable `InsecureSkipVerify`.
+	:insecure:          A boolean specifying whether or not to enable `InsecureSkipVerify`. This is an optional parameter, defaulting to `false` when not present.
+	:privLevel:         An integer designating the minimum privilege level required to use this API route.

Review Comment:
   privLevel is deprecated, we shouldn't use it moving forward.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] rawlinp commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
rawlinp commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r860035386


##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}
+				rp.ErrorHandler = func(w http.ResponseWriter, r *http.Request, err error) {
+					api.HandleErr(w, r, nil, http.StatusInternalServerError, nil, err)
+					return
+				}
+				routeCtx := context.WithValue(ctx, api.DBContextKey, db)
+				routeCtx = context.WithValue(routeCtx, api.PathParamsKey, map[string]string{})
+				r = r.WithContext(routeCtx)
+				r.Header.Add(middleware.RouteID, strconv.Itoa(backendRoute.ID))

Review Comment:
   Oh, rather than doing that as an HTTP header can we do it as a request context value? I.e.
   ```
   routeCtx = context.WithValue(routeCtx, middleware.RouteID, backendRoute.ID)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] srijeet0406 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
srijeet0406 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r860234889


##########
traffic_ops/traffic_ops_golang/traffic_ops_golang.go:
##########
@@ -234,6 +246,13 @@ func main() {
 
 	reloadProfilingConfig := func() {
 		setNewProfilingInfo(*configFileName, &profiling, &profilingLocation, cfg.Version)
+		backendConfig, err = setNewBackendConfig(backendConfigFileName)
+		if err != nil {
+			log.Errorf("could not reload backend config: %v", err)
+		} else {
+			d.BackendConfig = backendConfig

Review Comment:
   Yeah I was trying to pass all the data in the `ServerData` object, which I now realize isn't necessary. I have modified this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] ocket8888 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r863159413


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The regex matching the endpoint that will be served by the backend, for example, `^api/4.0/foo?$`.

Review Comment:
   fyi there's a specific [Sphinx text role for regular expressions](https://www.sphinx-doc.org/en/master/usage/restructuredtext/roles.html#role-regexp) - I think you meant for this to be monospace font e.g. using the `<code>` HTML element, but actually single grave accents (<kbd>\`</kbd>) uses the "default" role which is emphasis, so you're trying to render `^api/4.0/foo?$` but you're getting *^api/4.0/foo?$*
   
   tl;dr: use `:regexp:` for regular expressions, use double grave accents (<kbd>\`</kbd>) for code like:
   
   ```rst
   ``This is code``
   
   `This is italicized`
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] rawlinp merged pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
rawlinp merged PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] srijeet0406 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
srijeet0406 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r859953356


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,29 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The endpoint that will be served by the backend, for example, `api/4.0/foo`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the hosts and ports where the request (if matched) needs to be forwarded to, for example, `cdn-foo-backend-service-host:9090`.
+	:insecure:          A boolean specifying whether or not to enable `InsecureSkipVerify`. This is an optional parameter, defaulting to `false` when not present.
+	:permissions:       An array of permissions(strings) specifying the permissions required by the user to use this API route.

Review Comment:
   Yeah, that's how the permissions model works on our normal routes as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] srijeet0406 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
srijeet0406 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r860047944


##########
traffic_ops/traffic_ops_golang/routing/routing.go:
##########
@@ -280,8 +307,88 @@ func Handler(
 		h.ServeHTTP(w, r)
 		return
 	}
+	var backendRouteHandled bool
+	backendConfig := GetBackendConfig()
+	for i, backendRoute := range backendConfig.Routes {
+		if backendRoute.Path == r.URL.Path && backendRoute.Method == r.Method {
+			if backendRoute.Opts.Algorithm == "" || backendRoute.Opts.Algorithm == "roundrobin" {
+				index := backendRoute.Index % len(backendRoute.Hosts)
+				host := backendRoute.Hosts[index]
+				backendRoute.Index++
+				backendConfig.Routes[i] = backendRoute
+				backendRouteHandled = true
+				rp := httputil.NewSingleHostReverseProxy(&url.URL{
+					Host:   host,
+					Scheme: cfg.URL.Scheme,
+				})
+				if backendRoute.Insecure {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+					}
+				} else {
+					rp.Transport = &http.Transport{
+						TLSClientConfig: &tls.Config{},
+					}
+				}
+				rp.ErrorHandler = func(w http.ResponseWriter, r *http.Request, err error) {
+					api.HandleErr(w, r, nil, http.StatusInternalServerError, nil, err)
+					return
+				}
+				routeCtx := context.WithValue(ctx, api.DBContextKey, db)
+				routeCtx = context.WithValue(routeCtx, api.PathParamsKey, map[string]string{})
+				r = r.WithContext(routeCtx)
+				r.Header.Add(middleware.RouteID, strconv.Itoa(backendRoute.ID))

Review Comment:
   Yep, sure can. I've changed it in both the places this was being set as a header.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficcontrol] ocket8888 commented on a diff in pull request #6754: Convert Traffic Ops into a Service Oriented architecture(SOA) product

Posted by GitBox <gi...@apache.org>.
ocket8888 commented on code in PR #6754:
URL: https://github.com/apache/trafficcontrol/pull/6754#discussion_r863161299


##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The regex matching the endpoint that will be served by the backend, for example, `^api/4.0/foo?$`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.

Review Comment:
   same as above RE: double vs single grave accents



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The regex matching the endpoint that will be served by the backend, for example, `^api/4.0/foo?$`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the host object, which specifies the protocol, hostname and port where the request (if matched) needs to be forwarded to.
+
+		:protocol:     The protocol/scheme to be followed while forwarding the requests to the backend service.
+		:hostname:     The hostname of the server where the backend service is running.
+		:port:         The port (integer) on the backend server where the service is running.
+
+	:insecure:          A boolean specifying whether or not TO should verify the backend server's certificate chain and host name. This is not recommended for production use. This is an optional parameter, defaulting to `false` when not present.

Review Comment:
   same as above RE: double vs single grave accents



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The regex matching the endpoint that will be served by the backend, for example, `^api/4.0/foo?$`.
+	:method:            The HTTP method for the above mentioned path, for example, `GET` or `PUT`.
+	:routeId:           The integral identifier for the new route being added.
+	:hosts:             An array of the host object, which specifies the protocol, hostname and port where the request (if matched) needs to be forwarded to.
+
+		:protocol:     The protocol/scheme to be followed while forwarding the requests to the backend service.
+		:hostname:     The hostname of the server where the backend service is running.
+		:port:         The port (integer) on the backend server where the service is running.
+
+	:insecure:          A boolean specifying whether or not TO should verify the backend server's certificate chain and host name. This is not recommended for production use. This is an optional parameter, defaulting to `false` when not present.
+	:permissions:       An array of permissions (strings) specifying the permissions required by the user to use this API route.
+	:opts:              A collection of key value pairs to control how the requests should be forwarded/ handled, for example, `"alg": "roundrobin"`. Currently, only `roundrobin` is supported (which is also the default if nothing is specified) by Traffic Ops.

Review Comment:
   same as above RE: double vs single grave accents



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:

Review Comment:
   it's not hurting anything but this label is unused.
   ```rst
   backends.conf_
   ```
   is a link to a section titled `backends.conf` on the current page, not a link to a label. `:ref:` is used to refer to arbitrary labels from any page/section.



##########
docs/source/admin/traffic_ops.rst:
##########
@@ -586,6 +590,34 @@ This file sets authentication options for connections to Riak when used as the T
 
 .. impl-detail:: The name of this file is derived from the current database used in the implementation of Traffic Vault - `Riak KV <https://riak.com/products/riak-kv/index.html>`_.
 
+.. _backends.conf:
+
+backends.conf
+"""""""""""""
+This file deals with the configuration parameters of running Traffic Ops as a reverse proxy for certain endpoints that need to be served externally by other backend services. It is a JSON-format set of options and their respective values. `traffic_ops_golang`_ will use whatever file is specified (if any) by its :option:`--backendcfg` option. The keys of the file are described below.
+
+:routes: This is an array of options to configure Traffic Ops to forward requests of specified types to the appropriate backends.
+
+	:path:              The regex matching the endpoint that will be served by the backend, for example, `^api/4.0/foo?$`.

Review Comment:
   fyi there's a specific [Sphinx text role for regular expressions](https://www.sphinx-doc.org/en/master/usage/restructuredtext/roles.html#role-regexp) - I think you meant for this to be monospace font e.g. using the `<code>` HTML element, but actually single grave accents (<kbd>\`</kbd>) uses the "default" role which is emphasis, so you're trying to render `^api/4.0/foo?$` but you're getting *^api/4.0/foo?$*
   
   tl;dr: use `:regexp:` for regular expressions, use double grave accents (<kbd>\`</kbd>) for code.



##########
traffic_ops/app/conf/production/backends.conf:
##########
@@ -0,0 +1,48 @@
+{
+  "routes": [

Review Comment:
   Ah, it seems your editor somehow has become misconfigured to insert spaces instead of indentation. That happens sometimes, but it's usually easy to fix, so you can go back to indenting things instead of spacing them.
   
   Also: shouldn't `routes` be empty in a default configuration? Based on the values below, I'm not sure exactly what I'd expect the configuration to do. In particular it's a bit concerning that the docs strongly recommend that `insecure` not be set to `true` in production, but this production configuration does exactly that.



##########
traffic_ops/traffic_ops_golang/config/config.go:
##########
@@ -36,6 +36,31 @@ import (
 	"github.com/apache/trafficcontrol/lib/go-util"
 )
 
+type Options struct {
+	Algorithm string `json:"alg"`
+}
+
+type Host struct {
+	Protocol string `json:"protocol"`
+	Hostname string `json:"hostname"`
+	Port     int    `json:"port"`
+}
+
+type BackendRoute struct {
+	Path        string   `json:"path"`
+	Method      string   `json:"method"`
+	Hosts       []Host   `json:"hosts"`
+	Opts        Options  `json:"opts"`
+	ID          int      `json:"routeId"`
+	Insecure    bool     `json:"insecure"`
+	Permissions []string `json:"permissions"`
+	Index       int
+}
+
+type BackendConfig struct {

Review Comment:
   GoDocs pls



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@trafficcontrol.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org