From 83dfb1b75bb1f9de646963a5ea7218cedaccf7cf Mon Sep 17 00:00:00 2001 From: marc Date: Thu, 20 Aug 2015 15:04:43 -0400 Subject: [PATCH] Restrict KV endpoint to root and node users. Work towards #2089 * restrict all RPC endpoints to root and node users only. * restrict http kv endpoint to root and node users only. * sql http endpoint allows all users Nothing makes use of the rpc.RegisterPublic method yet, but sql will. --- acceptance/multiuser_test.go | 109 ---------------------------------- client/client_test.go | 10 ++-- kv/db.go | 2 +- multiraft/transport.go | 2 +- rpc/codec/rpc_test.go | 4 +- rpc/codec/server.go | 14 +---- rpc/send_test.go | 2 +- rpc/server.go | 40 +++++++++---- rpc/server_test.go | 4 +- security/auth.go | 10 +++- security/auth_test.go | 49 +++++++++------ server/authentication_test.go | 108 +++++++++++++++++++++++---------- server/raft_transport.go | 4 +- sql/server.go | 2 +- 14 files changed, 165 insertions(+), 195 deletions(-) delete mode 100644 acceptance/multiuser_test.go diff --git a/acceptance/multiuser_test.go b/acceptance/multiuser_test.go deleted file mode 100644 index f9e900e614e3..000000000000 --- a/acceptance/multiuser_test.go +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2015 The Cockroach Authors. -// -// Licensed 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. See the AUTHORS file -// for names of contributors. -// -// Author: Marc Berhault (marc@cockroachlabs.com) - -// +build acceptance - -package acceptance - -import ( - "testing" - "time" - - "github.com/cockroachdb/cockroach/acceptance/localcluster" - "github.com/cockroachdb/cockroach/client" - "github.com/cockroachdb/cockroach/config" - "github.com/cockroachdb/cockroach/security" -) - -// TestMultiuser starts up an N node cluster and performs various ops -// using different users. -func TestMultiuser(t *testing.T) { - l := localcluster.Create(*numNodes, stopper) - l.Start() - defer l.Stop() - - // Create client certificates for "foo" and "other". - if err := security.RunCreateClientCert(l.CertsDir, 512, "foo"); err != nil { - t.Fatal(err) - } - if err := security.RunCreateClientCert(l.CertsDir, 512, "other"); err != nil { - t.Fatal(err) - } - - checkRangeReplication(t, l, 20*time.Second) - - // Make clients. - rootClient := makeDBClientForUser(t, l, "root", 0) - fooClient := makeDBClientForUser(t, l, "foo", 0) - otherClient := makeDBClientForUser(t, l, "other", 0) - - // Set permissions configs. - configs := []struct { - prefix string - readers []string - writers []string - }{ - // Good to know: "root" is always allowed to read and write. - {"foo", []string{"foo"}, []string{"foo"}}, - {"foo/public", []string{"foo", "other"}, []string{"foo"}}, - {"tmp", []string{"foo", "other"}, []string{"foo", "other"}}, - } - for i, cfg := range configs { - protoConfig := &config.PermConfig{Read: cfg.readers, Write: cfg.writers} - if err := putPermConfig(rootClient, cfg.prefix, protoConfig); err != nil { - t.Fatalf("#%d: failed to write config %+v for prefix %q: %v", i, protoConfig, cfg.prefix, err) - } - } - - // Write some data. The value is just the key. - writes := []struct { - key string - db *client.DB - success bool - }{ - {"some-file", rootClient, true}, {"some-file", fooClient, false}, {"some-file", otherClient, false}, - {"foo/a", rootClient, true}, {"foo/a", fooClient, true}, {"foo/a", otherClient, false}, - {"foo/public/b", rootClient, true}, {"foo/public/b", fooClient, true}, {"foo/public/b", otherClient, false}, - {"tmp/c", rootClient, true}, {"tmp/c", fooClient, true}, {"tmp/c", otherClient, true}, - } - - for i, w := range writes { - err := w.db.Put(w.key, w.key) - if (err == nil) != w.success { - t.Errorf("test case #%d: %+v, got err=%v", i, w, err) - } - } - - // Read the previously-written files. They all succeeded at least once. - reads := []struct { - key string - db *client.DB - success bool - }{ - {"some-file", rootClient, true}, {"some-file", fooClient, false}, {"some-file", otherClient, false}, - {"foo/a", rootClient, true}, {"foo/a", fooClient, true}, {"foo/a", otherClient, false}, - {"foo/public/b", rootClient, true}, {"foo/public/b", fooClient, true}, {"foo/public/b", otherClient, true}, - {"tmp/c", rootClient, true}, {"tmp/c", fooClient, true}, {"tmp/c", otherClient, true}, - } - - for i, r := range reads { - _, err := r.db.Get(r.key) - if (err == nil) != r.success { - t.Errorf("test case #%d: %+v, got err=%v", i, r, err) - } - } -} diff --git a/client/client_test.go b/client/client_test.go index f2dd67e989e9..ca77f39518c2 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -76,7 +76,7 @@ func (ss *notifyingSender) Send(ctx context.Context, call proto.Call) { } func createTestClient(addr string) *client.DB { - return createTestClientFor(addr, testUser) + return createTestClientFor(addr, security.RootUser) } func createTestClientFor(addr, user string) *client.DB { @@ -554,6 +554,8 @@ func TestConcurrentIncrements(t *testing.T) { } // TestClientPermissions verifies permission enforcement. +// Only root and node users are now allowed to issue kv commands. +// We still enforce the permissions config through. // This relies on: // - r/w permissions config for 'testUser' on the 'testUser' prefix. // - permissive checks for 'root' on all paths @@ -576,13 +578,13 @@ func TestClientPermissions(t *testing.T) { {"foo", test, false}, {"foo", root, true}, - {testUser + "/foo", test, true}, + {testUser + "/foo", test, false}, {testUser + "/foo", root, true}, - {testUser + "foo", test, true}, + {testUser + "foo", test, false}, {testUser + "foo", root, true}, - {testUser, test, true}, + {testUser, test, false}, {testUser, root, true}, {"unknown/foo", test, false}, diff --git a/kv/db.go b/kv/db.go index 0a766dc86234..c817b4cc3aa2 100644 --- a/kv/db.go +++ b/kv/db.go @@ -171,7 +171,7 @@ func (s *DBServer) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // Check request user against client certificate user. - if err := authenticationHook(args); err != nil { + if err := authenticationHook(args, false /*not public*/); err != nil { http.Error(w, err.Error(), http.StatusUnauthorized) return } diff --git a/multiraft/transport.go b/multiraft/transport.go index d99e76c3b382..d151d006706e 100644 --- a/multiraft/transport.go +++ b/multiraft/transport.go @@ -111,7 +111,7 @@ func (lt *localRPCTransport) Listen(id proto.RaftNodeID, server ServerInterface) Stopper: lt.stopper, DisableCache: true, }) - err := rpcServer.RegisterAsync(raftMessageName, + err := rpcServer.RegisterAsync(raftMessageName, false, /*not public*/ func(argsI gogoproto.Message, callback func(gogoproto.Message, error)) { protoArgs := argsI.(*proto.RaftMessageRequest) args := RaftMessageRequest{ diff --git a/rpc/codec/rpc_test.go b/rpc/codec/rpc_test.go index 722a1dc8c2b4..f8245ab4e9f9 100644 --- a/rpc/codec/rpc_test.go +++ b/rpc/codec/rpc_test.go @@ -112,7 +112,7 @@ func listenAndServeArithAndEchoService(network, addr string) (net.Addr, error) { log.Infof("clients.Accept(): %v\n", err) continue } - go srv.ServeCodec(NewServerCodec(conn, nil /* no request hook */)) + go srv.ServeCodec(NewServerCodec(conn)) } }() return clients.Addr(), nil @@ -379,7 +379,7 @@ func benchmarkEchoProtoRPC(b *testing.B, size int) { if *startEchoServer { l, err := listenAndServeEchoService("tcp", *echoAddr, func(srv *rpc.Server, conn io.ReadWriteCloser) { - go srv.ServeCodec(NewServerCodec(conn, nil /* no request hook */)) + go srv.ServeCodec(NewServerCodec(conn)) }) if err != nil { b.Fatal("could not start server") diff --git a/rpc/codec/server.go b/rpc/codec/server.go index e3731404e1d4..05ac2617cb59 100644 --- a/rpc/codec/server.go +++ b/rpc/codec/server.go @@ -37,9 +37,6 @@ type serverCodec struct { methods []string - // Post body-decoding hook. May be nil in tests. - requestBodyHook func(proto.Message) error - // temporary work space respBodyBuf bytes.Buffer respHeaderBuf bytes.Buffer @@ -49,14 +46,13 @@ type serverCodec struct { // NewServerCodec returns a serverCodec that communicates with the ClientCodec // on the other end of the given conn. -func NewServerCodec(conn io.ReadWriteCloser, requestBodyHook func(proto.Message) error) rpc.ServerCodec { +func NewServerCodec(conn io.ReadWriteCloser) rpc.ServerCodec { return &serverCodec{ baseConn: baseConn{ r: bufio.NewReader(conn), w: bufio.NewWriter(conn), c: conn, }, - requestBodyHook: requestBodyHook, } } @@ -101,13 +97,7 @@ func (c *serverCodec) ReadRequestBody(x interface{}) error { return err } c.reqHeader.Reset() - - if c.requestBodyHook == nil || x == nil { - // Only call requestBodyHook if we are actually decoding a frame - // instead of discarding it. - return nil - } - return c.requestBodyHook(request) + return nil } func (c *serverCodec) WriteResponse(r *rpc.Response, x interface{}) error { diff --git a/rpc/send_test.go b/rpc/send_test.go index c099cc583254..1dccc13d00a8 100644 --- a/rpc/send_test.go +++ b/rpc/send_test.go @@ -211,7 +211,7 @@ func TestClientNotReady(t *testing.T) { addr: addr, methods: map[string]method{}, } - if err := s.Register("Heartbeat.Ping", (&Heartbeat{}).Ping, &proto.PingRequest{}); err != nil { + if err := s.RegisterPublic("Heartbeat.Ping", (&Heartbeat{}).Ping, &proto.PingRequest{}); err != nil { t.Fatal(err) } if err := s.Start(); err != nil { diff --git a/rpc/server.go b/rpc/server.go index b61d267c5327..d824ebfdd00a 100644 --- a/rpc/server.go +++ b/rpc/server.go @@ -37,6 +37,7 @@ import ( type method struct { handler func(proto.Message, func(proto.Message, error)) reqType reflect.Type + public bool } type serverResponse struct { @@ -94,9 +95,18 @@ func NewServer(addr net.Addr, context *Context) *Server { // argument of the same type as `reqPrototype`. Both the argument and // return value of 'handler' should be a pointer to a protocol message // type. The handler function will be executed in a new goroutine. -func (s *Server) Register(name string, handler func(proto.Message) (proto.Message, error), +// Only system users (root and node) are allowed to use these endpoints. +func (s *Server) Register(name string, + handler func(proto.Message) (proto.Message, error), reqPrototype proto.Message) error { - return s.RegisterAsync(name, syncAdapter(handler).exec, reqPrototype) + return s.RegisterAsync(name, false /*not public*/, syncAdapter(handler).exec, reqPrototype) +} + +// RegisterPublic is similar to Register, but allows non-system users. +func (s *Server) RegisterPublic(name string, + handler func(proto.Message) (proto.Message, error), + reqPrototype proto.Message) error { + return s.RegisterAsync(name, true /*public*/, syncAdapter(handler).exec, reqPrototype) } // RegisterAsync registers an asynchronous method handler. Instead of @@ -107,7 +117,10 @@ func (s *Server) Register(name string, handler func(proto.Message) (proto.Messag // channel promptly). However, the fact that they are started in the // RPC server's goroutine guarantees that the order of requests as // they were read from the connection is preserved. -func (s *Server) RegisterAsync(name string, handler func(proto.Message, func(proto.Message, error)), +// If 'public' is true, all users may call this method, otherwise +// system users only (root and node). +func (s *Server) RegisterAsync(name string, public bool, + handler func(proto.Message, func(proto.Message, error)), reqPrototype proto.Message) error { s.mu.Lock() defer s.mu.Unlock() @@ -124,6 +137,7 @@ func (s *Server) RegisterAsync(name string, handler func(proto.Message, func(pro s.methods[name] = method{ handler: handler, reqType: reqType, + public: public, } return nil } @@ -171,7 +185,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { security.LogTLSState("RPC", r.TLS) io.WriteString(conn, "HTTP/1.0 "+connected+"\n\n") - codec := codec.NewServerCodec(conn, authHook) + codec := codec.NewServerCodec(conn) responses := make(chan serverResponse) var wg sync.WaitGroup wg.Add(1) @@ -179,7 +193,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { s.sendResponses(codec, responses) wg.Done() }() - s.readRequests(codec, responses) + s.readRequests(codec, authHook, responses) wg.Wait() codec.Close() @@ -334,7 +348,7 @@ func (s *Server) Close() { // when the handler finishes the response is written to the responses // channel. When the connection is closed (and any pending requests // have finished), we close the responses channel. -func (s *Server) readRequests(codec rpc.ServerCodec, responses chan<- serverResponse) { +func (s *Server) readRequests(codec rpc.ServerCodec, authHook func(proto.Message, bool) error, responses chan<- serverResponse) { var wg sync.WaitGroup defer func() { wg.Wait() @@ -342,7 +356,7 @@ func (s *Server) readRequests(codec rpc.ServerCodec, responses chan<- serverResp }() for { - req, meth, args, err := s.readRequest(codec) + req, meth, args, err := s.readRequest(codec, authHook) if err != nil { if err == io.EOF || err == io.ErrUnexpectedEOF || isClosedConnection(err) { return @@ -372,8 +386,8 @@ func (s *Server) readRequests(codec rpc.ServerCodec, responses chan<- serverResp } // readRequest reads a single request from a connection. -func (s *Server) readRequest(codec rpc.ServerCodec) (req rpc.Request, m method, - args proto.Message, err error) { +func (s *Server) readRequest(codec rpc.ServerCodec, authHook func(proto.Message, bool) error) ( + req rpc.Request, m method, args proto.Message, err error) { if err = codec.ReadRequestHeader(&req); err != nil { return } @@ -388,7 +402,13 @@ func (s *Server) readRequest(codec rpc.ServerCodec) (req rpc.Request, m method, if ok { args = reflect.New(m.reqType.Elem()).Interface().(proto.Message) } - err = codec.ReadRequestBody(args) + if err = codec.ReadRequestBody(args); err != nil { + return + } + if args == nil { + return + } + err = authHook(args, m.public) return } diff --git a/rpc/server_test.go b/rpc/server_test.go index 0e45d85806fb..d9021869f294 100644 --- a/rpc/server_test.go +++ b/rpc/server_test.go @@ -73,10 +73,10 @@ func TestDuplicateRegistration(t *testing.T) { s := NewServer(util.CreateTestAddr("tcp"), NewNodeTestContext(nil, stopper)) heartbeat := &Heartbeat{} - if err := s.Register("Foo.Bar", heartbeat.Ping, &proto.PingRequest{}); err != nil { + if err := s.RegisterPublic("Foo.Bar", heartbeat.Ping, &proto.PingRequest{}); err != nil { t.Fatalf("unexpected failure on first registration: %s", err) } - if err := s.Register("Foo.Bar", heartbeat.Ping, &proto.PingRequest{}); err == nil { + if err := s.RegisterPublic("Foo.Bar", heartbeat.Ping, &proto.PingRequest{}); err == nil { t.Fatalf("unexpected success on second registration") } } diff --git a/security/auth.go b/security/auth.go index 17df68160c9c..528e58cc1d53 100644 --- a/security/auth.go +++ b/security/auth.go @@ -85,10 +85,10 @@ func GetCertificateUser(tlsState *tls.ConnectionState) (string, error) { // AuthenticationHook builds an authentication hook based on the // security mode and client certificate. // Must be called at connection time and passed the TLS state. -// Returns a func(proto.Message) error. The passed-in proto must implement +// Returns a func(proto.Message,bool) error. The passed-in proto must implement // the GetUser interface. func AuthenticationHook(insecureMode bool, tlsState *tls.ConnectionState) ( - func(request proto.Message) error, error) { + func(request proto.Message, public bool) error, error) { var certUser string var err error @@ -99,7 +99,7 @@ func AuthenticationHook(insecureMode bool, tlsState *tls.ConnectionState) ( } } - return func(request proto.Message) error { + return func(request proto.Message, public bool) error { // userRequest is an interface for RPC requests that have a "requested user". type userRequest interface { // GetUser returns the user from the request. @@ -119,6 +119,10 @@ func AuthenticationHook(insecureMode bool, tlsState *tls.ConnectionState) ( return util.Errorf("missing User in request: %+v", request) } + if !public && requestedUser != RootUser && requestedUser != NodeUser { + return util.Errorf("user %s is not allowed", requestedUser) + } + // If running in insecure mode, we have nothing to verify it against. if insecureMode { return nil diff --git a/security/auth_test.go b/security/auth_test.go index 1f115f1cb281..95a1d386b4fd 100644 --- a/security/auth_test.go +++ b/security/auth_test.go @@ -99,30 +99,41 @@ func TestAuthenticationHook(t *testing.T) { badRequest := &cockroach_proto.GetResponse{} testCases := []struct { - insecure bool - tls *tls.ConnectionState - request proto.Message - buildHookSuccess bool - runHookSuccess bool + insecure bool + tls *tls.ConnectionState + request proto.Message + buildHookSuccess bool + publicHookSuccess bool + privateHookSuccess bool }{ // Insecure mode, nil request. - {true, nil, nil, true, false}, + {true, nil, nil, true, false, false}, // Insecure mode, bad request. - {true, nil, badRequest, true, false}, + {true, nil, badRequest, true, false, false}, // Insecure mode, userRequest with empty user. - {true, nil, makeUserRequest(""), true, false}, + {true, nil, makeUserRequest(""), true, false, false}, // Insecure mode, userRequest with good user. - {true, nil, makeUserRequest("foo"), true, true}, + {true, nil, makeUserRequest("foo"), true, true, false}, + // Insecure mode, userRequest with root user. + {true, nil, makeUserRequest(security.RootUser), true, true, true}, + // Insecure mode, userRequest with node user. + {true, nil, makeUserRequest(security.NodeUser), true, true, true}, // Secure mode, no TLS state. - {false, nil, nil, false, false}, + {false, nil, nil, false, false, false}, // Secure mode, user mismatch. - {false, makeFakeTLSState([]string{"foo"}, []int{1}), makeUserRequest("bar"), true, false}, + {false, makeFakeTLSState([]string{"foo"}, []int{1}), makeUserRequest("bar"), true, false, false}, // Secure mode, user mismatch, but client certificate is for the node user. - {false, makeFakeTLSState([]string{security.NodeUser}, []int{1}), makeUserRequest("bar"), true, true}, + {false, makeFakeTLSState([]string{security.NodeUser}, []int{1}), makeUserRequest("bar"), true, true, false}, // Secure mode, user mismatch, and the root user does not get blind permissions. - {false, makeFakeTLSState([]string{security.RootUser}, []int{1}), makeUserRequest("bar"), true, false}, + {false, makeFakeTLSState([]string{security.RootUser}, []int{1}), makeUserRequest("bar"), true, false, false}, // Secure mode, matching users. - {false, makeFakeTLSState([]string{"foo"}, []int{1}), makeUserRequest("foo"), true, true}, + {false, makeFakeTLSState([]string{"foo"}, []int{1}), makeUserRequest("foo"), true, true, false}, + // Secure mode, root acting as itself. + {false, makeFakeTLSState([]string{security.RootUser}, []int{1}), makeUserRequest(security.RootUser), true, true, true}, + // Secure mode, node acting as itself. + {false, makeFakeTLSState([]string{security.NodeUser}, []int{1}), makeUserRequest(security.NodeUser), true, true, true}, + // Secure mode, node acting as root. + {false, makeFakeTLSState([]string{security.NodeUser}, []int{1}), makeUserRequest(security.RootUser), true, true, true}, } for tcNum, tc := range testCases { @@ -133,9 +144,13 @@ func TestAuthenticationHook(t *testing.T) { if err != nil { continue } - err = hook(tc.request) - if (err == nil) != tc.runHookSuccess { - t.Fatalf("#%d: expected success=%t, got err=%v", tcNum, tc.runHookSuccess, err) + err = hook(tc.request, true /*public*/) + if (err == nil) != tc.publicHookSuccess { + t.Fatalf("#%d: expected success=%t, got err=%v", tcNum, tc.publicHookSuccess, err) + } + err = hook(tc.request, false /*not public*/) + if (err == nil) != tc.privateHookSuccess { + t.Fatalf("#%d: expected success=%t, got err=%v", tcNum, tc.privateHookSuccess, err) } } } diff --git a/server/authentication_test.go b/server/authentication_test.go index a37899080c32..1a91ac6a10a1 100644 --- a/server/authentication_test.go +++ b/server/authentication_test.go @@ -18,35 +18,67 @@ package server import ( + "bytes" "fmt" + "io" "net/http" "testing" "github.com/cockroachdb/cockroach/base" "github.com/cockroachdb/cockroach/kv" + "github.com/cockroachdb/cockroach/proto" "github.com/cockroachdb/cockroach/sql/driver" "github.com/cockroachdb/cockroach/testutils" "github.com/cockroachdb/cockroach/ts" + "github.com/cockroachdb/cockroach/util" "github.com/cockroachdb/cockroach/util/leaktest" + gogoproto "github.com/gogo/protobuf/proto" ) -func doHTTPReq(t *testing.T, client *http.Client, method, url string) (*http.Response, error) { - req, err := http.NewRequest(method, url, nil) +func doHTTPReq(t *testing.T, client *http.Client, method, url string, body gogoproto.Message) (*http.Response, error) { + var b io.Reader + if body != nil { + buf, err := gogoproto.Marshal(body) + if err != nil { + t.Fatal(err) + } + b = bytes.NewReader(buf) + } + req, err := http.NewRequest(method, url, b) if err != nil { t.Fatalf("%s %s: error building request: %s", method, url, err) } + if b != nil { + req.Header.Add(util.ContentTypeHeader, util.ProtoContentType) + } return client.Do(req) } -// Verify client certificate enforcement. +func kvGetForUser(context *base.Context) gogoproto.Message { + ret := &proto.GetRequest{} + ret.User = context.User + ret.Key = proto.Key("/") + return ret +} + +func sqlForUser(context *base.Context) gogoproto.Message { + ret := &driver.Request{} + ret.User = context.User + return ret +} + +// Verify client certificate enforcement and user whitelisting. func TestSSLEnforcement(t *testing.T) { defer leaktest.AfterTest(t) s := StartTestServer(t) defer s.Stop() - // HTTPS with client certs. + // HTTPS with client certs for "root". certsContext := testutils.NewRootTestBaseContext() + // HTTPS with client certs for testuser. + testCertsContext := testutils.NewRootTestBaseContext() + testCertsContext.User = TestUser // HTTPS without client certs. noCertsContext := testutils.NewRootTestBaseContext() noCertsContext.Certs = "" @@ -56,44 +88,59 @@ func TestSSLEnforcement(t *testing.T) { testCases := []struct { method, key string + body gogoproto.Message ctx *base.Context success bool // request sent successfully (may be non-200) code int // http response code }{ // /ui/: basic file server: no auth. - {"GET", "/index.html", certsContext, true, http.StatusOK}, - {"GET", "/index.html", noCertsContext, true, http.StatusOK}, - {"GET", "/index.html", insecureContext, false, -1}, + {"GET", "/index.html", nil, certsContext, true, http.StatusOK}, + {"GET", "/index.html", nil, testCertsContext, true, http.StatusOK}, + {"GET", "/index.html", nil, noCertsContext, true, http.StatusOK}, + {"GET", "/index.html", nil, insecureContext, false, -1}, // /_admin/: server.adminServer: no auth. - {"GET", healthPath, certsContext, true, http.StatusOK}, - {"GET", healthPath, noCertsContext, true, http.StatusOK}, - {"GET", healthPath, insecureContext, false, -1}, + {"GET", healthPath, nil, certsContext, true, http.StatusOK}, + {"GET", healthPath, nil, testCertsContext, true, http.StatusOK}, + {"GET", healthPath, nil, noCertsContext, true, http.StatusOK}, + {"GET", healthPath, nil, insecureContext, false, -1}, // /debug/: server.adminServer: no auth. - {"GET", debugEndpoint + "vars", certsContext, true, http.StatusOK}, - {"GET", debugEndpoint + "vars", noCertsContext, true, http.StatusOK}, - {"GET", debugEndpoint + "vars", insecureContext, false, -1}, + {"GET", debugEndpoint + "vars", nil, certsContext, true, http.StatusOK}, + {"GET", debugEndpoint + "vars", nil, testCertsContext, true, http.StatusOK}, + {"GET", debugEndpoint + "vars", nil, noCertsContext, true, http.StatusOK}, + {"GET", debugEndpoint + "vars", nil, insecureContext, false, -1}, // /_status/nodes: server.statusServer: no auth. - {"GET", statusNodesPrefix, certsContext, true, http.StatusOK}, - {"GET", statusNodesPrefix, noCertsContext, true, http.StatusOK}, - {"GET", statusNodesPrefix, insecureContext, false, -1}, + {"GET", statusNodesPrefix, nil, certsContext, true, http.StatusOK}, + {"GET", statusNodesPrefix, nil, testCertsContext, true, http.StatusOK}, + {"GET", statusNodesPrefix, nil, noCertsContext, true, http.StatusOK}, + {"GET", statusNodesPrefix, nil, insecureContext, false, -1}, // /ts/: ts.Server: no auth. - {"GET", ts.URLPrefix, certsContext, true, http.StatusNotFound}, - {"GET", ts.URLPrefix, noCertsContext, true, http.StatusNotFound}, - {"GET", ts.URLPrefix, insecureContext, false, -1}, - - // /kv/db/: kv.DBServer. These are proto reqs, but we can at least get past auth. - {"GET", kv.DBPrefix + "Get", certsContext, true, http.StatusBadRequest}, - {"GET", kv.DBPrefix + "Get", noCertsContext, true, http.StatusUnauthorized}, - {"GET", kv.DBPrefix + "Get", insecureContext, false, -1}, - - // /sql/: sql.Server. These are proto reqs, but we can at least get past auth. - {"GET", driver.Endpoint + "Get", certsContext, true, http.StatusNotFound}, - {"GET", driver.Endpoint + "Get", noCertsContext, true, http.StatusUnauthorized}, - {"GET", driver.Endpoint + "Get", insecureContext, false, -1}, + {"GET", ts.URLPrefix, nil, certsContext, true, http.StatusNotFound}, + {"GET", ts.URLPrefix, nil, testCertsContext, true, http.StatusNotFound}, + {"GET", ts.URLPrefix, nil, noCertsContext, true, http.StatusNotFound}, + {"GET", ts.URLPrefix, nil, insecureContext, false, -1}, + + // /kv/db/: kv.DBServer. These are proto reqs. The important field is header.User. + {"POST", kv.DBPrefix + "Get", kvGetForUser(certsContext), certsContext, + true, http.StatusOK}, + {"POST", kv.DBPrefix + "Get", kvGetForUser(testCertsContext), testCertsContext, + true, http.StatusUnauthorized}, + {"POST", kv.DBPrefix + "Get", kvGetForUser(noCertsContext), noCertsContext, + true, http.StatusUnauthorized}, + {"POST", kv.DBPrefix + "Get", kvGetForUser(insecureContext), insecureContext, false, -1}, + + // /sql/: sql.Server. These are proto reqs. The important field is header.User. + {"POST", driver.Endpoint + driver.Execute.String(), sqlForUser(certsContext), + certsContext, true, http.StatusOK}, + {"POST", driver.Endpoint + driver.Execute.String(), sqlForUser(testCertsContext), + testCertsContext, true, http.StatusOK}, + {"POST", driver.Endpoint + driver.Execute.String(), sqlForUser(noCertsContext), + noCertsContext, true, http.StatusUnauthorized}, + {"POST", driver.Endpoint + driver.Execute.String(), sqlForUser(insecureContext), + insecureContext, false, -1}, } for tcNum, tc := range testCases { @@ -102,7 +149,8 @@ func TestSSLEnforcement(t *testing.T) { t.Fatalf("[%d]: failed to get http client: %v", tcNum, err) } resp, err := doHTTPReq(t, client, tc.method, - fmt.Sprintf("%s://%s%s", tc.ctx.RequestScheme(), s.ServingAddr(), tc.key)) + fmt.Sprintf("%s://%s%s", tc.ctx.RequestScheme(), s.ServingAddr(), tc.key), + tc.body) if (err == nil) != tc.success { t.Fatalf("[%d]: expected success=%t, got err=%v", tcNum, tc.success, err) } diff --git a/server/raft_transport.go b/server/raft_transport.go index 3527211900fd..425cb3d4a17d 100644 --- a/server/raft_transport.go +++ b/server/raft_transport.go @@ -66,8 +66,8 @@ func newRPCTransport(gossip *gossip.Gossip, rpcServer *rpc.Server, rpcContext *r } if t.rpcServer != nil { - if err := t.rpcServer.RegisterAsync(raftMessageName, t.RaftMessage, - &proto.RaftMessageRequest{}); err != nil { + if err := t.rpcServer.RegisterAsync(raftMessageName, false, /*not public*/ + t.RaftMessage, &proto.RaftMessageRequest{}); err != nil { return nil, err } } diff --git a/sql/server.go b/sql/server.go index 9636992d05df..598545f2fc24 100644 --- a/sql/server.go +++ b/sql/server.go @@ -101,7 +101,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // Check request user against client certificate user. - if err := authenticationHook(&args); err != nil { + if err := authenticationHook(&args, true /*public*/); err != nil { http.Error(w, err.Error(), http.StatusUnauthorized) return }