diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 07ac9774cb23..d92739e7c33f 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -53,6 +53,7 @@ go_library( "server_http.go", "server_sql.go", "server_systemlog_gc.go", + "session_writer.go", "settings_cache.go", "sql_stats.go", "start_listen.go", diff --git a/pkg/server/api_v2_auth.go b/pkg/server/api_v2_auth.go index 001b2f74444a..ef51fe5a3dc2 100644 --- a/pkg/server/api_v2_auth.go +++ b/pkg/server/api_v2_auth.go @@ -14,6 +14,7 @@ import ( "context" "encoding/base64" "net/http" + "strings" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -325,7 +326,17 @@ func (a *authenticationV2Mux) getSession( var decoded []byte var err error for i := range possibleSessions { - decoded, err = base64.StdEncoding.DecodeString(possibleSessions[i]) + var session string + // This case is if the session cookie has a multi-tenant pattern. + if strings.Contains(possibleSessions[i], ",") { + session, err = findSessionCookieValue(possibleSessions[i], req.Cookies()) + if err != nil { + return "", nil, http.StatusBadRequest, err + } + } else { + session = possibleSessions[i] + } + decoded, err = base64.StdEncoding.DecodeString(session) if err != nil { log.Warningf(ctx, "attempted to decode session but failed: %v", err) continue diff --git a/pkg/server/authentication.go b/pkg/server/authentication.go index b4bc6914b5ad..7656c5786c60 100644 --- a/pkg/server/authentication.go +++ b/pkg/server/authentication.go @@ -19,6 +19,7 @@ import ( "fmt" "net/http" "strconv" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -30,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -627,7 +629,16 @@ func (am *authenticationMux) getSession( continue } found = true - cookie, err = decodeSessionCookie(c) + sessionCookie := c + // This case is if the session cookie has a multi-tenant pattern. + if strings.Contains(c.Value, ",") { + sessionVal, err := findSessionCookieValue(c.Value, cookies) + if err != nil { + return "", nil, apiInternalError(req.Context(), err) + } + sessionCookie.Value = sessionVal + } + cookie, err = decodeSessionCookie(sessionCookie) if err != nil { // Multiple cookies with the same name may be included in the // header. We continue searching even if we find a matching @@ -696,3 +707,66 @@ func forwardAuthenticationMetadata(ctx context.Context, _ *http.Request) metadat } return md } + +// sessionCookieValue defines the data needed to construct the +// aggregate session cookie in the order provided. +type sessionCookieValue struct { + // The name of the tenant. + name string + // The value of set-cookie. + setCookie string +} + +// createAggregatedSessionCookieValue is used for multi-tenant login. +// It takes a tenant name to set cookie map and converts it to a single +// string which is the aggregated session. Currently the format of the +// aggregated session is: session,tenant_name&session2,tenant_name2 etc. +func createAggregatedSessionCookieValue(sessionCookieValue []sessionCookieValue) string { + var sessionsStr string + for _, val := range sessionCookieValue { + sessionCookieSlice := strings.Split(strings.ReplaceAll(val.setCookie, "session=", ""), ";") + sessionsStr += sessionCookieSlice[0] + "," + val.name + "&" + } + if len(sessionsStr) > 0 { + sessionsStr = sessionsStr[:len(sessionsStr)-1] + } + return sessionsStr +} + +// findSessionCookieValue finds the encoded session in an aggregated +// session cookie value established in multi-tenant clusters. If the +// method cannot find a match between the tenant name and session +// or cannot find the tenant cookie value, it will return an empty +// string to indicate this. +func findSessionCookieValue(sessionStr string, cookies []*http.Cookie) (string, error) { + tenantName := findTenantCookieValue(cookies) + if tenantName == "" { + return "", errors.New("unable to find tenant cookie") + } + sessionSlice := strings.FieldsFunc(sessionStr, func(r rune) bool { + return r == ',' || r == '&' + }) + var encodedSession string + for idx, val := range sessionSlice { + if val == tenantName && idx > 0 { + encodedSession = sessionSlice[idx-1] + } + } + if encodedSession == "" { + return "", errors.Newf("unable to find session cookie value that matches tenant %q", tenantName) + } + return encodedSession, nil +} + +// findTenantCookieValue iterates through all request cookies +// in order to find the value of the tenant cookie. If +// the tenant cookie is not found, it assumes the default +// to be the system tenant. +func findTenantCookieValue(cookies []*http.Cookie) string { + for _, c := range cookies { + if c.Name == TenantSelectCookieName { + return c.Value + } + } + return catconstants.SystemTenantName +} diff --git a/pkg/server/authentication_test.go b/pkg/server/authentication_test.go index 11ac5956aceb..56b89345fed9 100644 --- a/pkg/server/authentication_test.go +++ b/pkg/server/authentication_test.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/errors" "github.com/gogo/protobuf/jsonpb" "github.com/lib/pq" + "github.com/stretchr/testify/require" "golang.org/x/crypto/bcrypt" "google.golang.org/grpc" "google.golang.org/grpc/credentials" @@ -838,3 +839,70 @@ func TestGRPCAuthentication(t *testing.T) { }) } } + +func TestCreateAggregatedSessionCookieValue(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + tests := []struct { + name string + mapArg []sessionCookieValue + resExpected string + }{ + {"standard arg", []sessionCookieValue{ + {name: "system", setCookie: "session=abcd1234"}, + {name: "app", setCookie: "session=efgh5678"}}, + "abcd1234,system&efgh5678,app", + }, + {"empty arg", []sessionCookieValue{}, ""}, + } + for _, test := range tests { + t.Run(fmt.Sprintf("create-session-cookie/%s", test.name), func(t *testing.T) { + res := createAggregatedSessionCookieValue(test.mapArg) + require.Equal(t, test.resExpected, res) + }) + } +} + +func TestFindSessionCookieValue(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + normalSessionStr := "abcd1234,system&efgh5678,app" + tests := []struct { + name string + sessionArg string + cookieArg []*http.Cookie + resExpected string + errorExpected bool + }{ + {"standard args", normalSessionStr, []*http.Cookie{ + { + Name: TenantSelectCookieName, + Value: "system", + Path: "/", + }, + }, "abcd1234", false}, + {"no tenant cookie", normalSessionStr, []*http.Cookie{}, "abcd1234", false}, + {"invalid tenant cookie", normalSessionStr, []*http.Cookie{ + { + Name: TenantSelectCookieName, + Value: "", + Path: "/", + }, + }, "", true}, + {"no tenant name match", normalSessionStr, []*http.Cookie{ + { + Name: TenantSelectCookieName, + Value: "app2", + Path: "/", + }, + }, "", true}, + } + for _, test := range tests { + t.Run(fmt.Sprintf("find-session-cookie/%s", test.name), func(t *testing.T) { + res, err := findSessionCookieValue(test.sessionArg, test.cookieArg) + require.Equal(t, test.resExpected, res) + require.Equal(t, test.errorExpected, err != nil) + }) + } +} diff --git a/pkg/server/server_controller.go b/pkg/server/server_controller.go index 5c39ad65c449..bee39fca7397 100644 --- a/pkg/server/server_controller.go +++ b/pkg/server/server_controller.go @@ -11,14 +11,16 @@ package server import ( + "bytes" "context" - "fmt" + "io" "net" "net/http" "net/url" "os" "path/filepath" "strconv" + "strings" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -195,40 +197,219 @@ const TenantSelectCookieName = `tenant` // If no tenant is specified, the default tenant is used. func (c *serverController) httpMux(w http.ResponseWriter, r *http.Request) { ctx := r.Context() - tenantName := getTenantNameFromHTTPRequest(r) + tenantName, nameProvided := getTenantNameFromHTTPRequest(r) s, err := c.getOrCreateServer(ctx, tenantName) if err != nil { log.Warningf(ctx, "unable to start server for tenant %q: %v", tenantName, err) w.WriteHeader(http.StatusInternalServerError) - fmt.Fprintf(w, "cannot find tenant") return } - + // if the client didnt specify tenant name call these for login/logout. + if !nameProvided { + switch r.URL.Path { + case loginPath: + c.attemptLoginToAllTenants().ServeHTTP(w, r) + return + case logoutPath: + c.attemptLogoutFromAllTenants().ServeHTTP(w, r) + return + } + } s.getHTTPHandlerFn()(w, r) } -func getTenantNameFromHTTPRequest(r *http.Request) string { +func getTenantNameFromHTTPRequest(r *http.Request) (string, bool) { // Highest priority is manual override on the URL query parameters. const tenantNameParamInQueryURL = "tenant_name" if tenantName := r.URL.Query().Get(tenantNameParamInQueryURL); tenantName != "" { - return tenantName + return tenantName, true } // If not in parameters, try an explicit header. if tenantName := r.Header.Get(TenantSelectHeader); tenantName != "" { - return tenantName + return tenantName, true } // No parameter, no explicit header. Is there a cookie? if c, _ := r.Cookie(TenantSelectCookieName); c != nil && c.Value != "" { - return c.Value + return c.Value, true } // No luck so far. // // TODO(knz): Make the default tenant route for HTTP configurable. // See: https://github.com/cockroachdb/cockroach/issues/91741 - return catconstants.SystemTenantName + return catconstants.SystemTenantName, false +} + +func (c *serverController) getCurrentTenantNames() []string { + var serverNames []string + c.mu.Lock() + for name := range c.mu.servers { + serverNames = append(serverNames, name) + } + c.mu.Unlock() + return serverNames +} + +func getSessionFromCookie(sessionStr string, name string) (string, error) { + sessions := strings.Split(sessionStr, "&") + for _, session := range sessions { + ids := strings.Split(session, ",") + if len(ids) != 2 { + return "", errors.New("session is not in the correct format") + } + if ids[1] == name { + return ids[0], nil + } + } + return "", errors.New("session not found") +} + +// attemptLoginToAllTenants attempts login for each of the tenants and +// if successful, appends the encoded session and tenant name to the +// new session cookie. If login fails for all tenants, the StatusUnauthorized +// code will be set in the header. +func (c *serverController) attemptLoginToAllTenants() http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + ctx := r.Context() + tenantNames := c.getCurrentTenantNames() + var tenantNameToSetCookieSlice []sessionCookieValue + // The request body needs to be cloned since r.Clone() does not do it. + clonedBody, err := io.ReadAll(r.Body) + if err != nil { + log.Warning(ctx, "unable to write body") + w.WriteHeader(http.StatusInternalServerError) + return + } + defer r.Body.Close() + + for _, name := range tenantNames { + // Make a new sessionWriter for every tenant. A fresh header is needed + // each time since the grpc method writes to it. + sw := &sessionWriter{header: w.Header().Clone()} + newReq := r.Clone(ctx) + newReq.Body = io.NopCloser(bytes.NewBuffer(clonedBody)) + server, err := c.getOrCreateServer(ctx, name) + if err != nil { + log.Warningf(ctx, "unable to find tserver for tenant %q: %v", name, err) + w.WriteHeader(http.StatusInternalServerError) + return + } + // Invoke the handler, passing the new sessionWriter and the cloned + // request. + server.getHTTPHandlerFn().ServeHTTP(sw, newReq) + // Extract the entire set-cookie from the header. The session cookie will be + // embedded within set-cookie. + setCookieHeader := sw.Header().Get("set-cookie") + if len(setCookieHeader) == 0 { + log.Warningf(ctx, "unable to find session cookie for tenant %q", name) + } else { + tenantNameToSetCookieSlice = append(tenantNameToSetCookieSlice, sessionCookieValue{ + name: name, + setCookie: setCookieHeader, + }) + } + } + // If the map has entries, the method to create the aggregated session should + // be called and cookies should be set. Otherwise, login was not successful + // for any of the tenants. + if len(tenantNameToSetCookieSlice) > 0 { + sessionsStr := createAggregatedSessionCookieValue(tenantNameToSetCookieSlice) + cookie := http.Cookie{ + Name: SessionCookieName, + Value: sessionsStr, + Path: "/", + HttpOnly: false, + } + http.SetCookie(w, &cookie) + // The tenant cookie needs to be set at some point in order for + // the dropdown to have a current selection on first load. Subject to change + // once this issue is resolved: https://github.com/cockroachdb/cockroach/issues/91741. + cookie = http.Cookie{ + Name: TenantSelectCookieName, + Value: catconstants.SystemTenantName, + Path: "/", + HttpOnly: false, + } + http.SetCookie(w, &cookie) + w.WriteHeader(http.StatusOK) + } else { + w.WriteHeader(http.StatusUnauthorized) + } + }) +} + +// attemptLogoutFromAllTenants attempts logout for each of the tenants and +// clears both the session cookie and tenant cookie. If logout fails, the +// StatusInternalServerError code will be set in the header. +func (c *serverController) attemptLogoutFromAllTenants() http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + ctx := r.Context() + tenantNames := c.getCurrentTenantNames() + // The request body needs to be cloned since r.Clone() does not do it. + clonedBody, err := io.ReadAll(r.Body) + if err != nil { + log.Warning(ctx, "unable to write body") + w.WriteHeader(http.StatusInternalServerError) + return + } + sessionCookie, err := r.Cookie(SessionCookieName) + if err != nil { + log.Warning(ctx, "unable to find session cookie") + w.WriteHeader(http.StatusInternalServerError) + return + } + for _, name := range tenantNames { + // Make a new sessionWriter for every tenant. A fresh header is needed + // each time since the grpc method writes to it. + sw := &sessionWriter{header: w.Header().Clone()} + newReq := r.Clone(ctx) + newReq.Body = io.NopCloser(bytes.NewBuffer(clonedBody)) + // Extract the session which matches to the current tenant name. + relevantSession, err := getSessionFromCookie(sessionCookie.Value, name) + if err != nil { + log.Warningf(ctx, "%q", err) + w.WriteHeader(http.StatusInternalServerError) + return + } + // Set the matching session in the cookie so that the grpc method can + // logout correctly. + newReq.Header.Set("Cookie", "session="+relevantSession) + server, err := c.getOrCreateServer(ctx, name) + if err != nil { + log.Warningf(ctx, "unable to find tserver for tenant %q: %v", name, err) + w.WriteHeader(http.StatusInternalServerError) + return + } + server.getHTTPHandlerFn().ServeHTTP(sw, newReq) + // If a logout was unsuccessful, set cookie will be empty in which case + // set the header to an error status and return. + if sw.header.Get("Set-Cookie") == "" { + log.Warningf(ctx, "logout for tenant %q failed", name) + w.WriteHeader(http.StatusInternalServerError) + return + } + } + // Clear session and tenant cookies after all logouts have completed. + cookie := http.Cookie{ + Name: SessionCookieName, + Value: "", + Path: "/", + HttpOnly: true, + Expires: timeutil.Unix(0, 0), + } + http.SetCookie(w, &cookie) + cookie = http.Cookie{ + Name: TenantSelectCookieName, + Value: "", + Path: "/", + HttpOnly: false, + Expires: timeutil.Unix(0, 0), + } + http.SetCookie(w, &cookie) + w.WriteHeader(http.StatusOK) + }) } // TestingGetSQLAddrForTenant extracts the SQL address for the target tenant. diff --git a/pkg/server/session_writer.go b/pkg/server/session_writer.go new file mode 100644 index 000000000000..20f17741d777 --- /dev/null +++ b/pkg/server/session_writer.go @@ -0,0 +1,37 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package server + +import ( + "bytes" + "net/http" +) + +// sessionWriter implements http.ResponseWriter. It is used +// to extract cookies written to the header when passed in to +// a ServeHTTP function. +type sessionWriter struct { + header http.Header + buf bytes.Buffer + code int +} + +func (sw *sessionWriter) Header() http.Header { + return sw.header +} + +func (sw *sessionWriter) WriteHeader(statusCode int) { + sw.code = statusCode +} + +func (sw *sessionWriter) Write(data []byte) (int, error) { + return sw.buf.Write(data) +} diff --git a/pkg/ui/workspaces/db-console/src/components/pageHeader/pageHeader.styl b/pkg/ui/workspaces/db-console/src/components/pageHeader/pageHeader.styl index c2db48ddf42f..b5c4afd61bf7 100644 --- a/pkg/ui/workspaces/db-console/src/components/pageHeader/pageHeader.styl +++ b/pkg/ui/workspaces/db-console/src/components/pageHeader/pageHeader.styl @@ -20,6 +20,8 @@ .text font-family $font-family--semi-bold color $colors--neutral-8 + :last-child + margin-left auto .page-header > * margin-right $spacing-small diff --git a/pkg/ui/workspaces/db-console/src/redux/cookies.ts b/pkg/ui/workspaces/db-console/src/redux/cookies.ts new file mode 100644 index 000000000000..721594a9103e --- /dev/null +++ b/pkg/ui/workspaces/db-console/src/redux/cookies.ts @@ -0,0 +1,60 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +export const getCurrentNodeIDFromCookie = (cookieName: string): string[] => { + return document.cookie + .split(";") + .map(cookieString => { + return cookieString.split("=").map(kv => { + return kv.trim(); + }); + }) + .find(cookie => { + return cookie[0] === cookieName; + }); +}; + +export const selectTenantsFromCookie = (): string[] => { + const sessionCookieArr = document.cookie + .split(";") + .filter(row => row.trim().startsWith("session=")); + return sessionCookieArr.length > 0 + ? sessionCookieArr[0] + .replace(/["]/g, "") + .replace("session=", "") + .split(/[,&]/g) + .filter((_, idx) => idx % 2 == 1) + : []; +}; + +export const selectCurrentTenantFromCookie = (): string | null => { + const tenantCookieStr = document.cookie + .split(";") + .filter(row => row.trim().startsWith("tenant="))[0]; + return tenantCookieStr ? tenantCookieStr.replace("tenant=", "") : null; +}; + +export const setCookie = ( + key: string, + val: string, + expires?: string, + path?: string, +) => { + let cookieStr = `${key}=${val};`; + if (expires) { + cookieStr += `expires=${expires};`; + } + if (path) { + cookieStr += `path=${path}`; + } else { + cookieStr += "/"; + } + document.cookie = cookieStr; +}; diff --git a/pkg/ui/workspaces/db-console/src/redux/login.ts b/pkg/ui/workspaces/db-console/src/redux/login.ts index 64491f828985..5183dd7d16a6 100644 --- a/pkg/ui/workspaces/db-console/src/redux/login.ts +++ b/pkg/ui/workspaces/db-console/src/redux/login.ts @@ -20,6 +20,7 @@ import { cockroach } from "src/js/protos"; import { getDataFromServer } from "src/util/dataFromServer"; import UserLoginRequest = cockroach.server.serverpb.UserLoginRequest; +import { selectTenantsFromCookie, setCookie } from "./cookies"; const dataFromServer = getDataFromServer(); @@ -234,7 +235,12 @@ export function doLogout(): ThunkAction< > { return dispatch => { dispatch(logoutBeginAction); - + const tenants = selectTenantsFromCookie(); + // If in multi-tenant environment, we need to clear the tenant cookie so that + // we can do a multi-tenant logout. + if (tenants.length > 1) { + setCookie("tenant", ""); + } // Make request to log out, reloading the page whether it succeeds or not. // If there was a successful log out but the network dropped the response somehow, // you'll get the login page on reload. If The logout actually didn't work, you'll diff --git a/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.spec.tsx b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.spec.tsx new file mode 100644 index 000000000000..df8f17172585 --- /dev/null +++ b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.spec.tsx @@ -0,0 +1,43 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +import { + selectTenantsFromCookie, + selectCurrentTenantFromCookie, +} from "oss/src/redux/cookies"; +import React from "react"; +import TenantDropdown from "./tenantDropdown"; +import { shallow } from "enzyme"; + +jest.mock("src/redux/tenantOptions", () => ({ + selectTenantsFromCookies: jest.fn(), + selectCurrentTenantFromCookies: jest.fn(), +})); + +describe("TenantDropdown", () => { + it("returns null if there are no tenants in the session cookie", () => { + ( + selectTenantsFromCookie as jest.MockedFn + ).mockReturnValueOnce([]); + const wrapper = shallow(); + expect(wrapper.isEmptyRender()); + }); + it("returns a dropdown list of tenant options if there are tenant in the session cookie", () => { + ( + selectTenantsFromCookie as jest.MockedFn + ).mockReturnValueOnce(["system", "app"]); + ( + selectCurrentTenantFromCookie as jest.MockedFn< + typeof selectCurrentTenantFromCookie + > + ).mockReturnValueOnce("system"); + const wrapper = shallow(); + expect(wrapper.find({ children: "Tenant system" }).length).toEqual(1); + }); +}); diff --git a/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.styl b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.styl new file mode 100644 index 000000000000..9244a2458a84 --- /dev/null +++ b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.styl @@ -0,0 +1,18 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +@require '~src/components/core/index.styl' + +.tenant-selected + color $colors--neutral-7 + padding-right 6px + font-weight 600 + font-size 14px + font-family $font-family--semi-bold diff --git a/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.tsx b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.tsx new file mode 100644 index 000000000000..0c4c41b0da22 --- /dev/null +++ b/pkg/ui/workspaces/db-console/src/views/app/components/tenantDropdown/tenantDropdown.tsx @@ -0,0 +1,57 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +import { + selectCurrentTenantFromCookie, + selectTenantsFromCookie, + setCookie, +} from "src/redux/cookies"; +import React from "react"; +import { Dropdown } from "@cockroachlabs/cluster-ui"; +import ErrorBoundary from "../errorMessage/errorBoundary"; +import "./tenantDropdown.styl"; + +const tenantIDKey = "tenant"; + +const TenantDropdown = () => { + const tenants = selectTenantsFromCookie(); + const currentTenant = selectCurrentTenantFromCookie(); + + const createDropdownItems = () => { + return ( + tenants?.map(tenantID => { + return { name: "Tenant " + tenantID, value: tenantID }; + }) || [] + ); + }; + + const setTenantCookie = (tenant: string) => { + if (tenant !== currentTenant) { + setCookie(tenantIDKey, tenant); + location.reload(); + } + }; + + if (tenants.length == 0) { + return null; + } + + return ( + + setTenantCookie(tenantID)} + > +
{"Tenant " + currentTenant}
+
+
+ ); +}; + +export default TenantDropdown; diff --git a/pkg/ui/workspaces/db-console/src/views/app/containers/layout/index.tsx b/pkg/ui/workspaces/db-console/src/views/app/containers/layout/index.tsx index 105068622728..b8a7502a8475 100644 --- a/pkg/ui/workspaces/db-console/src/views/app/containers/layout/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/app/containers/layout/index.tsx @@ -40,6 +40,7 @@ import { Badge } from "@cockroachlabs/cluster-ui"; import "./layout.styl"; import "./layoutPanel.styl"; import { getDataFromServer } from "src/util/dataFromServer"; +import TenantDropdown from "../../components/tenantDropdown/tenantDropdown"; export interface LayoutProps { clusterName: string; @@ -99,6 +100,7 @@ class Layout extends React.Component { {clusterName || `Cluster id: ${clusterId || ""}`} +
diff --git a/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx b/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx index c34bed38f20e..f9147d8633e9 100644 --- a/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/reports/containers/debug/index.tsx @@ -28,6 +28,7 @@ import { AdminUIState } from "src/redux/state"; import { nodeIDsStringifiedSelector } from "src/redux/nodes"; import { refreshNodes, refreshUserSQLRoles } from "src/redux/apiReducers"; import { selectHasViewActivityRedactedRole } from "src/redux/user"; +import { getCurrentNodeIDFromCookie, setCookie } from "src/redux/cookies"; const COMMUNITY_URL = "https://www.cockroachlabs.com/community/"; @@ -166,18 +167,11 @@ const ProxyToNodeSelector = (props: ProxyToNodeSelectorProps) => { // currentNodeIDCookie will either be empty or contain two elements // with the cookie name and value we're looking for - const currentNodeIDCookie: string[] = document.cookie - .split(";") - .map(cookieString => { - return cookieString.split("=").map(kv => { - return kv.trim(); - }); - }) - .find(cookie => { - return cookie[0] === remoteNodeIDCookieName; - }); + const currentNodeIDCookie: string[] = getCurrentNodeIDFromCookie( + remoteNodeIDCookieName, + ); const setNodeIDCookie = (nodeID: string) => { - document.cookie = `${remoteNodeIDCookieName}=${nodeID};path=/`; + setCookie(remoteNodeIDCookieName, nodeID); location.reload(); }; let currentNodeID = props.nodeID; diff --git a/pkg/util/httputil/http.go b/pkg/util/httputil/http.go index 5b3edfd6b813..0ebc5da78136 100644 --- a/pkg/util/httputil/http.go +++ b/pkg/util/httputil/http.go @@ -113,6 +113,7 @@ func PostJSONWithRequest( func doJSONRequest( httpClient http.Client, req *http.Request, response protoutil.Message, ) (*http.Response, error) { + req.Header.Set("X-Cockroach-Tenant", "system") if timeout := httpClient.Timeout; timeout > 0 { req.Header.Set("Grpc-Timeout", strconv.FormatInt(timeout.Nanoseconds(), 10)+"n") }