You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by sr...@apache.org on 2022/03/28 21:57:36 UTC

[trafficcontrol] branch master updated: Add TO in-memory User cache option to improve performance (#6671)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5d13d3b  Add TO in-memory User cache option to improve performance (#6671)
5d13d3b is described below

commit 5d13d3bc102104ba2592e677ac81d1dfd0f3d07a
Author: Rawlin Peters <ra...@apache.org>
AuthorDate: Mon Mar 28 15:57:25 2022 -0600

    Add TO in-memory User cache option to improve performance (#6671)
    
    * Add TO in-memory User cache option to improve performance
    
    Currently, logging in a user can take up to 4 different DB queries, and
    most route handlers require at least 1 DB query to look up the user and
    role of the requester.
    
    By periodically reading users and roles from the database (using the
    interval defined by `user_cache_refresh_interval_sec` in cdn.conf) and
    storing them in memory, Traffic Ops can quickly check the user's data
    and role without having to read from the database at request time.
    
    This makes logins extremely efficient because they require no database
    queries at request time. Additionally, overall database load is reduced
    because most route handlers have reduced the number of queries they make
    by 1.
    
    From a security perspective, making logins extremely efficient is
    advantageous because it reduces the effectiveness of denial of service
    attacks targeting the login endpoint (which would normally make at least
    1 database query to check that the user is allowed to log in). With this
    feature enabled, a DoS attack against the login endpoint would not
    exhaust limited database resources. Note: other endpoints are not as
    susceptible to DoS attacks due to requiring a valid cookie before any
    database queries are made.
    
    The downside to enabling this feature is that updates to Users and/or
    Roles are not enforced by Traffic Ops immediately when the updates are
    made. If `user_cache_refresh_interval_sec` is N, it may take up to N
    seconds before Traffic Ops will enforce updates to Users and/or Roles.
    However, as long as this interval is sufficiently short, this should not
    be a problem.
    
    * Return additional bool from getUserNameFromCacheByToken
---
 CHANGELOG.md                                       |   1 +
 docs/source/admin/traffic_ops.rst                  |   6 +
 .../ansible/roles/traffic_ops/defaults/main.yml    |   1 +
 .../roles/traffic_ops/templates/cdn.conf.j2        |   1 +
 traffic_ops/app/conf/cdn.conf                      |   1 +
 traffic_ops/traffic_ops_golang/auth/authorize.go   |  57 ++++--
 traffic_ops/traffic_ops_golang/auth/usercache.go   | 209 +++++++++++++++++++++
 .../traffic_ops_golang/auth/usercache_test.go      |  87 +++++++++
 .../traffic_ops_golang/cachegroup/dspost.go        |   2 +-
 traffic_ops/traffic_ops_golang/config/config.go    |  56 +++---
 .../traffic_ops_golang/config/config_test.go       |   4 +
 .../traffic_ops_golang/login/logout_test.go        |  12 +-
 .../traffic_ops_golang/traffic_ops_golang.go       |   2 +
 13 files changed, 395 insertions(+), 44 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index a217fa3..75b11f1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -10,6 +10,7 @@ The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/).
 - Added a Traffic Ops endpoints to `PUT` a requested configuration change for a full configuration or per host and an endpoint to approve or deny the request.
 - Traffic Monitor config option `distributed_polling` which enables the ability for Traffic Monitor to poll a subset of the CDN and divide into "local peer groups" and "distributed peer groups". Traffic Monitors in the same group are local peers, while Traffic Monitors in other groups are distibuted peers. Each TM group polls the same set of cachegroups and gets availability data for the other cachegroups from other TM groups. This allows each TM to be responsible for polling a subset of [...]
 - Added support for a new Traffic Ops GLOBAL profile parameter -- `tm_query_status_override` -- to override which status of Traffic Monitors to query (default: ONLINE).
+- Traffic Ops: added new `cdn.conf` option -- `user_cache_refresh_interval_sec` -- which enables an in-memory users cache to improve performance. Default: 0 (disabled).
 - Traffic Router: Add support for `file`-protocol URLs for the `geolocation.polling.url` for the Geolocation database.
 - 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.
diff --git a/docs/source/admin/traffic_ops.rst b/docs/source/admin/traffic_ops.rst
index 623a630..601391b 100644
--- a/docs/source/admin/traffic_ops.rst
+++ b/docs/source/admin/traffic_ops.rst
@@ -501,6 +501,12 @@ This file deals with the configuration parameters of running Traffic Ops itself.
 
 	:dcdn_id: A string representing this :abbr:`CDN (Content Delivery Network)` to be used in the :abbr:`JWT (JSON Web Token)` and subsequently in :abbr:`CDNi (Content Delivery Network Interconnect)` operations.
 
+:user_cache_refresh_interval_sec: This optional integer value specifies the interval (in seconds) between refreshing the in-memory Users cache. Default: 0 (disabled).
+
+	.. warning:: Enabling the Users cache improves performance by reducing the number of queries made to the Traffic Ops database, but it means that it may take up to this many seconds before any changes to Users and/or Roles are enforced.
+
+	.. versionadded:: 7.0
+
 
 Example cdn.conf
 ''''''''''''''''
diff --git a/infrastructure/ansible/roles/traffic_ops/defaults/main.yml b/infrastructure/ansible/roles/traffic_ops/defaults/main.yml
index 15d467d..bec28e6 100644
--- a/infrastructure/ansible/roles/traffic_ops/defaults/main.yml
+++ b/infrastructure/ansible/roles/traffic_ops/defaults/main.yml
@@ -44,6 +44,7 @@ to_admin_user_password:
 to_disable_auto_cert_deletion: false
 to_use_ims: true
 to_use_rbp: true
+to_user_cache_refresh_interval_sec: 0
 to_heartbeat_timeout: 20
 to_hypnotoad_number_of_workers: 12
 to_cors_access_control_allow_origin: "http://localhost:8080"
diff --git a/infrastructure/ansible/roles/traffic_ops/templates/cdn.conf.j2 b/infrastructure/ansible/roles/traffic_ops/templates/cdn.conf.j2
index 40f1609..4244b7f 100644
--- a/infrastructure/ansible/roles/traffic_ops/templates/cdn.conf.j2
+++ b/infrastructure/ansible/roles/traffic_ops/templates/cdn.conf.j2
@@ -49,6 +49,7 @@
       "password" : "{{ to_smtp_password }}",
       "address" : "{{ to_smtp_address }}"
    },
+   "user_cache_refresh_interval_sec": {{ to_user_cache_refresh_interval_sec }},
    "disable_auto_cert_deletion": {{ to_disable_auto_cert_deletion | bool | lower }},
    "use_ims": {{ to_use_ims | bool | lower }},
    "role_based_permissions": {{ to_use_rbp | bool | lower }},
diff --git a/traffic_ops/app/conf/cdn.conf b/traffic_ops/app/conf/cdn.conf
index 62f672a..eb3ce57 100644
--- a/traffic_ops/app/conf/cdn.conf
+++ b/traffic_ops/app/conf/cdn.conf
@@ -44,6 +44,7 @@
         }
     },
     "disable_auto_cert_deletion": false,
+    "user_cache_refresh_interval_sec": 0,
     "use_ims": false,
     "role_based_permissions": true,
     "cors" : {
diff --git a/traffic_ops/traffic_ops_golang/auth/authorize.go b/traffic_ops/traffic_ops_golang/auth/authorize.go
index df230a1..d1913d9 100644
--- a/traffic_ops/traffic_ops_golang/auth/authorize.go
+++ b/traffic_ops/traffic_ops_golang/auth/authorize.go
@@ -87,8 +87,6 @@ const PrivLevelUnauthenticated = 0
 
 const PrivLevelReadOnly = 10
 
-const PrivLevelORT = 11
-
 const PrivLevelSteering = 15
 
 const PrivLevelFederation = 15
@@ -108,6 +106,14 @@ const CurrentUserKey key = iota
 
 // GetCurrentUserFromDB  - returns the id and privilege level of the given user along with the username, or -1 as the id, - as the userName and PrivLevelInvalid if the user doesn't exist, along with a user facing error, a system error to log, and an error code to return
 func GetCurrentUserFromDB(DB *sqlx.DB, user string, timeout time.Duration) (CurrentUser, error, error, int) {
+	invalidUser := CurrentUser{"-", -1, PrivLevelInvalid, TenantIDInvalid, -1, "", []string{}, "", nil}
+	if usersCacheIsEnabled() {
+		u, exists := getUserFromCache(user)
+		if !exists {
+			return invalidUser, errors.New("user not found"), fmt.Errorf("checking user '%s' info: user not in cache", user), http.StatusUnauthorized
+		}
+		return u.CurrentUser, nil, nil, http.StatusOK
+	}
 	qry := `
 SELECT
   r.priv_level,
@@ -115,7 +121,7 @@ SELECT
   r.name as role_name, 
   u.id,
   u.username,
-  COALESCE(u.tenant_id, -1) AS tenant_id,
+  u.tenant_id,
   ARRAY(SELECT rc.cap_name FROM role_capability AS rc WHERE rc.role_id=r.id) AS capabilities,
   u.ucdn
 FROM
@@ -135,7 +141,6 @@ WHERE
 
 	err := DB.GetContext(dbCtx, &currentUserInfo, qry, user)
 	if err != nil {
-		invalidUser := CurrentUser{"-", -1, PrivLevelInvalid, TenantIDInvalid, -1, "", []string{}, "", nil}
 		if errors.Is(err, sql.ErrNoRows) {
 			return invalidUser, errors.New("user not found"), fmt.Errorf("checking user %v info: user not in database", user), http.StatusUnauthorized
 		}
@@ -166,6 +171,14 @@ func GetCurrentUser(ctx context.Context) (*CurrentUser, error) {
 }
 
 func CheckLocalUserIsAllowed(form PasswordForm, db *sqlx.DB, ctx context.Context) (bool, error, error) {
+	if usersCacheIsEnabled() {
+		u, exists := getUserFromCache(form.Username)
+		if !exists {
+			return false, fmt.Errorf("user '%s' not found in cache", form.Username), nil
+		}
+		allowed := u.RoleName != disallowed
+		return allowed, nil, nil
+	}
 	var roleName string
 
 	err := db.GetContext(ctx, &roleName, "SELECT role.name FROM role INNER JOIN tm_user ON tm_user.role = role.id where username=$1", form.Username)
@@ -185,6 +198,13 @@ func CheckLocalUserIsAllowed(form PasswordForm, db *sqlx.DB, ctx context.Context
 
 // GetUserUcdn returns the Upstream CDN to which the user belongs for CDNi operations.
 func GetUserUcdn(form PasswordForm, db *sqlx.DB, ctx context.Context) (string, error) {
+	if usersCacheIsEnabled() {
+		u, exists := getUserFromCache(form.Username)
+		if !exists {
+			return "", fmt.Errorf("user '%s' not found in cache", form.Username)
+		}
+		return u.UCDN, nil
+	}
 	var ucdn string
 
 	err := db.GetContext(ctx, &ucdn, "SELECT ucdn FROM tm_user where username=$1", form.Username)
@@ -197,15 +217,26 @@ func GetUserUcdn(form PasswordForm, db *sqlx.DB, ctx context.Context) (string, e
 
 func CheckLocalUserPassword(form PasswordForm, db *sqlx.DB, ctx context.Context) (bool, error, error) {
 	var hashedPassword string
-
-	err := db.GetContext(ctx, &hashedPassword, "SELECT local_passwd FROM tm_user WHERE username=$1", form.Username)
-	if err != nil {
-		if err == context.DeadlineExceeded || err == context.Canceled {
-			return false, nil, err
+	if usersCacheIsEnabled() {
+		u, exists := getUserFromCache(form.Username)
+		if !exists {
+			return false, fmt.Errorf("user '%s' not found in cache", form.Username), nil
+		}
+		if u.LocalPasswd == nil {
+			return false, nil, nil
+		}
+		hashedPassword = *u.LocalPasswd
+	} else {
+		err := db.GetContext(ctx, &hashedPassword, "SELECT local_passwd FROM tm_user WHERE username=$1", form.Username)
+		if err != nil {
+			if err == context.DeadlineExceeded || err == context.Canceled {
+				return false, nil, err
+			}
+			return false, err, nil
 		}
-		return false, err, nil
 	}
-	err = VerifySCRYPTPassword(form.Password, hashedPassword)
+
+	err := VerifySCRYPTPassword(form.Password, hashedPassword)
 	if err != nil {
 		hashedInput, err := sha1Hex(form.Password)
 		if err != nil {
@@ -222,6 +253,10 @@ func CheckLocalUserPassword(form PasswordForm, db *sqlx.DB, ctx context.Context)
 // CheckLocalUserToken checks the passed token against the records in the db for a match, up to a
 // maximum duration of timeout.
 func CheckLocalUserToken(token string, db *sqlx.DB, timeout time.Duration) (bool, string, error) {
+	if usersCacheIsEnabled() {
+		username, matched := getUserNameFromCacheByToken(token)
+		return matched, username, nil
+	}
 	dbCtx, dbClose := context.WithTimeout(context.Background(), timeout)
 	defer dbClose()
 
diff --git a/traffic_ops/traffic_ops_golang/auth/usercache.go b/traffic_ops/traffic_ops_golang/auth/usercache.go
new file mode 100644
index 0000000..95520bb
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/auth/usercache.go
@@ -0,0 +1,209 @@
+package auth
+
+/*
+ * 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 (
+	"context"
+	"database/sql"
+	"errors"
+	"sync"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-log"
+
+	"github.com/lib/pq"
+)
+
+const (
+	getUsersQuery = `
+		SELECT
+			u.id,
+			u.local_passwd,
+			u.role,
+			u.tenant_id,
+			u.token,
+			u.ucdn,
+			u.username
+		FROM
+			tm_user AS u
+	`
+	getRolesQuery = `
+		SELECT
+			ARRAY(SELECT rc.cap_name FROM role_capability AS rc WHERE rc.role_id=r.id) AS capabilities,
+			r.id as role,
+			r.name as role_name,
+			r.priv_level
+		FROM role r
+	`
+)
+
+type user struct {
+	CurrentUser
+	LocalPasswd *string
+	Token       *string
+}
+
+type role struct {
+	Capabilities pq.StringArray
+	ID           int
+	Name         string
+	PrivLevel    int
+}
+
+type users struct {
+	userMap          map[string]user
+	usernamesByToken map[string]string
+	*sync.RWMutex
+	initialized bool
+	enabled     bool // note: enabled is only written to once at startup, before serving requests, so it doesn't need synchronized access
+}
+
+var usersCache = users{RWMutex: &sync.RWMutex{}}
+
+func usersCacheIsEnabled() bool {
+	if usersCache.enabled {
+		usersCache.RLock()
+		defer usersCache.RUnlock()
+		return usersCache.initialized
+	}
+	return false
+}
+
+// getUserFromCache returns the user with the given username and a boolean indicating whether the user exists.
+func getUserFromCache(username string) (user, bool) {
+	usersCache.RLock()
+	defer usersCache.RUnlock()
+	u, exists := usersCache.userMap[username]
+	return u, exists
+}
+
+// getUserNameFromCacheByToken returns the username with the given token and a boolean indicating whether a matching token was found.
+func getUserNameFromCacheByToken(token string) (string, bool) {
+	usersCache.RLock()
+	defer usersCache.RUnlock()
+	t, exists := usersCache.usernamesByToken[token]
+	return t, exists
+}
+
+var once = sync.Once{}
+
+// InitUsersCache attempts to initialize the in-memory users data (if enabled) then
+// starts a goroutine to periodically refresh the in-memory data from the database.
+func InitUsersCache(interval time.Duration, db *sql.DB, timeout time.Duration) {
+	once.Do(func() {
+		if interval <= 0 {
+			return
+		}
+		usersCache.enabled = true
+		refreshUsersCache(db, timeout)
+		startUsersCacheRefresher(interval, db, timeout)
+	})
+}
+
+func startUsersCacheRefresher(interval time.Duration, db *sql.DB, timeout time.Duration) {
+	go func() {
+		for {
+			time.Sleep(interval)
+			refreshUsersCache(db, timeout)
+		}
+	}()
+}
+
+func refreshUsersCache(db *sql.DB, timeout time.Duration) {
+	usersCache.Lock()
+	defer usersCache.Unlock()
+	newUsers, err := getUsers(db, timeout)
+	if err != nil {
+		log.Errorf("refreshing users cache: %s", err.Error())
+	} else {
+		usersCache.userMap = newUsers
+		usersCache.usernamesByToken = createTokenToUsernameMap(newUsers)
+		usersCache.initialized = true
+		log.Infof("refreshed users cache (len = %d)", len(usersCache.userMap))
+	}
+}
+
+func createTokenToUsernameMap(users map[string]user) map[string]string {
+	tokenToUserName := make(map[string]string)
+	for username, u := range users {
+		if u.Token == nil || u.RoleName == disallowed {
+			continue
+		}
+		tokenToUserName[*u.Token] = username
+	}
+	return tokenToUserName
+}
+
+func getUsers(db *sql.DB, timeout time.Duration) (map[string]user, error) {
+	dbCtx, dbClose := context.WithTimeout(context.Background(), timeout)
+	defer dbClose()
+	roles := make(map[int]role)
+	newUsers := make(map[string]user)
+
+	tx, err := db.BeginTx(dbCtx, nil)
+	if err != nil {
+		return nil, errors.New("beginning users transaction: " + err.Error())
+	}
+	defer func() {
+		if err := tx.Commit(); err != nil && err != sql.ErrTxDone {
+			log.Errorln("committing users transaction: " + err.Error())
+		}
+	}()
+	rolesRows, err := tx.QueryContext(dbCtx, getRolesQuery)
+	if err != nil {
+		return nil, errors.New("querying roles: " + err.Error())
+	}
+	defer log.Close(rolesRows, "closing role rows")
+	for rolesRows.Next() {
+		r := role{}
+		if err := rolesRows.Scan(&r.Capabilities, &r.ID, &r.Name, &r.PrivLevel); err != nil {
+			return nil, errors.New("scanning roles: " + err.Error())
+		}
+		roles[r.ID] = r
+	}
+	if err = rolesRows.Err(); err != nil {
+		return nil, errors.New("iterating over role rows: " + err.Error())
+	}
+
+	rows, err := tx.QueryContext(dbCtx, getUsersQuery)
+	if err != nil {
+		return nil, errors.New("querying users: " + err.Error())
+	}
+	defer log.Close(rows, "closing users rows")
+	for rows.Next() {
+		u := user{}
+		if err := rows.Scan(&u.ID, &u.LocalPasswd, &u.Role, &u.TenantID, &u.Token, &u.UCDN, &u.UserName); err != nil {
+			return nil, errors.New("scanning users: " + err.Error())
+		}
+		r := roles[u.Role]
+		u.RoleName = r.Name
+		u.PrivLevel = r.PrivLevel
+		u.Capabilities = r.Capabilities
+		u.perms = make(map[string]struct{}, len(u.Capabilities))
+		for _, perm := range u.Capabilities {
+			u.perms[perm] = struct{}{}
+		}
+		newUsers[u.UserName] = u
+	}
+	if err = rows.Err(); err != nil {
+		return nil, errors.New("iterating over user rows: " + err.Error())
+	}
+	return newUsers, nil
+}
diff --git a/traffic_ops/traffic_ops_golang/auth/usercache_test.go b/traffic_ops/traffic_ops_golang/auth/usercache_test.go
new file mode 100644
index 0000000..ce10ebdf
--- /dev/null
+++ b/traffic_ops/traffic_ops_golang/auth/usercache_test.go
@@ -0,0 +1,87 @@
+package auth
+
+/*
+ * 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 (
+	"reflect"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/apache/trafficcontrol/lib/go-util"
+
+	"gopkg.in/DATA-DOG/go-sqlmock.v1"
+)
+
+func TestGetUsers(t *testing.T) {
+	db, mock, err := sqlmock.New()
+	if err != nil {
+		t.Fatalf("creating new sqlmock: %v", err)
+	}
+	defer db.Close()
+	expectedRoles := []role{
+		{
+			Capabilities: []string{"foo", "bar"},
+			ID:           1,
+			Name:         "foo_role",
+			PrivLevel:    42,
+		},
+	}
+	expectedUsers := map[string]user{
+		"user1": {
+			CurrentUser: CurrentUser{
+				UserName:     "user1",
+				ID:           1,
+				PrivLevel:    42,
+				TenantID:     1,
+				Role:         1,
+				RoleName:     "foo_role",
+				Capabilities: []string{"foo", "bar"},
+				UCDN:         "ucdn1",
+				perms: map[string]struct{}{
+					"foo": {},
+					"bar": {},
+				},
+			},
+			LocalPasswd: util.StrPtr("foo"),
+			Token:       util.StrPtr("bar"),
+		},
+	}
+	roleRows := sqlmock.NewRows([]string{"capabilities", "role", "role_name", "priv_level"})
+	userRows := sqlmock.NewRows([]string{"id", "local_passwd", "role", "tenant_id", "token", "ucdn", "username"})
+
+	for _, r := range expectedRoles {
+		roleRows.AddRow("{"+strings.Join(r.Capabilities, ",")+"}", r.ID, r.Name, r.PrivLevel)
+	}
+	for _, u := range expectedUsers {
+		userRows.AddRow(u.ID, u.LocalPasswd, u.Role, u.TenantID, u.Token, u.UCDN, u.UserName)
+	}
+	mock.ExpectBegin()
+	mock.ExpectQuery("SELECT.+").WillReturnRows(roleRows)
+	mock.ExpectQuery("SELECT.+").WillReturnRows(userRows)
+	mock.ExpectCommit()
+	actualUsers, err := getUsers(db, 10*time.Second)
+	if err != nil {
+		t.Fatalf("getUsers expected: nil error, actual: %v", err)
+	}
+	if !reflect.DeepEqual(expectedUsers, actualUsers) {
+		t.Errorf("getUsers expected: %v, actual: %v", expectedUsers, actualUsers)
+	}
+}
diff --git a/traffic_ops/traffic_ops_golang/cachegroup/dspost.go b/traffic_ops/traffic_ops_golang/cachegroup/dspost.go
index 2f982b9..e0eeb2b 100644
--- a/traffic_ops/traffic_ops_golang/cachegroup/dspost.go
+++ b/traffic_ops/traffic_ops_golang/cachegroup/dspost.go
@@ -328,7 +328,7 @@ INSERT INTO profile_parameter (parameter, profile) (
 
 func getDSTenants(tx *sql.Tx, dsIDs []int) ([]int, error) {
 	q := `
-SELECT COALESCE(tenant_id, 0) FROM deliveryservice
+SELECT tenant_id FROM deliveryservice
 WHERE deliveryservice.id = ANY($1)
 `
 	rows, err := tx.Query(q, pq.Array(dsIDs))
diff --git a/traffic_ops/traffic_ops_golang/config/config.go b/traffic_ops/traffic_ops_golang/config/config.go
index 147d876..f9680c1 100644
--- a/traffic_ops/traffic_ops_golang/config/config.go
+++ b/traffic_ops/traffic_ops_golang/config/config.go
@@ -38,32 +38,33 @@ import (
 
 // Config reflects the structure of the cdn.conf file
 type Config struct {
-	URL                     *url.URL `json:"-"`
-	CertPath                string   `json:"-"`
-	KeyPath                 string   `json:"-"`
-	ConfigHypnotoad         `json:"hypnotoad"`
-	ConfigTrafficOpsGolang  `json:"traffic_ops_golang"`
-	ConfigTO                *ConfigTO   `json:"to"`
-	SMTP                    *ConfigSMTP `json:"smtp"`
-	ConfigPortal            `json:"portal"`
-	ConfigLetsEncrypt       `json:"lets_encrypt"`
-	ConfigAcmeRenewal       `json:"acme_renewal"`
-	AcmeAccounts            []ConfigAcmeAccount `json:"acme_accounts"`
-	DB                      ConfigDatabase      `json:"db"`
-	Secrets                 []string            `json:"secrets"`
-	TrafficVaultEnabled     bool
-	ConfigLDAP              *ConfigLDAP
-	LDAPEnabled             bool
-	LDAPConfPath            string `json:"ldap_conf_location"`
-	ConfigInflux            *ConfigInflux
-	InfluxEnabled           bool
-	InfluxDBConfPath        string `json:"influxdb_conf_path"`
-	Version                 string
-	DisableAutoCertDeletion bool                    `json:"disable_auto_cert_deletion"`
-	UseIMS                  bool                    `json:"use_ims"`
-	RoleBasedPermissions    bool                    `json:"role_based_permissions"`
-	DefaultCertificateInfo  *DefaultCertificateInfo `json:"default_certificate_info"`
-	Cdni                    *CdniConf               `json:"cdni"`
+	URL                         *url.URL `json:"-"`
+	CertPath                    string   `json:"-"`
+	KeyPath                     string   `json:"-"`
+	ConfigHypnotoad             `json:"hypnotoad"`
+	ConfigTrafficOpsGolang      `json:"traffic_ops_golang"`
+	ConfigTO                    *ConfigTO   `json:"to"`
+	SMTP                        *ConfigSMTP `json:"smtp"`
+	ConfigPortal                `json:"portal"`
+	ConfigLetsEncrypt           `json:"lets_encrypt"`
+	ConfigAcmeRenewal           `json:"acme_renewal"`
+	AcmeAccounts                []ConfigAcmeAccount `json:"acme_accounts"`
+	DB                          ConfigDatabase      `json:"db"`
+	Secrets                     []string            `json:"secrets"`
+	TrafficVaultEnabled         bool
+	ConfigLDAP                  *ConfigLDAP
+	UserCacheRefreshIntervalSec int `json:"user_cache_refresh_interval_sec"`
+	LDAPEnabled                 bool
+	LDAPConfPath                string `json:"ldap_conf_location"`
+	ConfigInflux                *ConfigInflux
+	InfluxEnabled               bool
+	InfluxDBConfPath            string `json:"influxdb_conf_path"`
+	Version                     string
+	DisableAutoCertDeletion     bool                    `json:"disable_auto_cert_deletion"`
+	UseIMS                      bool                    `json:"use_ims"`
+	RoleBasedPermissions        bool                    `json:"role_based_permissions"`
+	DefaultCertificateInfo      *DefaultCertificateInfo `json:"default_certificate_info"`
+	Cdni                        *CdniConf               `json:"cdni"`
 }
 
 // ConfigHypnotoad carries http setting for hypnotoad (mojolicious) server
@@ -432,6 +433,9 @@ func ParseConfig(cfg Config) (Config, error) {
 	if cfg.DBQueryTimeoutSeconds == 0 {
 		cfg.DBQueryTimeoutSeconds = DefaultDBQueryTimeoutSecs
 	}
+	if cfg.UserCacheRefreshIntervalSec < 0 {
+		cfg.UserCacheRefreshIntervalSec = 0
+	}
 
 	invalidTOURLStr := ""
 	var err error
diff --git a/traffic_ops/traffic_ops_golang/config/config_test.go b/traffic_ops/traffic_ops_golang/config/config_test.go
index 4bb03ea..70c03d1 100644
--- a/traffic_ops/traffic_ops_golang/config/config_test.go
+++ b/traffic_ops/traffic_ops_golang/config/config_test.go
@@ -98,6 +98,7 @@ const (
 		"pid_file" : "/var/run/traffic_ops.pid",
 		"workers" : 12
 	},
+	"user_cache_refresh_interval_sec": 30,
 	"disable_auto_cert_deletion": true,
 	"traffic_ops_golang" : {
 		"port" : "443",
@@ -237,6 +238,9 @@ func TestLoadConfig(t *testing.T) {
 	if cfg.TrafficVaultBackend != "something" {
 		t.Errorf("expected traffic_vault_backend to be 'something', actual: '%s'", cfg.TrafficVaultBackend)
 	}
+	if cfg.UserCacheRefreshIntervalSec != 30 {
+		t.Errorf("expected user_refresh_interval_sec: 30, actual: %d", cfg.UserCacheRefreshIntervalSec)
+	}
 	tvConfig := make(map[string]string)
 	err = json.Unmarshal(cfg.TrafficVaultConfig, &tvConfig)
 	if err != nil {
diff --git a/traffic_ops/traffic_ops_golang/login/logout_test.go b/traffic_ops/traffic_ops_golang/login/logout_test.go
index 99e9689..daf89e1 100644
--- a/traffic_ops/traffic_ops_golang/login/logout_test.go
+++ b/traffic_ops/traffic_ops_golang/login/logout_test.go
@@ -57,12 +57,12 @@ func TestLogout(t *testing.T) {
 	defer db.Close()
 
 	cols := []string{
-		"r.priv_level",
-		"r.id as role",
-		"u.id",
-		"u.username",
-		"COALESCE(u.tenant_id, -1) AS tenant_id",
-		"ARRAY(SELECT rc.cap_name FROM role_capability AS rc WHERE rc.role_id=r.id) AS capabilities",
+		"priv_level",
+		"role",
+		"id",
+		"username",
+		"tenant_id",
+		"capabilities",
 	}
 
 	mock.ExpectBegin()
diff --git a/traffic_ops/traffic_ops_golang/traffic_ops_golang.go b/traffic_ops/traffic_ops_golang/traffic_ops_golang.go
index 09e651e..a04c486 100644
--- a/traffic_ops/traffic_ops_golang/traffic_ops_golang.go
+++ b/traffic_ops/traffic_ops_golang/traffic_ops_golang.go
@@ -143,6 +143,8 @@ func main() {
 	db.SetMaxIdleConns(cfg.DBMaxIdleConnections)
 	db.SetConnMaxLifetime(time.Duration(cfg.DBConnMaxLifetimeSeconds) * time.Second)
 
+	auth.InitUsersCache(time.Duration(cfg.UserCacheRefreshIntervalSec)*time.Second, db.DB, time.Duration(cfg.DBQueryTimeoutSeconds)*time.Second)
+
 	trafficVault := setupTrafficVault(*riakConfigFileName, &cfg)
 
 	// TODO combine