Skip to content

Commit

Permalink
server: add /api/v2/ tree with auth/pagination, port listSessions end…
Browse files Browse the repository at this point in the history
…point

This change adds the skeleton for a new API tree that lives in
`/api/v2/` in the http listener, and currently reimplements the `/sessions/`
endpoint that is also implemented in `/_status/`. The new v2 API tree avoids
the need to use GRPC Gateway, as well as cookie-based authentication which is
less intuitive and idiomatic for REST APIs. Instead, for authentication,
it uses a new session header that needs to be set on every request.

As many RPC fan-out APIs use statusServer.iterateNodes, this change
implements a pagination-aware method, paginatedIterateNodes, that
works on a sorted set of node IDs and arranges results in such a way
to be able to return the next `limit` results of an arbitary slice
after the `next` cursor passed in. An example of how this works in practice
is the new `/api/v2/sessions/` endpoint.

A dependency on gorilla/mux is added to be able to pattern-match
arguments in the URL. This was already an indirect dependency; now it's
a direct dependency of cockroach.

TODO that are likely to fall over into future PRs:
 - API Documentation, need to explore using swagger here.
 - Porting over remaining /_admin/ and /_status/ APIs, incl. SQL based ones

Part of #55947.

Release note (api change): Adds a new API tree, in /api/v2/*, currently
undocumented, that avoids the use of and cookie-based
authentication in favour of sessions in headers, and support for pagination.
  • Loading branch information
itsbilal committed Feb 1, 2021
1 parent 60235e0 commit a889429
Show file tree
Hide file tree
Showing 17 changed files with 2,404 additions and 23 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1169,8 +1169,8 @@ def go_deps():
name = "com_github_gorilla_mux",
build_file_proto_mode = "disable_global",
importpath = "github.com/gorilla/mux",
sum = "h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc=",
version = "v1.7.4",
sum = "h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI=",
version = "v1.8.0",
)
go_repository(
name = "com_github_gorilla_securecookie",
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ require (
github.com/google/pprof v0.0.0-20190109223431-e84dfd68c163
github.com/googleapis/gax-go v2.0.2+incompatible // indirect
github.com/gorhill/cronexpr v0.0.0-20140423231348-a557574d6c02
github.com/gorilla/mux v1.7.4 // indirect
github.com/gorilla/mux v1.8.0
github.com/goware/modvendor v0.3.0
github.com/grpc-ecosystem/grpc-gateway v1.13.0
github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -358,8 +358,8 @@ github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
github.com/gorhill/cronexpr v0.0.0-20140423231348-a557574d6c02 h1:Spo+4PFAGDqULAsZ7J69MOxq4/fwgZ0zvmDTBqpq7yU=
github.com/gorhill/cronexpr v0.0.0-20140423231348-a557574d6c02/go.mod h1:g2644b03hfBX9Ov0ZBDgXXens4rxSxmqFBbhvKv2yVA=
github.com/gorilla/mux v1.7.4 h1:VuZ8uybHlWmqV03+zRzdwKL4tUnIp1MAQtp1mIFE1bc=
github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So=
github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI=
github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So=
github.com/gorilla/securecookie v1.1.1 h1:miw7JPhV+b/lAHSXz4qd/nN9jRiAFV5FwjeKyCS8BvQ=
github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4=
github.com/gorilla/sessions v1.2.0 h1:S7P+1Hm5V/AT9cjEcUD5uDaQSX0OE577aCXgoaKpYbQ=
Expand Down
7 changes: 7 additions & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,8 @@ go_library(
name = "server",
srcs = [
"admin.go",
"api.go",
"api_auth.go",
"api_error.go",
"authentication.go",
"auto_upgrade.go",
Expand All @@ -19,6 +21,7 @@ go_library(
"node.go",
"node_engine_health.go",
"node_tombstone_storage.go",
"pagination.go",
"problem_ranges.go",
"rlimit_bsd.go",
"rlimit_darwin.go",
Expand Down Expand Up @@ -164,6 +167,7 @@ go_library(
"@com_github_cockroachdb_sentry_go//:sentry-go",
"@com_github_elastic_gosigar//:gosigar",
"@com_github_gogo_protobuf//proto",
"@com_github_gorilla_mux//:mux",
"@com_github_grpc_ecosystem_grpc_gateway//runtime:go_default_library",
"@com_github_grpc_ecosystem_grpc_gateway//utilities:go_default_library",
"@com_github_marusama_semaphore//:semaphore",
Expand Down Expand Up @@ -236,6 +240,7 @@ go_test(
"multi_store_test.go",
"node_test.go",
"node_tombstone_storage_test.go",
"pagination_test.go",
"servemode_test.go",
"server_systemlog_gc_test.go",
"server_test.go",
Expand All @@ -247,6 +252,7 @@ go_test(
"testserver_test.go",
"version_cluster_test.go",
],
data = glob(["testdata/**"]),
embed = [":server"],
shard_count = 16,
deps = [
Expand Down Expand Up @@ -315,6 +321,7 @@ go_test(
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_gogo_protobuf//jsonpb",
Expand Down
176 changes: 176 additions & 0 deletions pkg/server/api.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,176 @@
// Copyright 2021 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 (
"context"
"encoding/json"
"fmt"
"net/http"

"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/gorilla/mux"
)

const (
apiV2Path = "/api/v2/"
apiV2AuthHeader = "X-Cockroach-API-Session"
)

func writeJSONResponse(ctx context.Context, w http.ResponseWriter, code int, payload interface{}) {
w.Header().Set("Content-Type", "application/json")
w.WriteHeader(code)

res, err := json.Marshal(payload)
if err != nil {
apiV2InternalError(ctx, err, w)
}
_, _ = w.Write(res)
}

// apiV2Server implements version 2 API endpoints, under apiV2Path. The
// implementation of some endpoints is delegated to sub-servers (eg. auth
// endpoints like `/login` and `/logout` are passed onto authServer), while
// others are implemented directly by apiV2Server.
//
// To register a new API endpoint, add it to the route definitions in
// registerRoutes().
type apiV2Server struct {
admin *adminServer
authServer *authenticationV2Server
status *statusServer
mux *mux.Router
}

// newAPIV2Server returns a new apiV2Server.
func newAPIV2Server(ctx context.Context, s *Server) *apiV2Server {
authServer := newAuthenticationV2Server(ctx, s, apiV2Path)
innerMux := mux.NewRouter()

authMux := newAuthenticationV2Mux(authServer, innerMux)
outerMux := mux.NewRouter()
a := &apiV2Server{
admin: s.admin,
authServer: authServer,
status: s.status,
mux: outerMux,
}
a.registerRoutes(innerMux, authMux)
return a
}

// registerRoutes registers endpoints under the current API server.
func (a *apiV2Server) registerRoutes(innerMux *mux.Router, authMux http.Handler) {
var noOption roleoption.Option

// Add any new API endpoint definitions here, even if a sub-server handles
// them. Arguments:
//
// - `url` is the path string that, if matched by the user request, is
// routed to this endpoint. Pattern-matching handled by gorilla.Mux; see
// https://github.com/gorilla/mux#matching-routes for supported patterns.
// - `handler` is the http.HandlerFunc to be called if this endpoint url
// matches.
// - `requiresAuth` is a bool that denotes whether this endpoint requires
// authentication. If the user isn't authenticated, an HTTP 401 error is
// returned. If the user is authenticated, the http.Request's context
// contains the current user's username.
// - `role` and `option` are used to determine if the current user is
// authorized to access this endpoint. If the user is not at least of type
// `role`, or does not have the roleoption `option`, an HTTP 403 forbidden
// error is returned.
routeDefinitions := []struct {
url string
handler http.HandlerFunc
requiresAuth bool
role apiRole
option roleoption.Option
}{
// Pass through auth-related endpoints to the auth server.
{"login/", a.authServer.ServeHTTP, false /* requiresAuth */, regularRole, noOption},
{"logout/", a.authServer.ServeHTTP, false /* requiresAuth */, regularRole, noOption},

// Directly register other endpoints in the api server.
{"sessions/", a.listSessions, true /* requiresAuth */, adminRole, noOption},
}

// For all routes requiring authentication, have the outer mux (a.mux)
// send requests through to the authMux, and also register the relevant route
// in innerMux. Routes not requiring login can directly be handled in a.mux.
for _, route := range routeDefinitions {
var handler http.Handler
handler = &callCountDecorator{
counter: telemetry.GetCounter(fmt.Sprintf("api.v2.%s", route.url)),
inner: http.Handler(route.handler),
}
if route.requiresAuth {
a.mux.Handle(apiV2Path+route.url, authMux)
if route.role != regularRole {
handler = &roleAuthorizationMux{
ie: a.admin.ie,
role: route.role,
option: route.option,
inner: handler,
}
}
innerMux.Handle(apiV2Path+route.url, handler)
} else {
a.mux.Handle(apiV2Path+route.url, handler)
}
}
}

// ServeHTTP implements the http.Handler interface.
func (a *apiV2Server) ServeHTTP(w http.ResponseWriter, req *http.Request) {
a.mux.ServeHTTP(w, req)
}

type callCountDecorator struct {
counter telemetry.Counter
inner http.Handler
}

func (c *callCountDecorator) ServeHTTP(w http.ResponseWriter, req *http.Request) {
telemetry.Inc(c.counter)
c.inner.ServeHTTP(w, req)
}

type listSessionsResponse struct {
serverpb.ListSessionsResponse

Next string `json:"next"`
}

func (a *apiV2Server) listSessions(w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
limit, start := getRPCPaginationValues(r)
reqUsername := r.URL.Query().Get("username")
req := &serverpb.ListSessionsRequest{Username: reqUsername}
response := &listSessionsResponse{}
outgoingCtx := apiToOutgoingGatewayCtx(ctx, r)

responseProto, pagState, err := a.status.listSessionsHelper(outgoingCtx, req, limit, start)
if err != nil {
apiV2InternalError(ctx, err, w)
return
}
var nextBytes []byte
if nextBytes, err = pagState.MarshalText(); err != nil {
err := serverpb.ListSessionsError{Message: err.Error()}
response.Errors = append(response.Errors, err)
} else {
response.Next = string(nextBytes)
}
response.ListSessionsResponse = *responseProto
writeJSONResponse(ctx, w, http.StatusOK, response)
}
Loading

0 comments on commit a889429

Please sign in to comment.