Skip to content

Commit

Permalink
server, ui: template logged in user into index.html
Browse files Browse the repository at this point in the history
...instead of serving a static index.html generated at build time via a
webpack plugin. This allows the UI to decide whether it needs to show a
login UI (and show the logged in username if there is one) without
making an additional request to the backend.

Release note: None
  • Loading branch information
Pete Vilter authored and couchand committed May 16, 2018
1 parent aa9293b commit bedcbed
Show file tree
Hide file tree
Showing 8 changed files with 587 additions and 59 deletions.
82 changes: 54 additions & 28 deletions pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,46 +280,43 @@ RETURNING id
type authenticationMux struct {
server *authenticationServer
inner http.Handler

allowAnonymous bool
}

func newAuthenticationMux(s *authenticationServer, inner http.Handler) *authenticationMux {
func newAuthenticationMuxAllowAnonymous(
s *authenticationServer, inner http.Handler,
) *authenticationMux {
return &authenticationMux{
server: s,
inner: inner,
server: s,
inner: inner,
allowAnonymous: true,
}
}

func (am *authenticationMux) ServeHTTP(w http.ResponseWriter, req *http.Request) {
// Validate the returned cookie.
rawCookie, err := req.Cookie(sessionCookieName)
if err != nil {
err = errors.Wrap(err, "a valid authentication cookie is required")
http.Error(w, err.Error(), http.StatusUnauthorized)
return
func newAuthenticationMuxDisallowAnonymous(
s *authenticationServer, inner http.Handler,
) *authenticationMux {
return &authenticationMux{
server: s,
inner: inner,
allowAnonymous: false,
}
}

cookie, err := decodeSessionCookie(rawCookie)
if err != nil {
err = errors.Wrap(err, "a valid authentication cookie is required")
http.Error(w, err.Error(), http.StatusUnauthorized)
return
}
type loggedInUserKey struct{}

valid, _, err := am.server.verifySession(req.Context(), cookie)
if err != nil {
http.Error(w, apiInternalError(req.Context(), err).Error(), http.StatusInternalServerError)
return
}
if !valid {
http.Error(w, "The provided authentication session could not be validated.", http.StatusUnauthorized)
func (am *authenticationMux) ServeHTTP(w http.ResponseWriter, req *http.Request) {
username, httpCode, err := am.getSession(w, req)
if !am.allowAnonymous && err != nil {
http.Error(w, err.Error(), httpCode)
return
}

// TODO(mrtracy): At this point, we should set the session ID and username
// on the request context. However, GRPC Gateway does not correctly use the
// request context, and even if it did we are not providing any
// authorization for API methods (only authentication).
am.inner.ServeHTTP(w, req)
newCtx := context.WithValue(req.Context(), loggedInUserKey{}, username)
newReq := req.WithContext(newCtx)

am.inner.ServeHTTP(w, newReq)
}

func encodeSessionCookie(sessionCookie *serverpb.SessionCookie) (*http.Cookie, error) {
Expand All @@ -337,6 +334,35 @@ func encodeSessionCookie(sessionCookie *serverpb.SessionCookie) (*http.Cookie, e
}, nil
}

// getSession decodes the cookie from the request, looks up the corresponding session, and
// returns the logged in user name. If there's an error, it returns an error value and the
// HTTP error code.
func (am *authenticationMux) getSession(w http.ResponseWriter, req *http.Request) (string, int, error) {
// Validate the returned cookie.
rawCookie, err := req.Cookie(sessionCookieName)
if err != nil {
return "", http.StatusUnauthorized, err
}

cookie, err := decodeSessionCookie(rawCookie)
if err != nil {
err = errors.Wrap(err, "a valid authentication cookie is required")
return "", http.StatusUnauthorized, err
}

valid, username, err := am.server.verifySession(req.Context(), cookie)
if err != nil {
err := apiInternalError(req.Context(), err)
return "", http.StatusInternalServerError, err
}
if !valid {
err := errors.New("the provided authentication session could not be validated")
return "", http.StatusUnauthorized, err
}

return username, 0, nil
}

func decodeSessionCookie(encodedCookie *http.Cookie) (*serverpb.SessionCookie, error) {
// Cookie value should be a base64 encoded protobuf.
cookieBytes, err := base64.StdEncoding.DecodeString(encodedCookie.Value)
Expand Down
61 changes: 50 additions & 11 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@ import (
"compress/gzip"
"context"
"crypto/tls"
"encoding/json"
"fmt"
"html/template"
"io"
"io/ioutil"
"math"
Expand All @@ -33,11 +35,6 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/sql/stats"

"github.com/elazarl/go-bindata-assetfs"
raven "github.com/getsentry/raven-go"
gwruntime "github.com/grpc-ecosystem/grpc-gateway/runtime"
Expand All @@ -62,6 +59,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/distsqlrun"
"github.com/cockroachdb/cockroach/pkg/sql/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sqlmigrations"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/engine"
Expand Down Expand Up @@ -1172,14 +1173,18 @@ func (s *Server) Start(ctx context.Context) error {
// endpoints.
s.mux.Handle(debug.Endpoint, debug.NewServer(s.st))

// Also throw the landing page in there. It won't work well, but it's better than a 404.
// The remaining endpoints will be opened late, when we're sure that the subsystems they
// talk to are functional.
s.mux.Handle("/", http.FileServer(&assetfs.AssetFS{
fileServer := http.FileServer(&assetfs.AssetFS{
Asset: ui.Asset,
AssetDir: ui.AssetDir,
AssetInfo: ui.AssetInfo,
}))
})

// Serve UI assets. This needs to be before the gRPC handlers are registered, otherwise
// the `s.mux.Handle("/", ...)` would cover all URLs, allowing anonymous access.
maybeAuthMux := newAuthenticationMuxAllowAnonymous(
s.authentication, serveUIAssets(fileServer, s.cfg),
)
s.mux.Handle("/", maybeAuthMux)

// Initialize grpc-gateway mux and context in order to get the /health
// endpoint working even before the node has fully initialized.
Expand All @@ -1202,7 +1207,7 @@ func (s *Server) Start(ctx context.Context) error {

var authHandler http.Handler = gwMux
if s.cfg.RequireWebSession() {
authHandler = newAuthenticationMux(s.authentication, authHandler)
authHandler = newAuthenticationMuxDisallowAnonymous(s.authentication, authHandler)
}

// Setup HTTP<->gRPC handlers.
Expand Down Expand Up @@ -1905,3 +1910,37 @@ func officialAddr(

return util.NewUnresolvedAddr(lnAddr.Network(), net.JoinHostPort(host, port)), nil
}

func serveUIAssets(fileServer http.Handler, cfg Config) http.Handler {
return http.HandlerFunc(func(writer http.ResponseWriter, request *http.Request) {
if request.URL.Path != "/" {
fileServer.ServeHTTP(writer, request)
return
}

// Construct arguments for template.
tmplArgs := ui.IndexHTMLArgs{
LoginEnabled: cfg.RequireWebSession(),
}
loggedInUser, ok := request.Context().Value(loggedInUserKey{}).(string)
if ok && loggedInUser != "" {
tmplArgs.LoggedInUser = &loggedInUser
}

argsJSON, err := json.Marshal(tmplArgs)
if err != nil {
http.Error(writer, err.Error(), 500)
}

// Execute the template.
writer.Header().Add("Content-Type", "text/html")
if err := ui.IndexHTMLTemplate.Execute(writer, map[string]template.JS{
"DataFromServer": template.JS(string(argsJSON)),
}); err != nil {
wrappedErr := errors.Wrap(err, "templating index.html")
http.Error(writer, wrappedErr.Error(), 500)
log.Error(request.Context(), wrappedErr)
return
}
})
}
97 changes: 97 additions & 0 deletions pkg/server/server_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -935,3 +935,100 @@ func TestPersistHLCUpperBound(t *testing.T) {
})
}
}

func TestServeIndexHTML(t *testing.T) {
defer leaktest.AfterTest(t)()

const htmlTemplate = `<!DOCTYPE html>
<html>
<head>
<title>Cockroach Console</title>
<meta charset="UTF-8">
<link href="favicon.ico" rel="shortcut icon">
</head>
<body>
<div id="react-layout"></div>
<script>
window.dataFromServer = %s;
</script>
<script src="protos.dll.js" type="text/javascript"></script>
<script src="vendor.dll.js" type="text/javascript"></script>
<script src="bundle.js" type="text/javascript"></script>
</body>
</html>
`

t.Run("Insecure mode", func(t *testing.T) {
s, _, _ := serverutils.StartServer(t, base.TestServerArgs{
Insecure: true,
})
defer s.Stopper().Stop(context.TODO())
tsrv := s.(*TestServer)

client, err := tsrv.GetHTTPClient()
if err != nil {
t.Fatal(err)
}

resp, err := client.Get(s.AdminURL())
if err != nil {
t.Fatal(err)
}
if resp.StatusCode != 200 {
t.Fatalf("expected status code 200; got %d", resp.StatusCode)
}
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
t.Fatal(err)
}
respString := string(respBytes)
expected := fmt.Sprintf(htmlTemplate, `{"LoginEnabled":false,"LoggedInUser":null}`)
if respString != expected {
t.Fatalf("expected %s; got %s", expected, respString)
}
})

t.Run("Secure mode", func(t *testing.T) {
s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.TODO())
tsrv := s.(*TestServer)

loggedInClient, err := tsrv.GetAuthenticatedHTTPClient()
if err != nil {
t.Fatal(err)
}
loggedOutClient, err := tsrv.GetHTTPClient()
if err != nil {
t.Fatal(err)
}

cases := []struct {
client http.Client
json string
}{
{loggedInClient, `{"LoginEnabled":true,"LoggedInUser":"authentic_user"}`},
{loggedOutClient, `{"LoginEnabled":true,"LoggedInUser":null}`},
}

for _, testCase := range cases {
resp, err := testCase.client.Get(s.AdminURL())
if err != nil {
t.Fatal(err)
}
if resp.StatusCode != 200 {
t.Fatalf("expected status code 200; got %d", resp.StatusCode)
}
respBytes, err := ioutil.ReadAll(resp.Body)
if err != nil {
t.Fatal(err)
}
respString := string(respBytes)
expected := fmt.Sprintf(htmlTemplate, testCase.json)
if respString != expected {
t.Fatalf("expected %s; got %s", expected, respString)
}
}
})
}
4 changes: 3 additions & 1 deletion pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -463,14 +463,16 @@ func (ts *TestServer) GetHTTPClient() (http.Client, error) {
return ts.Cfg.GetHTTPClient()
}

const authenticatedUserName = "authentic_user"

// GetAuthenticatedHTTPClient implements TestServerInterface.
func (ts *TestServer) GetAuthenticatedHTTPClient() (http.Client, error) {
ts.authClient.once.Do(func() {
// Create an authentication session for an arbitrary user. We do not
// currently have an authorization mechanism, so a specific user is not
// necessary.
ts.authClient.err = func() error {
id, secret, err := ts.authentication.newAuthSession(context.TODO(), "authentic_user")
id, secret, err := ts.authentication.newAuthSession(context.TODO(), authenticatedUserName)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ui/package.json
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
"bytebuffer": "^5.0.1",
"cache-loader": "^1.0.3",
"chai": "^4.1.0",
"copy-webpack-plugin": "^4.5.1",
"css-loader": "^0.28.0",
"d3": "<4.0.0",
"dagre-layout": "^0.8.0",
Expand Down
39 changes: 39 additions & 0 deletions pkg/ui/ui.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ package ui

import (
"fmt"
"html/template"
"os"

"github.com/cockroachdb/cockroach/pkg/build"
Expand Down Expand Up @@ -67,3 +68,41 @@ var AssetDir = func(name string) ([]string, error) {
// AssetInfo loads and returns metadata for the asset with the given name. It
// returns an error if the asset could not be found or could not be loaded.
var AssetInfo func(name string) (os.FileInfo, error)

// IndexHTMLTemplate takes arguments about the current session and returns HTML which
// includes the UI JavaScript bundles, plus a script tag which sets the currently logged in user
// so that the UI JavaScript can decide whether to show a login page.
var IndexHTMLTemplate *template.Template

// IndexHTMLArgs are the arguments to IndexHTMLTemplate.
type IndexHTMLArgs struct {
LoginEnabled bool
LoggedInUser *string
}

func init() {
t, err := template.New("index").Parse(`<!DOCTYPE html>
<html>
<head>
<title>Cockroach Console</title>
<meta charset="UTF-8">
<link href="favicon.ico" rel="shortcut icon">
</head>
<body>
<div id="react-layout"></div>
<script>
window.dataFromServer = {{ .DataFromServer }};
</script>
<script src="protos.dll.js" type="text/javascript"></script>
<script src="vendor.dll.js" type="text/javascript"></script>
<script src="bundle.js" type="text/javascript"></script>
</body>
</html>
`)
if err != nil {
panic(fmt.Sprintf("can't parse template: %s", err))
}
IndexHTMLTemplate = t
}
Loading

0 comments on commit bedcbed

Please sign in to comment.