From 3fe5d7b2c04c9e0b4dc77090ebd0057703549cde Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Wed, 5 Jun 2024 23:57:11 -0500 Subject: [PATCH 1/8] support optional TLS for etcd, brokers, and consumers The approach closely parallels the Etcd certificate authorization model and can support mutual (client and server) certificate verification. In server contexts, for $foo (of broker, consumer): --foo.server-cert-file / FOO_SERVER_CERT_FILE is a path to a cerificate for the server to present to clients. This option toggles whether TLS is used. If absent, all other TLS settings are ignored. --foo.server-cert-key-file / FOO_SERVER_CERT_KEY_FILE is the corresponding private key. --foo.server-ca-file / FOO_SERVER_CA_FILE is a trusted certificate authority for verification of client certificates. If this option is omitted, client certificates are verified if presented but are not required. If this option is set, client certificates are required and verified. --foo.peer-cert-file / FOO_PEER_CERT_FILE is a path to a cerificate for the server to present to peers in a peer-to-peer client context. --foo.peer-cert-key-file / FOO_PEER_CERT_KEY_FILE is the corresponding private key. --foo.peer-ca-file / FOO_PEER_CA_FILE is a trusted certificate authority for verification of peer server certificates. If not specified, the system CA pool is used instead. In client contexts, for $foo (of etcd, broker, consumer): --foo.cert-file / FOO_CERT_FILE is a path to a client cerificate to present to the server context. --foo.cert-key-file / FOO_CERT_KEY_FILE is the corresponding private key. --foo.trusted-ca-file / FOO_TRUSTED_CA_FILE is a trusted certificate authority for verification of server certificates. Gazette's "dispatcher" GRPC load balancer is also updated to be aware of TLS or non-TLS contexts, as well as the appropriate dynamic authority which should be verified against a TLS certificate. `protocol.NewDispatchedCredentials` should be used with the `grpc.WithTransportCredentials` DialOption to allow dynamic load balancing across a mix of TLS (https://) and insecure (http://) hosts. --- broker/e2e_test.go | 7 +- broker/fragment/spool_unix.go | 7 +- broker/fragment/spool_win.go | 7 +- broker/protocol/dispatcher.go | 56 ++++++++- broker/protocol/dispatcher_test.go | 56 ++++----- broker/protocol/endpoint.go | 8 +- brokertest/broker_test.go | 7 +- cmd/gazette/main.go | 16 ++- go.mod | 21 ++-- go.sum | 39 +++--- mainboilerplate/client.go | 23 ++-- mainboilerplate/etcd.go | 34 +++-- mainboilerplate/runconsumer/run_consumer.go | 16 ++- mainboilerplate/service.go | 34 ++--- server/server.go | 133 +++++++++++++++----- 15 files changed, 315 insertions(+), 149 deletions(-) diff --git a/broker/e2e_test.go b/broker/e2e_test.go index f1810fa5..5e37d849 100644 --- a/broker/e2e_test.go +++ b/broker/e2e_test.go @@ -2,6 +2,7 @@ package broker import ( "context" + "crypto/tls" "io" "testing" @@ -334,7 +335,11 @@ func applySpoolContentFixture(r *replica) { // may see "transport is closing" errors due to the loopback ClientConn being closed // before a final EOF response is read. func newDialedClient(t *testing.T, bk *testBroker) (*grpc.ClientConn, pb.JournalClient) { - var conn, err = grpc.Dial(bk.srv.Endpoint().URL().Host, grpc.WithInsecure()) + var tlsConfig = &tls.Config{InsecureSkipVerify: true} // Allow self-signed. + + var conn, err = grpc.Dial(bk.srv.Endpoint().URL().Host, + grpc.WithTransportCredentials(pb.NewDispatchedCredentials(tlsConfig, bk.srv.Endpoint())), + ) require.NoError(t, err) return conn, pb.NewJournalClient(conn) } diff --git a/broker/fragment/spool_unix.go b/broker/fragment/spool_unix.go index 2d9d0349..80445d48 100644 --- a/broker/fragment/spool_unix.go +++ b/broker/fragment/spool_unix.go @@ -1,9 +1,8 @@ -// +build !windows +//go:build !windows package fragment import ( - "io/ioutil" "os" ) @@ -14,8 +13,8 @@ import ( // to the OS after an explicit call to Close, or if the os.File is garbage- // collected (such that the runtime finalizer calls Close on our behalf). var newSpoolFile = func() (File, error) { - if f, err := ioutil.TempFile("", "spool"); err != nil { - return f, err + if f, err := os.CreateTemp("", "spool"); err != nil { + return nil, err } else { return f, os.Remove(f.Name()) } diff --git a/broker/fragment/spool_win.go b/broker/fragment/spool_win.go index 9d1a97f7..ee4d787d 100644 --- a/broker/fragment/spool_win.go +++ b/broker/fragment/spool_win.go @@ -1,9 +1,8 @@ -// +build windows +//go:build windows package fragment import ( - "io/ioutil" "os" "runtime" @@ -15,8 +14,8 @@ import ( // behavior is used because Windows does not support removing the one-and-only // hard link of a currently-open file. var newSpoolFile = func() (File, error) { - if f, err := ioutil.TempFile("", "spool"); err != nil { - return f, err + if f, err := os.CreateTemp("", "spool"); err != nil { + return nil, err } else { runtime.SetFinalizer(f, removeFileFinalizer) return f, nil diff --git a/broker/protocol/dispatcher.go b/broker/protocol/dispatcher.go index 5ba06377..bf35dcd8 100644 --- a/broker/protocol/dispatcher.go +++ b/broker/protocol/dispatcher.go @@ -2,13 +2,18 @@ package protocol import ( "context" + "crypto/tls" + "net" "sync" "time" "golang.org/x/net/trace" + "google.golang.org/grpc/attributes" "google.golang.org/grpc/balancer" "google.golang.org/grpc/codes" "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/resolver" "google.golang.org/grpc/status" ) @@ -58,6 +63,15 @@ func WithDispatchItemRoute(ctx context.Context, dr DispatchRouter, item string, dispatchRoute{route: rt, id: id, item: item, DispatchRouter: dr}) } +// GetDispatchRoute returns a Route and ProcessSpec_ID which haven been previously attached to the Context. +func GetDispatchRoute(ctx context.Context) (Route, ProcessSpec_ID, bool) { + if dr, ok := ctx.Value(dispatchRouteCtxKey{}).(dispatchRoute); ok { + return dr.route, dr.id, true + } else { + return Route{}, ProcessSpec_ID{}, false + } +} + // DispatchRouter routes item to Routes, and observes item Routes. type DispatchRouter interface { // Route an |item| to a Route, which may be empty if the Route is unknown. @@ -191,9 +205,7 @@ func (d *dispatcher) Pick(info balancer.PickInfo) (balancer.PickResult, error) { // Initiate a new SubConn to the ProcessSpec_ID. var err error if msc.subConn, err = d.cc.NewSubConn( - []resolver.Address{{ - Addr: d.idToAddr(dr.route, dispatchID), - }}, + []resolver.Address{d.idToAddr(dr.route, dispatchID)}, balancer.NewSubConnOptions{ StateListener: func(state balancer.SubConnState) { d.updateSubConnState(msc.subConn, state) @@ -288,13 +300,16 @@ func (d *dispatcher) less(lhs, rhs ProcessSpec_ID) bool { } // idToAddr returns a suitable address for the ID. -func (d *dispatcher) idToAddr(rt Route, id ProcessSpec_ID) string { +func (d *dispatcher) idToAddr(rt Route, id ProcessSpec_ID) resolver.Address { if id == (ProcessSpec_ID{}) { - return d.cc.Target() // Use the default service address. + return resolver.Address{Addr: d.cc.Target()} // Use the default service address. } for i := range rt.Members { if rt.Members[i] == id { - return rt.Endpoints[i].GRPCAddr() + return resolver.Address{ + Addr: rt.Endpoints[i].GRPCAddr(), + Attributes: attributes.New("endpoint", rt.Endpoints[i]), + } } } panic("ProcessSpec_ID must be in Route.Members") @@ -389,3 +404,32 @@ type ( ) var dispatchSweepInterval = time.Second * 30 + +// NewDispatchedCredentials returns a TransportCredentials implementation which +// dynamically uses TLS or insecure credentials depending on the scheme of the +// dispatched endpoint. It must be used with the dispatch balancer. +func NewDispatchedCredentials(tlsConfig *tls.Config, serviceEndpoint Endpoint) credentials.TransportCredentials { + return &dispatcherCredentials{ + TransportCredentials: credentials.NewTLS(tlsConfig), + serviceEndpoint: serviceEndpoint, + } +} + +type dispatcherCredentials struct { + credentials.TransportCredentials + serviceEndpoint Endpoint +} + +func (d *dispatcherCredentials) ClientHandshake(ctx context.Context, authority string, conn net.Conn) (net.Conn, credentials.AuthInfo, error) { + var endpoint = d.serviceEndpoint + + if v := credentials.ClientHandshakeInfoFromContext(ctx).Attributes.Value("endpoint"); v != nil { + endpoint = v.(Endpoint) + } + + if u := endpoint.URL(); u.Scheme == "https" { + return d.TransportCredentials.ClientHandshake(ctx, u.Host, conn) + } else { + return insecure.NewCredentials().ClientHandshake(ctx, u.Host, conn) + } +} diff --git a/broker/protocol/dispatcher_test.go b/broker/protocol/dispatcher_test.go index 86cb7ecd..0b6b5fe7 100644 --- a/broker/protocol/dispatcher_test.go +++ b/broker/protocol/dispatcher_test.go @@ -59,16 +59,16 @@ func (s *DispatcherSuite) TestDispatchCases(c *gc.C) { // SubConn to the default service address is started. var _, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "default.addr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "default.addr:80", disp: disp}}) cc.created = nil // Case: Default connection transitions to Ready. Expect it's now returned. - mockSubConn{Name: "default.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "default.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) result, err := disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.IsNil) c.Check(result.Done, gc.IsNil) - c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "default.addr", disp: disp}) + c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "default.addr:80", disp: disp}) // Case: Specific remote peer is dispatched to. ctx = WithDispatchRoute(context.Background(), @@ -76,58 +76,58 @@ func (s *DispatcherSuite) TestDispatchCases(c *gc.C) { result, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "remote.addr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "remote.addr:80", disp: disp}}) cc.created = nil - mockSubConn{Name: "remote.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "remote.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) result, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.IsNil) c.Check(result.Done, gc.IsNil) - c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "remote.addr", disp: disp}) + c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "remote.addr:80", disp: disp}) // Case: Route allows for multiple members. A local one is now dialed. ctx = WithDispatchRoute(context.Background(), buildRouteFixture(), ProcessSpec_ID{}) _, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "local.addr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "local.addr:80", disp: disp}}) cc.created = nil - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) result, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.IsNil) c.Check(result.Done, gc.IsNil) - c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.addr", disp: disp}) + c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.addr:80", disp: disp}) // Case: One local addr is marked as failed. Another is dialed. - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) _, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "local.otherAddr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "local.otherAddr:80", disp: disp}}) cc.created = nil - mockSubConn{Name: "local.otherAddr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "local.otherAddr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) result, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.IsNil) c.Check(result.Done, gc.IsNil) - c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.otherAddr", disp: disp}) + c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.otherAddr:80", disp: disp}) // Case: otherAddr is also failed. Expect that an error is returned, // rather than dispatch to remote addr. (Eg we prefer to wait for a // local replica to recover or the route to change, vs using a remote // endpoint which incurs more networking cost). - mockSubConn{Name: "local.otherAddr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) + mockSubConn{Name: "local.otherAddr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) _, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.Equals, balancer.ErrTransientFailure) // Case: local.addr is Ready again. However, primary is required and has failed. - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) - mockSubConn{Name: "remote.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "remote.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.TransientFailure}) ctx = WithDispatchRoute(context.Background(), buildRouteFixture(), ProcessSpec_ID{Zone: "remote", Suffix: "primary"}) @@ -151,7 +151,7 @@ func (s *DispatcherSuite) TestDispatchCases(c *gc.C) { result, err = disp.Pick(balancer.PickInfo{Ctx: ctx}) c.Check(err, gc.IsNil) c.Check(result.Done, gc.NotNil) - c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.addr", disp: disp}) + c.Check(result.SubConn, gc.Equals, mockSubConn{Name: "local.addr:80", disp: disp}) // Closure callback with an Unavailable error (only) will trigger an invalidation. result.Done(balancer.DoneInfo{Err: nil}) @@ -179,11 +179,11 @@ func (s *DispatcherSuite) TestDispatchMarkAndSweep(c *gc.C) { _, err = disp.Pick(balancer.PickInfo{Ctx: localCtx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "remote.addr", disp: disp}, mockSubConn{Name: "local.addr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "remote.addr:80", disp: disp}, {Name: "local.addr:80", disp: disp}}) cc.created = nil - mockSubConn{Name: "remote.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) + mockSubConn{Name: "remote.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Connecting}) disp.sweep() c.Check(cc.removed, gc.IsNil) @@ -207,14 +207,14 @@ func (s *DispatcherSuite) TestDispatchMarkAndSweep(c *gc.C) { // This time, expect that local.addr is swept. disp.sweep() - c.Check(cc.removed, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "local.addr", disp: disp}}) + c.Check(cc.removed, gc.DeepEquals, []mockSubConn{{Name: "local.addr:80", disp: disp}}) cc.removed = nil - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Shutdown}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Shutdown}) disp.sweep() // Now remote.addr is swept. - c.Check(cc.removed, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "remote.addr", disp: disp}}) + c.Check(cc.removed, gc.DeepEquals, []mockSubConn{{Name: "remote.addr:80", disp: disp}}) cc.removed = nil - mockSubConn{Name: "remote.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Shutdown}) + mockSubConn{Name: "remote.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Shutdown}) // No connections remain. c.Check(disp.idConn, gc.HasLen, 0) @@ -225,10 +225,10 @@ func (s *DispatcherSuite) TestDispatchMarkAndSweep(c *gc.C) { _, err = disp.Pick(balancer.PickInfo{Ctx: localCtx}) c.Check(err, gc.Equals, balancer.ErrNoSubConnAvailable) - c.Check(cc.created, gc.DeepEquals, []mockSubConn{mockSubConn{Name: "local.addr", disp: disp}}) + c.Check(cc.created, gc.DeepEquals, []mockSubConn{{Name: "local.addr:80", disp: disp}}) cc.created = nil - mockSubConn{Name: "local.addr", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + mockSubConn{Name: "local.addr:80", disp: disp}.UpdateState(balancer.SubConnState{ConnectivityState: connectivity.Ready}) _, err = disp.Pick(balancer.PickInfo{Ctx: localCtx}) c.Check(err, gc.IsNil) } @@ -269,7 +269,7 @@ func (c *mockClientConn) NewSubConn(a []resolver.Address, _ balancer.NewSubConnO func (c *mockClientConn) UpdateAddresses(balancer.SubConn, []resolver.Address) { panic("deprecated") } func (c *mockClientConn) UpdateState(balancer.State) {} func (c *mockClientConn) ResolveNow(resolver.ResolveNowOptions) {} -func (c *mockClientConn) Target() string { return "default.addr" } +func (c *mockClientConn) Target() string { return "default.addr:80" } func (c *mockClientConn) RemoveSubConn(sc balancer.SubConn) { sc.Shutdown() } @@ -294,7 +294,7 @@ func buildRouteFixture() Route { {Zone: "local", Suffix: "replica"}, {Zone: "local", Suffix: "other-replica"}, }, - Endpoints: []Endpoint{"http://remote.addr", "http://local.addr", "http://local.otherAddr"}, + Endpoints: []Endpoint{"http://remote.addr:80", "http://local.addr:80", "http://local.otherAddr:80"}, } } diff --git a/broker/protocol/endpoint.go b/broker/protocol/endpoint.go index c14ed541..38e945f6 100644 --- a/broker/protocol/endpoint.go +++ b/broker/protocol/endpoint.go @@ -8,8 +8,8 @@ import ( // scheme defines the network transport and semantics of the host, path, // and query components. At present, supported schemes are: // -// * http://host(:port)/path?query -// +// - http://host(:port)/path?query +// - unix://host/absolute/path/to/unix/domain/socket type Endpoint string // Validate returns an error if the Endpoint is not well-formed. @@ -34,6 +34,10 @@ func (ep Endpoint) GRPCAddr() string { var addr string if u := ep.URL(); u.Scheme == "unix" { addr = "unix://" + u.Path + } else if u.Port() == "" && u.Scheme == "https" { + addr = u.Host + ":443" + } else if u.Port() == "" && u.Scheme == "http" { + addr = u.Host + ":80" } else { addr = u.Host } diff --git a/brokertest/broker_test.go b/brokertest/broker_test.go index 5b5622cf..b12dacb3 100644 --- a/brokertest/broker_test.go +++ b/brokertest/broker_test.go @@ -3,6 +3,7 @@ package brokertest import ( "bufio" "context" + "crypto/tls" "io" "net/http" "testing" @@ -248,7 +249,11 @@ func updateReplication(t require.TestingT, ctx context.Context, bk pb.JournalCli // may see "transport is closing" errors due to the loopback ClientConn being closed // before the final EOF response is read. func newDialedClient(t *testing.T, bk *Broker) (*grpc.ClientConn, pb.RoutedJournalClient) { - var conn, err = grpc.Dial(bk.Endpoint().URL().Host, grpc.WithInsecure()) + var tlsConfig = &tls.Config{InsecureSkipVerify: true} // Allow self-signed. + + var conn, err = grpc.Dial(bk.Endpoint().URL().Host, + grpc.WithTransportCredentials(pb.NewDispatchedCredentials(tlsConfig, bk.Endpoint())), + ) require.NoError(t, err) return conn, pb.NewRoutedJournalClient(pb.NewJournalClient(conn), pb.NoopDispatchRouter{}) } diff --git a/cmd/gazette/main.go b/cmd/gazette/main.go index de330a96..adf49abd 100644 --- a/cmd/gazette/main.go +++ b/cmd/gazette/main.go @@ -2,6 +2,7 @@ package main import ( "context" + "crypto/tls" "os" "os/signal" "syscall" @@ -56,8 +57,21 @@ func (cmdServe) Execute(args []string) error { }).Info("broker configuration") pb.RegisterGRPCDispatcher(Config.Broker.Zone) + var err error + var serverTLS, peerTLS *tls.Config + + if Config.Broker.ServerCertFile != "" { + serverTLS, err = server.BuildTLSConfig( + Config.Broker.ServerCertFile, Config.Broker.ServerCertKeyFile, Config.Broker.ServerCAFile) + mbp.Must(err, "building server TLS config") + + peerTLS, err = server.BuildTLSConfig( + Config.Broker.PeerCertFile, Config.Broker.PeerCertKeyFile, Config.Broker.PeerCAFile) + mbp.Must(err, "building peer TLS config") + } + // Bind our server listener, grabbing a random available port if Port is zero. - var srv, err = server.New("", Config.Broker.Port) + srv, err := server.New("", Config.Broker.Host, Config.Broker.Port, serverTLS, peerTLS) mbp.Must(err, "building Server instance") // If a file:// root was provided, ensure it exists and apply it. diff --git a/go.mod b/go.mod index e6aacd12..6aa1da5c 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module go.gazette.dev/core -go 1.19 +go 1.22 require ( cloud.google.com/go/storage v1.30.1 @@ -14,7 +14,8 @@ require ( github.com/dustin/go-humanize v1.0.0 github.com/dustinkirkland/golang-petname v0.0.0-20191129215211-8e5a1ed0cff0 github.com/gogo/protobuf v1.3.2 - github.com/golang/protobuf v1.5.3 + github.com/golang-jwt/jwt/v5 v5.2.1 + github.com/golang/protobuf v1.5.4 github.com/golang/snappy v0.0.4 github.com/google/uuid v1.6.0 github.com/gorilla/schema v1.4.1 @@ -38,10 +39,11 @@ require ( go.etcd.io/etcd/api/v3 v3.5.0 go.etcd.io/etcd/client/v3 v3.5.0 golang.org/x/net v0.26.0 - golang.org/x/oauth2 v0.11.0 + golang.org/x/oauth2 v0.20.0 golang.org/x/sync v0.7.0 google.golang.org/api v0.126.0 - google.golang.org/grpc v1.59.0 + google.golang.org/grpc v1.65.0 + google.golang.org/protobuf v1.34.1 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c gopkg.in/yaml.v2 v2.4.0 k8s.io/api v0.17.16 @@ -49,19 +51,17 @@ require ( require ( cloud.google.com/go v0.110.7 // indirect - cloud.google.com/go/compute v1.23.0 // indirect - cloud.google.com/go/compute/metadata v0.2.3 // indirect + cloud.google.com/go/compute/metadata v0.3.0 // indirect cloud.google.com/go/iam v1.1.1 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cockroachdb/apd v1.1.0 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96 // indirect - github.com/golang-jwt/jwt/v5 v5.2.1 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/google/gofuzz v1.0.0 // indirect github.com/google/s2a-go v0.1.4 // indirect @@ -100,9 +100,8 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/apimachinery v0.17.16 // indirect diff --git a/go.sum b/go.sum index 3b448b6a..dfd6aa30 100644 --- a/go.sum +++ b/go.sum @@ -21,10 +21,8 @@ cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvf cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= -cloud.google.com/go/compute v1.23.0/go.mod h1:4tCnrn48xsqlwSAiLf1HXMQk8CONslYbdiEZc9FEIbM= -cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= -cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= +cloud.google.com/go/compute/metadata v0.3.0 h1:Tz+eQXMEqDIKRsmY3cHTL6FVaynIjX2QxYC4trgAKZc= +cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y= @@ -98,8 +96,8 @@ github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -206,8 +204,8 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= -github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20160524151835-7d79101e329e/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -223,13 +221,15 @@ github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= +github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -548,8 +548,8 @@ golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4Iltr golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.11.0 h1:vPL4xzxBM4niKCW6g9whtaWVXTJf1U5e4aZxxFx/gbU= -golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk= +golang.org/x/oauth2 v0.20.0 h1:4mQdhULixXKP1rwYBW0vAijoXnkTG0BLCDRzfe1idMo= +golang.org/x/oauth2 v0.20.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -689,6 +689,7 @@ golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -754,10 +755,10 @@ google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d h1:VBu5YqKPv6XiJ199exd8Br+Aetz+o08F+PLMnwJQHAY= google.golang.org/genproto v0.0.0-20230822172742-b8732ec3820d/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4= -google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d h1:DoPTO70H+bcDXcd39vOqb2viZxgqeBeSGtZ55yZU4/Q= -google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d h1:uvYuEyMHKNt+lT4K3bN6fGswmK8qSvcreM3BwjDh+y4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157 h1:7whR9kGa5LUwFtpLm2ArCEejtnxlGeLbAyjFY8sGNFw= +google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157 h1:Zy9XzmMEflZ/MAaA7vNcoebnRAld7FsPW1EeBB7V0m8= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -776,8 +777,8 @@ google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= -google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= -google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= +google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -790,8 +791,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/mainboilerplate/client.go b/mainboilerplate/client.go index e36d56c6..7c83d803 100644 --- a/mainboilerplate/client.go +++ b/mainboilerplate/client.go @@ -10,20 +10,32 @@ import ( "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" pc "go.gazette.dev/core/consumer/protocol" + "go.gazette.dev/core/server" "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/backoff" ) // AddressConfig of a remote service. type AddressConfig struct { - Address pb.Endpoint `long:"address" env:"ADDRESS" default:"http://localhost:8080" description:"Service address endpoint"` + Address pb.Endpoint `long:"address" env:"ADDRESS" default:"http://localhost:8080" description:"Service address endpoint"` + CertFile string `long:"cert-file" env:"CERT_FILE" default:"" description:"Path to the client TLS certificate"` + CertKeyFile string `long:"cert-key-file" env:"CERT_KEY_FILE" default:"" description:"Path to the client TLS private key"` + TrustedCAFile string `long:"trusted-ca-file" env:"TRUSTED_CA_FILE" default:"" description:"Path to the trusted CA for client verification of server certificates"` } // MustDial dials the server address using a protocol.Dispatcher balancer, and panics on error. func (c *AddressConfig) MustDial(ctx context.Context) *grpc.ClientConn { - var cc, err = grpc.DialContext(ctx, + var tlsConfig, err = server.BuildTLSConfig(c.CertFile, c.CertKeyFile, c.TrustedCAFile) + Must(err, "failed to build TLS config") + + // Use a tighter bound for the maximum back-off delay (default is 120s). + var backoffConfig = backoff.DefaultConfig + backoffConfig.MaxDelay = 5 * time.Second + + cc, err := grpc.DialContext(ctx, c.Address.GRPCAddr(), - grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithTransportCredentials(pb.NewDispatchedCredentials(tlsConfig, c.Address)), + grpc.WithConnectParams(grpc.ConnectParams{Backoff: backoffConfig}), // A single Gazette broker frequently serves LOTS of Journals. // Readers will start many concurrent reads of various journals, // but may process them in arbitrary orders, which means a journal @@ -33,9 +45,6 @@ func (c *AddressConfig) MustDial(ctx context.Context) *grpc.ClientConn { // only stream-level flow control. grpc.WithInitialConnWindowSize(math.MaxInt32), grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pb.DispatcherGRPCBalancerName)), - // Use a tighter bound for the maximum back-off delay (default is 120s). - // TODO(johnny): Make this configurable? - grpc.WithBackoffMaxDelay(time.Second*5), // Instrument client for gRPC metric collection. grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor), grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor), diff --git a/mainboilerplate/etcd.go b/mainboilerplate/etcd.go index 549db450..43d68f29 100644 --- a/mainboilerplate/etcd.go +++ b/mainboilerplate/etcd.go @@ -2,31 +2,50 @@ package mainboilerplate import ( "context" + "crypto/tls" "time" log "github.com/sirupsen/logrus" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/broker/protocol" + "go.gazette.dev/core/server" "google.golang.org/grpc" ) // EtcdConfig configures the application Etcd session. type EtcdConfig struct { - Address protocol.Endpoint `long:"address" env:"ADDRESS" default:"http://localhost:2379" description:"Etcd service address endpoint"` - LeaseTTL time.Duration `long:"lease" env:"LEASE_TTL" default:"20s" description:"Time-to-live of Etcd lease"` + Address protocol.Endpoint `long:"address" env:"ADDRESS" default:"http://localhost:2379" description:"Etcd service address endpoint"` + CertFile string `long:"cert-file" env:"CERT_FILE" default:"" description:"Path to the client TLS certificate"` + CertKeyFile string `long:"cert-key-file" env:"CERT_KEY_FILE" default:"" description:"Path to the client TLS private key"` + TrustedCAFile string `long:"trusted-ca-file" env:"TRUSTED_CA_FILE" default:"" description:"Path to the trusted CA for client verification of server certificates"` + LeaseTTL time.Duration `long:"lease" env:"LEASE_TTL" default:"20s" description:"Time-to-live of Etcd lease"` } // MustDial builds an Etcd client connection. func (c *EtcdConfig) MustDial() *clientv3.Client { + var addr = c.Address.URL() + var tlsConfig *tls.Config + + switch addr.Scheme { + case "https": + var err error + tlsConfig, err = server.BuildTLSConfig(c.CertFile, c.CertKeyFile, c.TrustedCAFile) + Must(err, "failed to build TLS config") + case "unix": + // The Etcd client requires hostname is stripped from unix:// URLs. + addr.Host = "" + } + // Use a blocking dial to build a trial connection to Etcd. If we're actively // partitioned or mis-configured this avoids a K8s CrashLoopBackoff, and // there's nothing actionable to do anyway aside from wait (or be SIGTERM'd). var timer = time.AfterFunc(time.Second, func() { - log.WithField("addr", c.Address).Warn("dialing Etcd is taking a while (is network okay?)") + log.WithField("addr", addr.String()).Warn("dialing Etcd is taking a while (is network okay?)") }) - var trialEtcd, err = clientv3.New(clientv3.Config{ - Endpoints: []string{string(c.Address)}, + trialEtcd, err := clientv3.New(clientv3.Config{ + Endpoints: []string{addr.String()}, DialOptions: []grpc.DialOption{grpc.WithBlock()}, + TLS: tlsConfig, }) Must(err, "failed to build trial Etcd client") @@ -35,7 +54,7 @@ func (c *EtcdConfig) MustDial() *clientv3.Client { // Build our actual |etcd| connection, with much tighter timeout bounds. etcd, err := clientv3.New(clientv3.Config{ - Endpoints: []string{string(c.Address)}, + Endpoints: []string{addr.String()}, // Automatically and periodically sync the set of Etcd servers. // If a network split occurs, this allows for attempting different // members until a connectable one is found on our "side" of the network @@ -48,6 +67,7 @@ func (c *EtcdConfig) MustDial() *clientv3.Client { DialKeepAliveTimeout: c.LeaseTTL / 4, // Require a reasonably recent server cluster. RejectOldCluster: true, + TLS: tlsConfig, }) Must(err, "failed to build Etcd client") diff --git a/mainboilerplate/runconsumer/run_consumer.go b/mainboilerplate/runconsumer/run_consumer.go index 66b8c102..e7d9e670 100644 --- a/mainboilerplate/runconsumer/run_consumer.go +++ b/mainboilerplate/runconsumer/run_consumer.go @@ -6,6 +6,7 @@ package runconsumer import ( "context" + "crypto/tls" "fmt" "os" "os/signal" @@ -120,8 +121,21 @@ func (sc Cmd) Execute(args []string) error { }).Info("consumer configuration") pb.RegisterGRPCDispatcher(bc.Consumer.Zone) + var err error + var serverTLS, peerTLS *tls.Config + + if bc.Consumer.ServerCertFile != "" { + serverTLS, err = server.BuildTLSConfig( + bc.Consumer.ServerCertFile, bc.Consumer.ServerCertKeyFile, bc.Consumer.ServerCAFile) + mbp.Must(err, "building server TLS config") + + peerTLS, err = server.BuildTLSConfig( + bc.Consumer.PeerCertFile, bc.Consumer.PeerCertKeyFile, bc.Consumer.PeerCAFile) + mbp.Must(err, "building peer TLS config") + } + // Bind our server listener, grabbing a random available port if Port is zero. - var srv, err = server.New("", bc.Consumer.Port) + srv, err := server.New("", bc.Consumer.Host, bc.Consumer.Port, serverTLS, peerTLS) mbp.Must(err, "building Server instance") if bc.Broker.Cache.Size <= 0 { diff --git a/mainboilerplate/service.go b/mainboilerplate/service.go index 15f0a3e6..a1f36b5a 100644 --- a/mainboilerplate/service.go +++ b/mainboilerplate/service.go @@ -1,12 +1,6 @@ package mainboilerplate import ( - "fmt" - "math/rand" - "net" - "os" - "time" - petname "github.com/dustinkirkland/golang-petname" "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/server" @@ -20,33 +14,25 @@ type ZoneConfig struct { // ServiceConfig represents identification and addressing configuration of the process. type ServiceConfig struct { ZoneConfig - ID string `long:"id" env:"ID" description:"Unique ID of this process. Auto-generated if not set"` - Host string `long:"host" env:"HOST" description:"Addressable, advertised hostname or IP of this process. Hostname is used if not set"` - Port string `long:"port" env:"PORT" description:"Service port for HTTP and gRPC requests. A random port is used if not set. Port may also take the form 'unix:///path/to/socket' to use a Unix Domain Socket"` + ID string `long:"id" env:"ID" description:"Unique ID of this process. Auto-generated if not set"` + Host string `long:"host" env:"HOST" description:"Addressable, advertised hostname or IP of this process. Hostname is used if not set"` + Port string `long:"port" env:"PORT" description:"Service port for HTTP and gRPC requests. A random port is used if not set. Port may also take the form 'unix:///path/to/socket' to use a Unix Domain Socket"` + ServerCertFile string `long:"server-cert-file" env:"SERVER_CERT_FILE" default:"" description:"Path to the server TLS certificate. This option toggles whether TLS is used. If absent, all other TLS settings are ignored."` + ServerCertKeyFile string `long:"server-cert-key-file" env:"SERVER_CERT_KEY_FILE" default:"" description:"Path to the server TLS private key"` + ServerCAFile string `long:"server-ca-file" env:"SERVER_CA_FILE" default:"" description:"Path to the trusted CA for server verification of client certificates. When present, client certificates are required and verified against this CA. When absent, client certificates are not required but are verified against the system CA pool if presented."` + PeerCertFile string `long:"peer-cert-file" env:"PEER_CERT_FILE" default:"" description:"Path to the client TLS certificate for peer-to-peer requests"` + PeerCertKeyFile string `long:"peer-cert-key-file" env:"PEER_CERT_KEY_FILE" default:"" description:"Path to the client TLS private key for peer-to-peer requests"` + PeerCAFile string `long:"peer-ca-file" env:"PEER_CA_FILE" default:"" description:"Path to the trusted CA for client verification of peer server certificates. When absent, the system CA pool is used instead."` } // ProcessSpec of the ServiceConfig. func (cfg ServiceConfig) BuildProcessSpec(srv *server.Server) protocol.ProcessSpec { - var err error if cfg.ID == "" { - rand.Seed(time.Now().UnixNano()) // Seed generator for Generate's use. cfg.ID = petname.Generate(2, "-") } - if cfg.Host == "" { - cfg.Host, err = os.Hostname() - Must(err, "failed to determine hostname") - } - - var endpoint string - switch addr := srv.RawListener.Addr().(type) { - case *net.TCPAddr: - endpoint = fmt.Sprintf("http://%s:%d", cfg.Host, addr.Port) - case *net.UnixAddr: - endpoint = fmt.Sprintf("%s://%s%s", addr.Net, cfg.Host, addr.Name) - } return protocol.ProcessSpec{ Id: protocol.ProcessSpec_ID{Zone: cfg.Zone, Suffix: cfg.ID}, - Endpoint: protocol.Endpoint(endpoint), + Endpoint: srv.Endpoint(), } } diff --git a/server/server.go b/server/server.go index 0367e45b..17ecdbe1 100644 --- a/server/server.go +++ b/server/server.go @@ -2,10 +2,14 @@ package server import ( "context" + "crypto/tls" + "crypto/x509" "fmt" "net" "net/http" "net/url" + "os" + "strconv" "time" grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" @@ -15,16 +19,16 @@ import ( pb "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/task" "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/backoff" ) // Server bundles gRPC & HTTP servers, multiplexed over a single bound TCP // socket (using CMux). Additional protocols may be added to the Server by // interacting directly with its provided CMux. type Server struct { - // RawListener is the bound listener of the Server. - RawListener net.Listener - // CMux wraps RawListener to provide connection protocol multiplexing over + // Advertised Endpoint of this Server. + endpoint pb.Endpoint + // CMux wraps a listener to provide connection protocol multiplexing over // a single bound socket. gRPC and HTTP Listeners are provided by default. // Additional Listeners may be added directly via CMux.Match() -- though // it is then the user's responsibility to Serve the resulting Listeners. @@ -43,37 +47,65 @@ type Server struct { httpServer http.Server } -// New builds and returns a Server of the given TCP network interface |iface| -// and |port|. |port| may be empty, in which case a random free port is assigned. -func New(iface string, port string) (*Server, error) { - var network, addr string +// New builds and returns a Server of the given TCP network interface `iface` +// and `port` for serving traffic directed at `host`. +// `port` may be empty, in which case a random free port is assigned. +// if `tlsConfig` is non-nil, the Server uses TLS (and is otherwise in the clear). +func New(iface, host, port string, serverTLS, peerTLS *tls.Config) (*Server, error) { + var network, bind string if port == "" { - network, addr = "tcp", fmt.Sprintf("%s:0", iface) // Assign a random free port. + network, bind = "tcp", fmt.Sprintf("%s:0", iface) // Assign a random free port. } else if u, err := url.Parse(port); err == nil && u.Scheme == "unix" { - network, addr = "unix", u.Path + network, bind = "unix", u.Path + + // Ignore TLS for UDS. It's already local, and clients cannot use the + // advertised endpoint scheme to determine whether to expect TLS. + serverTLS, peerTLS = nil, nil } else { - network, addr = "tcp", fmt.Sprintf("%s:%s", iface, port) + network, bind = "tcp", fmt.Sprintf("%s:%s", iface, port) } - var raw, err = net.Listen(network, addr) + var listener, err = net.Listen(network, bind) if err != nil { - return nil, errors.Wrapf(err, "failed to bind service address (%s)", addr) + return nil, errors.Wrapf(err, "failed to bind service address (%s)", bind) + } + + // If no host was specified, use the hostname. + if host == "" { + if host, err = os.Hostname(); err != nil { + return nil, err + } + } + // If no port was specified, query the dynamic port which we bound. + if port == "" { + port = strconv.Itoa(listener.Addr().(*net.TCPAddr).Port) + } + + var endpoint string + if network == "unix" { + endpoint = fmt.Sprintf("unix://%s%s", host, bind) + } else { + endpoint = fmt.Sprintf("://%s:%s", host, port) + + if serverTLS != nil { + endpoint = "https" + endpoint + } else { + endpoint = "http" + endpoint + } } - return NewFromListener(raw) -} -// NewFromListener builds a new Server using the provided Listener, which can be customized by the -// caller. This is intended to support servers wishing to use TLS. -func NewFromListener(listener net.Listener) (*Server, error) { var srv = &Server{ - HTTPMux: http.DefaultServeMux, + endpoint: pb.Endpoint(endpoint), + HTTPMux: http.DefaultServeMux, GRPCServer: grpc.NewServer( grpc.StreamInterceptor(grpc_prometheus.StreamServerInterceptor), grpc.UnaryInterceptor(grpc_prometheus.UnaryServerInterceptor), ), - RawListener: listener, } - srv.CMux = cmux.New(srv.RawListener) + if serverTLS != nil { + listener = tls.NewListener(listener, serverTLS) + } + srv.CMux = cmux.New(listener) srv.CMux.HandleError(func(err error) bool { if _, ok := err.(net.Error); !ok { @@ -94,19 +126,21 @@ func NewFromListener(listener net.Listener) (*Server, error) { srv.GRPCListener = srv.CMux.MatchWithWriters( cmux.HTTP2MatchHeaderFieldSendSettings("content-type", "application/grpc")) - var err error + // This grpc.ClientConn connects to this server's loopback, and also + // to peer server addresses via the dispatch balancer. It has particular + // knowledge of what addresses *should* be reach-able (from Etcd + // advertisements). Use an aggressive back-off for server-to-server + // connections, as it's crucial for quick cluster recovery from + // partitions, etc. + var backoffConfig = backoff.DefaultConfig + backoffConfig.MaxDelay = time.Millisecond * 500 + srv.GRPCLoopback, err = grpc.DialContext( context.Background(), - srv.RawListener.Addr().String(), - grpc.WithTransportCredentials(insecure.NewCredentials()), + srv.endpoint.GRPCAddr(), + grpc.WithTransportCredentials(pb.NewDispatchedCredentials(peerTLS, srv.endpoint)), + grpc.WithConnectParams(grpc.ConnectParams{Backoff: backoffConfig}), grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pb.DispatcherGRPCBalancerName)), - // This grpc.ClientConn connects to this server's loopback, and also - // to peer server addresses via the dispatch balancer. It has particular - // knowledge of what addresses *should* be reach-able (from Etcd - // advertisements). Use an aggressive back-off for server-to-server - // connections, as it's crucial for quick cluster recovery from - // partitions, etc. - grpc.WithBackoffMaxDelay(time.Millisecond*500), // Instrument client for gRPC metric collection. grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor), grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor), @@ -122,10 +156,43 @@ func NewFromListener(listener net.Listener) (*Server, error) { return srv, nil } +func BuildTLSConfig(certPath, keyPath, trustedCAPath string) (*tls.Config, error) { + var tlsConfig = &tls.Config{ + ClientAuth: tls.VerifyClientCertIfGiven, + } + + // Load a presented certificate and key. + if certPath != "" { + var cert, err = tls.LoadX509KeyPair(certPath, keyPath) + if err != nil { + return nil, fmt.Errorf("failed to load certificate and key: %w", err) + } + tlsConfig.Certificates = []tls.Certificate{cert} + } + + // Load a trusted Certificate Authority, which is required for clients. + if trustedCAPath != "" { + caCert, err := os.ReadFile(trustedCAPath) + if err != nil { + return nil, fmt.Errorf("failed to read CA certificate: %w", err) + } + + var caCertPool = x509.NewCertPool() + if !caCertPool.AppendCertsFromPEM(caCert) { + return nil, fmt.Errorf("failed to append CA certificate to pool") + } + tlsConfig.ClientCAs = caCertPool + tlsConfig.RootCAs = caCertPool + tlsConfig.ClientAuth = tls.RequireAndVerifyClientCert + } + + return tlsConfig, nil +} + // MustLoopback builds and returns a new Server instance bound to a random // port on the loopback interface. It panics on error. func MustLoopback() *Server { - if srv, err := New("127.0.0.1", ""); err != nil { + if srv, err := New("127.0.0.1", "127.0.0.1", "", nil, nil); err != nil { log.WithField("err", err).Panic("failed to build Server") panic("not reached") } else { @@ -135,7 +202,7 @@ func MustLoopback() *Server { // Endpoint of the Server. func (s *Server) Endpoint() pb.Endpoint { - return pb.Endpoint("http://" + s.RawListener.Addr().String()) + return s.endpoint } // QueueTasks serving the CMux, HTTP, and gRPC component servers onto the task.Group. From bc377d8a8d2ef6b4193b022de223c7ac75622957 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Wed, 19 Jun 2024 21:09:05 -0500 Subject: [PATCH 2/8] protocol: LabelSelectors support prefix matches In uses of Gazette, labels often encode a hierarchical namespace such as "foo/bar/baz", and for such labels it's desireable to be able to create a LabelSelector that includes or excludes an entire sub-hierarchy of potential label values, such as including or excluding any label that begins with "foo/bar/". Extend Label to have a Prefix field which may only be set in the context of a LabelSelector (and is a Validate() error otherwise). In a LabelSelector context, Prefix instructs selector matching to match any label value which is prefixed by the given selector label. Introduce a convention of "my-label:prefix" as a special suffix which indicates that prefix matching is desired, and update LabelSelector parsing to round-trip such ":prefix" suffixes. Update the implementation of the special meta-label "prefix" to be in terms of a "name:prefix" selector label, and back out the bespoke implementation that has until-now been used for journal name prefix matching. --- broker/list_apply_api.go | 12 + broker/list_apply_api_test.go | 9 +- broker/protocol/journal_spec_extensions.go | 17 +- .../protocol/journal_spec_extensions_test.go | 7 +- broker/protocol/label_extensions.go | 108 +++-- broker/protocol/label_extensions_test.go | 67 ++-- broker/protocol/protocol.pb.go | 374 ++++++++++-------- broker/protocol/protocol.proto | 7 + 8 files changed, 352 insertions(+), 249 deletions(-) diff --git a/broker/list_apply_api.go b/broker/list_apply_api.go index 4d15e450..85f1fcc7 100644 --- a/broker/list_apply_api.go +++ b/broker/list_apply_api.go @@ -39,6 +39,18 @@ func (svc *Service) List(ctx context.Context, req *pb.ListRequest) (resp *pb.Lis } var metaLabels, allLabels pb.LabelSet + // Gazette has historically offered a special "prefix" label which matches + // slash-terminated prefixes of a journal name. Today, it's implemented in + // terms of a LabelSelector prefix match. + for _, set := range []*pb.LabelSet{&req.Selector.Include, &req.Selector.Exclude} { + if prefix := set.ValuesOf("prefix"); len(prefix) != 0 { + for _, val := range prefix { + set.AddValue("name:prefix", val) + } + set.Remove("prefix") + } + } + defer s.KS.Mu.RUnlock() s.KS.Mu.RLock() diff --git a/broker/list_apply_api_test.go b/broker/list_apply_api_test.go index 6244a917..56813adc 100644 --- a/broker/list_apply_api_test.go +++ b/broker/list_apply_api_test.go @@ -110,7 +110,14 @@ func TestListCases(t *testing.T) { require.NoError(t, err) verify(resp, specB) - // Case: Meta-label "prefix" selects journals by name prefix. + // Case: Meta-label "name:prefix" selects journals by name prefix. + resp, err = broker.client().List(ctx, &pb.ListRequest{ + Selector: pb.LabelSelector{Include: pb.MustLabelSet("name:prefix", "journal/1")}, + }) + require.NoError(t, err) + verify(resp, specA, specC) + + // Case: legacy meta-label "prefix" also selects journals by name prefix. resp, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("prefix", "journal/1/")}, }) diff --git a/broker/protocol/journal_spec_extensions.go b/broker/protocol/journal_spec_extensions.go index 042c755d..82a1ea3d 100644 --- a/broker/protocol/journal_spec_extensions.go +++ b/broker/protocol/journal_spec_extensions.go @@ -359,24 +359,17 @@ func SubtractJournalSpecs(a, b JournalSpec) JournalSpec { } // ExtractJournalSpecMetaLabels adds to the LabelSet a singular label "name", -// with value of the JournalSpec Name, and multi-label "prefix", having a value -// for each path component prefix of Name. +// with value of the JournalSpec Name. func ExtractJournalSpecMetaLabels(spec *JournalSpec, out LabelSet) LabelSet { - var name = spec.Name.String() - out.Labels = append(out.Labels[:0], Label{Name: "name", Value: name}) - - for i, j := 0, strings.IndexByte(name, '/'); j != -1; j = strings.IndexByte(name[i:], '/') { - i += j + 1 - out.Labels = append(out.Labels, Label{Name: "prefix", Value: name[:i]}) - } + out.Labels = append(out.Labels[:0], Label{Name: "name", Value: spec.Name.String()}) return out } // validateJournalLabelConstraints asserts expected invariants of MessageType, // MessageSubType, and ContentType labels: -// * ContentType must parse as a RFC 1521 MIME / media-type. -// * If MessageType is present, so is ContentType. -// * If MessageSubType is present, so is MessageType. +// - ContentType must parse as a RFC 1521 MIME / media-type. +// - If MessageType is present, so is ContentType. +// - If MessageSubType is present, so is MessageType. func validateJournalLabelConstraints(ls LabelSet) error { if err := ValidateSingleValueLabels(ls); err != nil { return err diff --git a/broker/protocol/journal_spec_extensions_test.go b/broker/protocol/journal_spec_extensions_test.go index a43187b4..526f3f60 100644 --- a/broker/protocol/journal_spec_extensions_test.go +++ b/broker/protocol/journal_spec_extensions_test.go @@ -179,12 +179,7 @@ func (s *JournalSuite) TestSpecValidationCases(c *gc.C) { func (s *JournalSuite) TestMetaLabelExtraction(c *gc.C) { c.Check(ExtractJournalSpecMetaLabels(&JournalSpec{Name: "path/to/my/journal"}, MustLabelSet("label", "buffer")), - gc.DeepEquals, MustLabelSet( - "name", "path/to/my/journal", - "prefix", "path/", - "prefix", "path/to/", - "prefix", "path/to/my/", - )) + gc.DeepEquals, MustLabelSet("name", "path/to/my/journal")) } func (s *JournalSuite) TestFlagYAMLRoundTrip(c *gc.C) { diff --git a/broker/protocol/label_extensions.go b/broker/protocol/label_extensions.go index c97bb2dc..0fe416f6 100644 --- a/broker/protocol/label_extensions.go +++ b/broker/protocol/label_extensions.go @@ -4,16 +4,22 @@ import ( "bytes" "regexp" "sort" + "strings" "go.gazette.dev/core/labels" ) // Validate returns an error if the Label is not well-formed. func (m Label) Validate() error { + return m.validate(false) +} +func (m Label) validate(allowPrefix bool) error { if err := ValidateToken(m.Name, TokenSymbols, minLabelLen, maxLabelLen); err != nil { return ExtendContext(err, "Name") } else if err = ValidateToken(m.Value, pathSymbols, 0, maxLabelValueLen); err != nil { return ExtendContext(err, "Value") + } else if m.Prefix && !allowPrefix { + return NewValidationError("Prefix may not be set outside of a LabelSelector") } return nil } @@ -27,7 +33,7 @@ func MustLabelSet(nv ...string) (set LabelSet) { for i := 0; i != len(nv); i += 2 { set.AddValue(nv[i], nv[i+1]) } - if err := set.Validate(); err != nil { + if err := set.validate(true); err != nil { panic(err.Error()) } return @@ -35,8 +41,11 @@ func MustLabelSet(nv ...string) (set LabelSet) { // Validate returns an error if the LabelSet is not well-formed. func (m LabelSet) Validate() error { + return m.validate(false) +} +func (m LabelSet) validate(allowPrefix bool) error { for i := range m.Labels { - if err := m.Labels[i].Validate(); err != nil { + if err := m.Labels[i].validate(allowPrefix); err != nil { return ExtendContext(err, "Labels[%d]", i) } else if i == 0 { continue @@ -90,7 +99,12 @@ func (m LabelSet) ValueOf(name string) string { } // SetValue adds Label |name| with |value|, replacing any existing Labels |name|. +// If |name| has the special suffix ":prefix", the Label is marked as a prefix +// match. It's only valid to use ":prefix" within the context of a LabelSelector. func (m *LabelSet) SetValue(name, value string) { + var prefix = strings.HasSuffix(name, ":prefix") + name = strings.TrimSuffix(name, ":prefix") + var begin = sort.Search(len(m.Labels), func(i int) bool { return m.Labels[i].Name >= name }) var end = begin @@ -103,12 +117,17 @@ func (m *LabelSet) SetValue(name, value string) { end = begin + 1 } - m.Labels[begin] = Label{Name: name, Value: value} + m.Labels[begin] = Label{Name: name, Value: value, Prefix: prefix} m.Labels = append(m.Labels[:begin+1], m.Labels[end:]...) // Cut |begin+1, end). } // AddValue adds Label |name| with |value|, retaining any existing Labels |name|. +// If |name| has the special suffix ":prefix", the Label is marked as a prefix +// match. It's only valid to use ":prefix" within the context of a LabelSelector. func (m *LabelSet) AddValue(name, value string) { + var prefix = strings.HasSuffix(name, ":prefix") + name = strings.TrimSuffix(name, ":prefix") + var ind = sort.Search(len(m.Labels), func(i int) bool { if m.Labels[i].Name != name { return m.Labels[i].Name > name @@ -116,7 +135,7 @@ func (m *LabelSet) AddValue(name, value string) { return m.Labels[i].Value >= value } }) - var label = Label{Name: name, Value: value} + var label = Label{Name: name, Value: value, Prefix: prefix} if ind != len(m.Labels) && m.Labels[ind] == label { // No-op. @@ -215,9 +234,9 @@ func SubtractLabelSet(lhs, rhs, out LabelSet) LabelSet { // Validate returns an error if the LabelSelector is not well-formed. func (m LabelSelector) Validate() error { - if err := m.Include.Validate(); err != nil { + if err := m.Include.validate(true); err != nil { return ExtendContext(err, "Include") - } else if err := m.Exclude.Validate(); err != nil { + } else if err := m.Exclude.validate(true); err != nil { return ExtendContext(err, "Exclude") } return nil @@ -237,7 +256,20 @@ func (m LabelSelector) Matches(s LabelSet) bool { func (s LabelSelector) String() string { var w = bytes.NewBuffer(nil) - var f = func(l []Label, exc bool) { + var f = func(input []Label, exc bool) { + var l []Label + + // Attach literal ":prefix" suffixes to names so that prefixes are + // distinguished during the subsequent self-join over `l`. + for _, ll := range input { + if ll.Prefix { + ll.Name = ll.Name + ":prefix" + } + l = append(l, ll) + } + // We may have changed Name ordering. Re-index. + sort.Slice(l, func(i, j int) bool { return l[i].Name < l[j].Name }) + var it = labelJoin{setL: l, setR: l, lenL: len(l), lenR: len(l)} for cur, ok := it.next(); ok; cur, ok = it.next() { if cur.lBeg+1 == cur.lEnd && l[cur.lBeg].Value == "" { @@ -298,7 +330,9 @@ func matchSelector(sel, set []Label, reqAll bool) bool { var matched bool for a, b := sel[cur.lBeg:cur.lEnd], set[cur.rBeg:cur.rEnd]; !matched && len(a) != 0 && len(b) != 0; { - if a[0].Value == "" || a[0].Value == b[0].Value { + if a[0].Value == "" || + (!a[0].Prefix && a[0].Value == b[0].Value) || + (a[0].Prefix && strings.HasPrefix(b[0].Value, a[0].Value)) { matched = true // Selector value "" implicitly matches any value of the label. } else if a[0].Value < b[0].Value { a = a[1:] @@ -378,16 +412,22 @@ func labelValuesEqual(it labelJoin, cur labelJoinCursor) bool { // expression types are equality, in-equality, set membership, set exclusion, // existence, and non-existence. Eg: // -// * "foo = bar" requires that label "foo" be present with value "bar" -// * "foo != bar" requires that label "foo" not be present with value "bar" -// * "foo" requires that label "foo" be present (with any value). -// * "!foo" requires that label "foo" not be present. -// * "foo in (bar,baz)" requires that "foo" be present with either "bar" or "baz". -// * "foo notin (bar,baz)" requires that "foo", if present, not have value "bar" or "baz". +// - "foo = bar" requires that label "foo" be present with value "bar" +// - "foo != bar" requires that label "foo" not be present with value "bar" +// - "foo" requires that label "foo" be present (with any value). +// - "!foo" requires that label "foo" not be present. +// - "foo in (bar,baz)" requires that "foo" be present with either "bar" or "baz". +// - "foo notin (bar,baz)" requires that "foo", if present, not have value "bar" or "baz". +// +// A label name within a selector can have a "my-label:prefix" suffix which +// tests whether the selector value is a prefix of the named label value, +// instead of using exact equality. +// For example, "foo:prefix in (one/two/, three/)" would match "one/two/three" +// and "three/four/five" but not "one/one/" or "four/five/six". // // Additional examples of composite expressions: -// * "topic in (topic/one, topic/two), prefix=/my/journal/prefix" -// * "env in (production, qa), tier not in (frontend,backend), partition" +// - "topic in (topic/one, topic/two), name:prefix=/my/journal/prefix" +// - "env in (production, qa), tier not in (frontend,backend), partition" // // ParseLabelSelector is invariant to _reasonable_ spacing: eg, "not in" and // "notin" may be used interchangeably, as may "==" and "=", with or without @@ -398,60 +438,46 @@ func ParseLabelSelector(s string) (LabelSelector, error) { for len(s) != 0 { var m []string if m = reSelectorEqual.FindStringSubmatch(s); m != nil { - out.Include.Labels = append(out.Include.Labels, Label{Name: m[1], Value: m[2]}) + out.Include.AddValue(m[1], m[2]) } else if m = reSelectorNotEqual.FindStringSubmatch(s); m != nil { - out.Exclude.Labels = append(out.Exclude.Labels, Label{Name: m[1], Value: m[2]}) + out.Exclude.AddValue(m[1], m[2]) } else if m = reSelectorSetIn.FindStringSubmatch(s); m != nil { - if parts, err := parseSetParts(m[1], m[2]); err != nil { + if err := parseSetParts(&out.Include, m[1], m[2]); err != nil { return LabelSelector{}, ExtendContext(err, "parsing %q", s) - } else { - out.Include.Labels = append(out.Include.Labels, parts...) } } else if m = reSelectorSetNotIn.FindStringSubmatch(s); m != nil { - if parts, err := parseSetParts(m[1], m[2]); err != nil { + if err := parseSetParts(&out.Exclude, m[1], m[2]); err != nil { return LabelSelector{}, ExtendContext(err, "parsing %q", s) - } else { - out.Exclude.Labels = append(out.Exclude.Labels, parts...) } } else if m = reSelectorSetExists.FindStringSubmatch(s); m != nil { - out.Include.Labels = append(out.Include.Labels, Label{Name: m[1]}) + out.Include.AddValue(m[1], "") } else if m = reSelectorSetNotExists.FindStringSubmatch(s); m != nil { - out.Exclude.Labels = append(out.Exclude.Labels, Label{Name: m[1]}) + out.Exclude.AddValue(m[1], "") } else { return LabelSelector{}, NewValidationError("could not match %q to a label selector expression", s) } s = s[len(m[0]):] } - for _, l := range [][]Label{out.Include.Labels, out.Exclude.Labels} { - sort.Slice(l, func(i, j int) bool { - if l[i].Name != l[j].Name { - return l[i].Name < l[j].Name - } - return l[i].Value < l[j].Value - }) - } return out, out.Validate() } -func parseSetParts(name, s string) ([]Label, error) { - var out []Label - +func parseSetParts(set *LabelSet, name, s string) error { for len(s) != 0 { var m []string if m = reSelectorSetValue.FindStringSubmatch(s); m != nil { - out = append(out, Label{Name: name, Value: m[1]}) + set.AddValue(name, m[1]) } else { - return nil, NewValidationError("could not match %q to a label selector set expression", s) + return NewValidationError("could not match %q to a label selector set expression", s) } s = s[len(m[0]):] } - return out, nil + return nil } var ( - reToken = ` ?([\pL\pN\` + regexp.QuoteMeta(TokenSymbols) + `]{2,})` + reToken = ` ?([\pL\pN\` + regexp.QuoteMeta(TokenSymbols) + `]{2,}(?:\:prefix)?)` rePath = ` ?([\pL\pN\` + regexp.QuoteMeta(pathSymbols) + `]{0,})` reCommaOrEnd = ` ?(?:,|$)` reParenthetical = ` ?\(([^)]+)\)` diff --git a/broker/protocol/label_extensions_test.go b/broker/protocol/label_extensions_test.go index b5f47918..2d4fc4a9 100644 --- a/broker/protocol/label_extensions_test.go +++ b/broker/protocol/label_extensions_test.go @@ -22,6 +22,7 @@ func (s *LabelSuite) TestLabelValidationCases(c *gc.C) { {strings.Repeat("a", maxLabelLen+1), "a-value", `Name: invalid length \(65; expected .*`}, {"a-label", "", ""}, // Success {"a-label", strings.Repeat("a", maxLabelValueLen+1), `Value: invalid length \(1025; expected .*`}, + {"a-label:prefix", "a-value", `Name: not a valid token \(a-label:prefix\)`}, } for _, tc := range cases { if tc.expect == "" { @@ -50,6 +51,7 @@ func (s *LabelSuite) TestSetValidationCases(c *gc.C) { {Name: "ccc", Value: ""}, {Name: "ccc", Value: ""}, {Name: "ccc", Value: "555"}, + {Name: "ddd", Value: "666", Prefix: true}, }, } @@ -68,16 +70,30 @@ func (s *LabelSuite) TestSetValidationCases(c *gc.C) { c.Check(set.Validate(), gc.ErrorMatches, `Label has empty & non-empty values \(index 5; label ccc value 444\)`) set.Labels[4].Value = "44" + c.Check(set.Validate(), gc.ErrorMatches, `Labels\[7\]: Prefix may not be set outside of a LabelSelector`) + set.Labels[7].Prefix = false + c.Check(set.Validate(), gc.IsNil) } func (s *LabelSuite) TestEqualityCases(c *gc.C) { var lhs = MustLabelSet("one", "1", "three", "3") - var rhs = MustLabelSet("one", "1") + var rhs = MustLabelSet("one", "1", "two:prefix", "pre/") c.Check(lhs.Equal(&rhs), gc.Equals, false) rhs.AddValue("three", "3") + c.Check(lhs.Equal(&rhs), gc.Equals, false) + lhs.AddValue("two:prefix", "pre/") c.Check(lhs.Equal(&rhs), gc.Equals, true) + + c.Check(lhs.Equal(&LabelSet{ + Labels: []Label{ + {Name: "one", Value: "1"}, + {Name: "three", Value: "3"}, + {Name: "two", Value: "pre/", Prefix: true}, + }, + }), gc.Equals, true) + rhs.AddValue("four", "4") c.Check(lhs.Equal(&rhs), gc.Equals, false) @@ -144,16 +160,16 @@ func (s *LabelSuite) TestSetAddAndRemove(c *gc.C) { c.Check(set, gc.DeepEquals, MustLabelSet("aa", "0", "aa", "2", "bb", "1", "bb", "3", "bb", "4")) - set.SetValue("bb", "5") // Replace at end. - set.SetValue("cc", "6") // Insert at end. - set.SetValue("00", "6") // Insert at beginning. + set.SetValue("bb:prefix", "5") // Replace at end. + set.SetValue("cc", "6") // Insert at end. + set.SetValue("00", "6") // Insert at beginning. - c.Check(set, gc.DeepEquals, MustLabelSet("00", "6", "aa", "0", "aa", "2", "bb", "5", "cc", "6")) + c.Check(set, gc.DeepEquals, MustLabelSet("00", "6", "aa", "0", "aa", "2", "bb:prefix", "5", "cc", "6")) set.SetValue("00", "7") // Replace at beginning set.SetValue("aa", "8") // Replace at middle. - c.Check(set, gc.DeepEquals, MustLabelSet("00", "7", "aa", "8", "bb", "5", "cc", "6")) + c.Check(set, gc.DeepEquals, MustLabelSet("00", "7", "aa", "8", "bb:prefix", "5", "cc", "6")) set.AddValue("bb", "9") set.AddValue("00", "00") @@ -250,6 +266,7 @@ func (s *LabelSuite) TestSelectorValidationCases(c *gc.C) { Include: LabelSet{ Labels: []Label{ {Name: "include", Value: "a-value"}, + {Name: "prefix-thing", Value: "prefix/", Prefix: true}, }, }, Exclude: LabelSet{ @@ -271,7 +288,7 @@ func (s *LabelSuite) TestSelectorValidationCases(c *gc.C) { func (s *LabelSuite) TestSelectorMatchingCases(c *gc.C) { var sel = LabelSelector{ Include: MustLabelSet( - "inc-1", "a-val", + "inc-1:prefix", "a-val/", "inc-2", "", "inc-3", "val-1", "inc-3", "val-2"), @@ -287,17 +304,17 @@ func (s *LabelSuite) TestSelectorMatchingCases(c *gc.C) { expect bool }{ {set: MustLabelSet(), expect: false}, // Not matched. - {set: MustLabelSet("foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-1"), expect: true}, // Matched. - {set: MustLabelSet("foo", "bar", "inc-1", "a-val", "inc-2", "foo", "inc-3", "val-1"), expect: true}, // Matched (invariant to inc-2 value). - {set: MustLabelSet("foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-2"), expect: true}, // Matched (alternate inc-3 value). + {set: MustLabelSet("foo", "bar", "inc-1", "a-val/a/1", "inc-2", "any", "inc-3", "val-1"), expect: true}, // Matched. + {set: MustLabelSet("foo", "bar", "inc-1", "a-val/b/2", "inc-2", "foo", "inc-3", "val-1"), expect: true}, // Matched (invariant to inc-2 value). + {set: MustLabelSet("foo", "bar", "inc-1", "a-val/c/3", "inc-2", "any", "inc-3", "val-2"), expect: true}, // Matched (alternate inc-3 value). - {set: MustLabelSet("foo", "bar", "inc-1", "bad-val", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (inc-1 not matched). - {set: MustLabelSet("foo", "bar", "inc-1", "a-val", "inc-3", "val-1"), expect: false}, // Not matched (inc-2 missing). - {set: MustLabelSet("foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-other"), expect: false}, // Not matched (inc-3 not matched). + {set: MustLabelSet("foo", "bar", "inc-1", "bad-val/4", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (inc-1 not matched). + {set: MustLabelSet("foo", "bar", "inc-1", "a-val/5", "inc-3", "val-1"), expect: false}, // Not matched (inc-2 missing). + {set: MustLabelSet("foo", "bar", "inc-1", "a-val/6", "inc-2", "any", "inc-3", "val-other"), expect: false}, // Not matched (inc-3 not matched). - {set: MustLabelSet("exc-1", "any", "foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (exc-1 matched). - {set: MustLabelSet("exc-2", "val-4", "foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (exc-2 matched). - {set: MustLabelSet("exc-2", "val-ok", "foo", "bar", "inc-1", "a-val", "inc-2", "any", "inc-3", "val-1"), expect: true}, // Matched (exc-2 not matched). + {set: MustLabelSet("exc-1", "any", "foo", "bar", "inc-1", "a-val/a/7", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (exc-1 matched). + {set: MustLabelSet("exc-2", "val-4", "foo", "bar", "inc-1", "a-val/8", "inc-2", "any", "inc-3", "val-1"), expect: false}, // Not matched (exc-2 matched). + {set: MustLabelSet("exc-2", "val-ok", "foo", "bar", "inc-1", "a-val/9/9", "inc-2", "any", "inc-3", "val-1"), expect: true}, // Matched (exc-2 not matched). } for _, tc := range cases { c.Check(sel.Matches(tc.set), gc.Equals, tc.expect) @@ -316,9 +333,9 @@ func (s *LabelSuite) TestOuterJoin(c *gc.C) { var lhs, rhs = MustLabelSet( "aaa", "l0", "aaa", "l1", - "ccc", "l2", - "ccc", "l3", - "ccc", "l4", + "ccc:prefix", "l2", + "ccc:prefix", "l3", + "ccc:prefix", "l4", "eee", "l5", "eee", "l6", ), MustLabelSet( @@ -361,6 +378,10 @@ func (s *LabelSuite) TestSelectorParsingCases(c *gc.C) { s: "foo = bar", expect: LabelSelector{Include: MustLabelSet("foo", "bar")}, }, + { + s: "foo:prefix =bar", + expect: LabelSelector{Include: MustLabelSet("foo:prefix", "bar")}, + }, { s: "foo != bar", expect: LabelSelector{Exclude: MustLabelSet("foo", "bar")}, @@ -382,17 +403,17 @@ func (s *LabelSuite) TestSelectorParsingCases(c *gc.C) { expect: LabelSelector{Exclude: MustLabelSet("foo", "apple", "foo", "pear")}, }, { - s: "foo==bar, baz !=bing ,apple in (fruit, banana)", + s: "foo==bar, baz !=bing ,apple:prefix in (fruit, banana)", expect: LabelSelector{ - Include: MustLabelSet("apple", "banana", "apple", "fruit", "foo", "bar"), + Include: MustLabelSet("apple:prefix", "banana", "apple:prefix", "fruit", "foo", "bar"), Exclude: MustLabelSet("baz", "bing"), }, }, { - s: "!foo,baz,bing not in (thing-one, thing-2),!bar,", + s: "!foo,baz,bing:prefix not in (thing-one, thing-2),!bar,", expect: LabelSelector{ Include: MustLabelSet("baz", ""), - Exclude: MustLabelSet("bar", "", "bing", "thing-2", "bing", "thing-one", "foo", ""), + Exclude: MustLabelSet("bar", "", "bing:prefix", "thing-2", "bing:prefix", "thing-one", "foo", ""), }, }, // Label values may include '='. diff --git a/broker/protocol/protocol.pb.go b/broker/protocol/protocol.pb.go index a1d20a99..819d3de1 100644 --- a/broker/protocol/protocol.pb.go +++ b/broker/protocol/protocol.pb.go @@ -12,10 +12,10 @@ import ( proto "github.com/gogo/protobuf/proto" github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" golang_proto "github.com/golang/protobuf/proto" - _ "github.com/golang/protobuf/ptypes/duration" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" status "google.golang.org/grpc/status" + _ "google.golang.org/protobuf/types/known/durationpb" io "io" math "math" math_bits "math/bits" @@ -225,8 +225,13 @@ func (JournalSpec_Flag) EnumDescriptor() ([]byte, []int) { // attributes which do not directly imply semantics to the core system, but // are meaningful to users or for higher-level Gazette tools. type Label struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Name of this label. + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // Value of this label. Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + // Does this label match on value prefix? + // May only be set true within a LabelSelector. + Prefix bool `protobuf:"varint,3,opt,name=prefix,proto3" json:"prefix,omitempty" yaml:",omitempty"` } func (m *Label) Reset() { *m = Label{} } @@ -1767,170 +1772,171 @@ func init() { } var fileDescriptor_0c0999e5af553218 = []byte{ - // 2597 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x3d, 0x70, 0x1b, 0xc7, - 0xf5, 0xe7, 0x01, 0x07, 0xe0, 0xf0, 0x00, 0x90, 0xc7, 0xb5, 0x25, 0x51, 0xb0, 0x45, 0xd0, 0xf0, - 0xc7, 0x9f, 0x92, 0x6d, 0xd0, 0xa6, 0xff, 0xb1, 0x1d, 0x65, 0x9c, 0x18, 0x20, 0x40, 0x09, 0x32, - 0x04, 0x60, 0x16, 0xa0, 0x65, 0xb9, 0xc8, 0xcd, 0xf1, 0x6e, 0x09, 0x5e, 0x78, 0xb8, 0x43, 0xee, - 0x16, 0x32, 0xe9, 0xce, 0x4d, 0xe2, 0xc2, 0x99, 0xc9, 0xa4, 0x72, 0x95, 0x71, 0x93, 0x3a, 0x99, - 0x49, 0x97, 0x34, 0x29, 0x95, 0xce, 0x65, 0x66, 0x92, 0x30, 0x13, 0x6b, 0x26, 0x93, 0x5a, 0xa5, - 0xab, 0xcc, 0x7e, 0x1c, 0x70, 0x04, 0x41, 0x51, 0x2a, 0xd8, 0x60, 0x76, 0xdf, 0xd7, 0xbe, 0x7d, - 0xef, 0xed, 0x6f, 0xdf, 0x1e, 0x60, 0x75, 0x37, 0xf0, 0x0f, 0x48, 0xb0, 0x31, 0x0a, 0x7c, 0xea, - 0x5b, 0xbe, 0x3b, 0x19, 0x54, 0xf8, 0x00, 0x69, 0xd1, 0xbc, 0xf8, 0xfc, 0xc0, 0x1f, 0xf8, 0x7c, - 0xb6, 0xc1, 0x46, 0x82, 0x5f, 0x5c, 0x1d, 0xf8, 0xfe, 0xc0, 0x25, 0x42, 0x6d, 0x77, 0xbc, 0xb7, - 0x61, 0x8f, 0x03, 0x93, 0x3a, 0xbe, 0x27, 0xf8, 0xe5, 0xf7, 0x20, 0xd5, 0x32, 0x77, 0x89, 0x8b, - 0x10, 0xa8, 0x9e, 0x39, 0x24, 0x2b, 0xca, 0x9a, 0xb2, 0x9e, 0xc5, 0x7c, 0x8c, 0x9e, 0x87, 0xd4, - 0x03, 0xd3, 0x1d, 0x93, 0x95, 0x04, 0x27, 0x8a, 0xc9, 0x4d, 0xf5, 0xbf, 0xdf, 0x94, 0x94, 0x72, - 0x1f, 0x34, 0xae, 0xd8, 0x23, 0x14, 0xd5, 0x20, 0xed, 0xb2, 0x71, 0xb8, 0xa2, 0xac, 0x25, 0xd7, - 0x73, 0x9b, 0x4b, 0x95, 0x89, 0x97, 0x5c, 0xa6, 0x76, 0xf5, 0xe1, 0x71, 0x69, 0xe1, 0xf1, 0x71, - 0x69, 0xf9, 0xc8, 0x1c, 0xba, 0x37, 0xcb, 0x6f, 0xf8, 0x43, 0x87, 0x92, 0xe1, 0x88, 0x1e, 0x95, - 0xb1, 0xd4, 0x94, 0x56, 0xbf, 0x50, 0xa0, 0x20, 0xcd, 0xba, 0xc4, 0xa2, 0x7e, 0x80, 0x36, 0x21, - 0xe3, 0x78, 0x96, 0x3b, 0xb6, 0x85, 0x6b, 0xb9, 0x4d, 0x34, 0x63, 0xbc, 0x47, 0x68, 0x4d, 0x65, - 0xf6, 0x71, 0x24, 0xc8, 0x74, 0xc8, 0xa1, 0xd0, 0x49, 0x9c, 0xa7, 0x23, 0x05, 0x6f, 0x6a, 0x5f, - 0x7f, 0x53, 0x5a, 0xe0, 0x3e, 0x7c, 0x95, 0x85, 0xdc, 0x1d, 0x7f, 0x1c, 0x78, 0xa6, 0xdb, 0x1b, - 0x11, 0x0b, 0xfd, 0x7f, 0x3c, 0x32, 0xb5, 0xb5, 0xb9, 0xdb, 0xf8, 0xfe, 0xb8, 0x94, 0x91, 0x3a, - 0x32, 0x76, 0xef, 0x41, 0x2e, 0x20, 0x23, 0xd7, 0xb1, 0x78, 0xb4, 0xb9, 0x1f, 0xa9, 0xda, 0xa5, - 0xf9, 0x31, 0x88, 0x4b, 0xa2, 0xee, 0x24, 0x98, 0xc9, 0x33, 0x7d, 0x7f, 0x85, 0xf9, 0xfe, 0xed, - 0x71, 0x49, 0x79, 0x7c, 0x5c, 0x5a, 0x99, 0xb5, 0xf7, 0x86, 0xe3, 0xb9, 0x8e, 0x47, 0x26, 0xa1, - 0x45, 0x3b, 0xa0, 0xed, 0x05, 0xe6, 0x60, 0x48, 0x3c, 0xba, 0xa2, 0x72, 0x9b, 0xab, 0x53, 0x9b, - 0xb1, 0x9d, 0x56, 0xb6, 0xa5, 0xd4, 0x93, 0xf2, 0x35, 0x31, 0x85, 0x7e, 0x02, 0xa9, 0x3d, 0xd7, - 0x1c, 0x84, 0x2b, 0xe9, 0x35, 0x65, 0xbd, 0x50, 0xbb, 0x7e, 0x56, 0x60, 0xf4, 0xd8, 0x12, 0xc6, - 0xb6, 0x6b, 0x0e, 0xb0, 0xd0, 0x43, 0x2d, 0x58, 0x1a, 0x9a, 0x87, 0x86, 0x39, 0x1a, 0x11, 0xcf, - 0x36, 0x02, 0x93, 0x92, 0x95, 0xcc, 0x9a, 0xb2, 0x9e, 0xac, 0xbd, 0xf2, 0xf8, 0xb8, 0xb4, 0x26, - 0x4c, 0xcd, 0x08, 0xc4, 0x3d, 0x29, 0x0c, 0xcd, 0xc3, 0x2a, 0x67, 0x61, 0x93, 0x92, 0xe2, 0x57, - 0x29, 0xd0, 0xa2, 0x0d, 0xa0, 0x37, 0x21, 0xed, 0x12, 0x6f, 0x40, 0xf7, 0x79, 0xd6, 0x92, 0x67, - 0x05, 0x5e, 0x0a, 0x21, 0x1f, 0x96, 0x2d, 0x7f, 0x38, 0x0a, 0x48, 0x18, 0x3a, 0xbe, 0x67, 0x58, - 0xbe, 0x4d, 0x2c, 0x9e, 0xb2, 0xc5, 0xcd, 0xe2, 0x34, 0x54, 0x5b, 0x53, 0x91, 0x2d, 0x26, 0x51, - 0x7b, 0xed, 0xf1, 0x71, 0xa9, 0x2c, 0xac, 0x9e, 0x52, 0x8f, 0x2f, 0xa3, 0x5b, 0x33, 0x9a, 0xe8, - 0xc7, 0x90, 0x0e, 0xa9, 0x1f, 0x10, 0x96, 0xe4, 0xe4, 0x7a, 0x96, 0x5b, 0x9a, 0x1b, 0xbc, 0x42, - 0xb4, 0xa5, 0x1e, 0x13, 0xc7, 0x52, 0x0b, 0x85, 0xa0, 0x07, 0x64, 0x2f, 0x20, 0xe1, 0xbe, 0xe1, - 0x78, 0x94, 0x04, 0x0f, 0x4c, 0x57, 0xa6, 0xf6, 0x6a, 0x45, 0x9c, 0xf8, 0x4a, 0x74, 0xe2, 0x2b, - 0x75, 0x79, 0xe2, 0x6b, 0x6f, 0xca, 0xac, 0xbe, 0x24, 0x16, 0x9a, 0x35, 0x10, 0x5b, 0xf8, 0xeb, - 0x7f, 0x95, 0x14, 0xbc, 0x24, 0x05, 0x9a, 0x92, 0x8f, 0x3e, 0x86, 0x6c, 0x40, 0x28, 0xf1, 0x78, - 0x41, 0xa7, 0xce, 0x5b, 0xed, 0xda, 0x99, 0x35, 0xc4, 0xad, 0x4f, 0x4d, 0xa1, 0x21, 0x2c, 0xee, - 0xb9, 0xe3, 0xf8, 0x56, 0xd2, 0xe7, 0x19, 0x7f, 0x5d, 0x1a, 0x2f, 0x09, 0xe3, 0x27, 0xd5, 0x67, - 0x97, 0x2a, 0x70, 0xf6, 0x64, 0x1b, 0x3f, 0x85, 0x4b, 0x23, 0x93, 0xee, 0x1b, 0x23, 0x3f, 0xa4, - 0x7b, 0xce, 0xa1, 0xc1, 0x44, 0xdd, 0xa8, 0xf8, 0xb2, 0xb5, 0x1b, 0x8f, 0x8f, 0x4b, 0xaf, 0x09, - 0xb3, 0x73, 0xc5, 0xe2, 0x89, 0x7d, 0x8e, 0x49, 0x74, 0x85, 0x40, 0x5f, 0xf2, 0x25, 0x92, 0x55, - 0x41, 0x65, 0xb5, 0x8e, 0x96, 0xa1, 0xd0, 0xee, 0xf4, 0x8d, 0x5e, 0xb7, 0xb1, 0xd5, 0xdc, 0x6e, - 0x36, 0xea, 0xfa, 0x02, 0xca, 0x83, 0xd6, 0x31, 0x70, 0xbd, 0xd3, 0x6e, 0xdd, 0xd7, 0x15, 0x31, - 0xbb, 0x87, 0xf9, 0x2c, 0x81, 0x00, 0xd2, 0x8c, 0x77, 0x0f, 0xeb, 0xaa, 0x34, 0xf4, 0x3b, 0x05, - 0x72, 0xdd, 0xc0, 0xb7, 0x48, 0x18, 0x72, 0x38, 0xaa, 0x40, 0xc2, 0xb1, 0x25, 0x16, 0xae, 0x4c, - 0x8b, 0x33, 0x26, 0x52, 0x69, 0xd6, 0x25, 0xba, 0x25, 0x1c, 0x1b, 0xad, 0x83, 0x46, 0x3c, 0x7b, - 0xe4, 0x3b, 0x1e, 0x15, 0x38, 0x5e, 0xcb, 0x7f, 0x7f, 0x5c, 0xd2, 0x1a, 0x92, 0x86, 0x27, 0xdc, - 0xe2, 0xbb, 0x90, 0x68, 0xd6, 0xd9, 0x45, 0xf0, 0xb9, 0xef, 0x4d, 0x2e, 0x02, 0x36, 0x46, 0x97, - 0x21, 0x1d, 0x8e, 0xf7, 0xf6, 0x9c, 0x43, 0x79, 0x13, 0xc8, 0x99, 0xf0, 0xf0, 0xa6, 0xfa, 0x25, - 0xf3, 0xf3, 0x97, 0x0a, 0x40, 0x8d, 0x5f, 0x56, 0xdc, 0xcd, 0x3e, 0xe4, 0x47, 0xc2, 0x25, 0x23, - 0x1c, 0x11, 0x4b, 0x3a, 0x7c, 0x69, 0xae, 0xc3, 0xb5, 0x62, 0x0c, 0xcf, 0x16, 0x65, 0xbd, 0x44, - 0x28, 0x96, 0x1b, 0xc5, 0x36, 0xff, 0x32, 0x14, 0x7e, 0x26, 0xd0, 0xc4, 0x70, 0x9d, 0xa1, 0x23, - 0x76, 0x54, 0xc0, 0x79, 0x49, 0x6c, 0x31, 0x5a, 0xf9, 0xef, 0x89, 0x18, 0x12, 0xbc, 0x0a, 0x19, - 0xc9, 0x94, 0x00, 0x9e, 0x8b, 0x63, 0x75, 0xc4, 0x43, 0x6b, 0x90, 0xda, 0x25, 0x03, 0x47, 0x00, - 0x75, 0xb2, 0x06, 0xdf, 0x1f, 0x97, 0xd2, 0x9d, 0xbd, 0xbd, 0x90, 0x50, 0x2c, 0x18, 0xe8, 0x45, - 0x48, 0x12, 0xcf, 0xe6, 0xa0, 0x7c, 0x92, 0xcf, 0xc8, 0xe8, 0x3a, 0x24, 0xc3, 0xf1, 0x50, 0x9e, - 0xc1, 0xe5, 0xe9, 0x2e, 0x7b, 0xb7, 0xab, 0x6f, 0xf7, 0xc6, 0x43, 0x99, 0x0f, 0x26, 0x83, 0x6e, - 0xcd, 0x03, 0x9b, 0xd4, 0x79, 0x60, 0x33, 0x07, 0x44, 0xde, 0x85, 0xc2, 0xae, 0x69, 0x1d, 0x38, - 0xde, 0xc0, 0xe0, 0xb0, 0xc0, 0x8f, 0x4d, 0xb6, 0xb6, 0x7c, 0x1a, 0x36, 0xf2, 0x52, 0x8e, 0xcf, - 0xd0, 0x55, 0xd0, 0x86, 0xbe, 0x6d, 0x50, 0x67, 0x28, 0x01, 0x17, 0x67, 0x86, 0xbe, 0xdd, 0x77, - 0x86, 0x04, 0xbd, 0x04, 0xf9, 0x78, 0xd1, 0xaf, 0x68, 0x3c, 0xdd, 0xb9, 0x58, 0x99, 0x97, 0x3f, - 0x82, 0x8c, 0xdc, 0x14, 0xeb, 0x0f, 0x46, 0x66, 0x40, 0xdf, 0xe6, 0x91, 0x4d, 0x63, 0x31, 0x89, - 0xa8, 0x9b, 0x3c, 0x94, 0x92, 0xba, 0x19, 0x51, 0xdf, 0xe1, 0x01, 0xcc, 0x08, 0xea, 0x3b, 0xe5, - 0x3f, 0x24, 0x20, 0x87, 0x89, 0x69, 0x63, 0xf2, 0xf3, 0x31, 0x09, 0x29, 0x5a, 0x87, 0xf4, 0x3e, - 0x31, 0x6d, 0x12, 0xc8, 0x7a, 0xd1, 0xa7, 0x01, 0xb9, 0xcd, 0xe9, 0x58, 0xf2, 0xe3, 0x79, 0x4d, - 0x3c, 0x21, 0xaf, 0x65, 0x48, 0xfb, 0x3c, 0x4d, 0x73, 0x12, 0x27, 0x39, 0xcc, 0xb5, 0x5d, 0xd7, - 0xb7, 0x0e, 0x78, 0xf6, 0x34, 0x2c, 0x26, 0x68, 0x0d, 0xf2, 0xb6, 0x6f, 0x78, 0x3e, 0x35, 0x46, - 0x81, 0x7f, 0x78, 0xc4, 0x33, 0xa4, 0x61, 0xb0, 0xfd, 0xb6, 0x4f, 0xbb, 0x8c, 0xc2, 0x8a, 0x71, - 0x48, 0xa8, 0x69, 0x9b, 0xd4, 0x34, 0x7c, 0xcf, 0x3d, 0xe2, 0xf1, 0xd7, 0x70, 0x3e, 0x22, 0x76, - 0x3c, 0xf7, 0x08, 0x5d, 0x07, 0x60, 0x97, 0x97, 0x74, 0x22, 0x73, 0xca, 0x89, 0x2c, 0xf1, 0x6c, - 0x31, 0x44, 0xaf, 0xc0, 0x22, 0x2f, 0x35, 0x63, 0x92, 0x1d, 0x8d, 0x67, 0x27, 0xcf, 0xa9, 0x77, - 0x45, 0x8a, 0xca, 0xbf, 0x4d, 0x40, 0x5e, 0x84, 0x2c, 0x1c, 0xf9, 0x5e, 0x48, 0x58, 0xcc, 0x42, - 0x6a, 0xd2, 0x71, 0xc8, 0x63, 0xb6, 0x18, 0x8f, 0x59, 0x8f, 0xd3, 0xb1, 0xe4, 0xc7, 0xa2, 0x9b, - 0x38, 0x27, 0xba, 0x4f, 0x13, 0xb6, 0xeb, 0x00, 0x9f, 0x05, 0x0e, 0x25, 0x06, 0xd3, 0xe1, 0xb1, - 0x9b, 0xd9, 0x19, 0xe7, 0x32, 0xc3, 0xa8, 0x12, 0xeb, 0x40, 0x52, 0xb3, 0x5d, 0x4d, 0x54, 0xaa, - 0xb1, 0xd6, 0xe2, 0x25, 0xc8, 0x47, 0x63, 0x63, 0x1c, 0x88, 0xfb, 0x20, 0x8b, 0x73, 0x11, 0x6d, - 0x27, 0x70, 0xd1, 0x0a, 0x64, 0x2c, 0xdf, 0x63, 0x57, 0x08, 0x0f, 0x6a, 0x1e, 0x47, 0xd3, 0xf2, - 0x97, 0x49, 0x28, 0xc8, 0xbe, 0xe0, 0xa2, 0xaa, 0x6a, 0xb6, 0x36, 0x92, 0xa7, 0x6a, 0x63, 0x1a, - 0xc0, 0xd4, 0x99, 0x01, 0xfc, 0x10, 0x96, 0xac, 0x7d, 0x62, 0x1d, 0x18, 0x01, 0x19, 0x38, 0x21, - 0x25, 0x41, 0x28, 0x2f, 0xbe, 0x2b, 0xa7, 0x5a, 0x3e, 0xd1, 0x0c, 0xe3, 0x45, 0x2e, 0x8f, 0x23, - 0x71, 0xf4, 0x23, 0x58, 0x1a, 0x7b, 0x0c, 0x44, 0xa6, 0x16, 0x32, 0x67, 0x35, 0x8d, 0x78, 0x91, - 0x8b, 0x4e, 0x95, 0xab, 0x80, 0xc2, 0xf1, 0x2e, 0x0d, 0x4c, 0x8b, 0xc6, 0xf4, 0xb5, 0x33, 0xf5, - 0x97, 0x23, 0xe9, 0xa9, 0x89, 0x58, 0x12, 0xd4, 0x13, 0x49, 0x90, 0x77, 0xd7, 0x6f, 0x12, 0xb0, - 0x18, 0xa5, 0xe2, 0x99, 0xab, 0xb5, 0x72, 0x5e, 0xb5, 0x4a, 0x50, 0x8d, 0x72, 0x77, 0x03, 0xd2, - 0x96, 0x3f, 0x64, 0x97, 0x42, 0xf2, 0xcc, 0x12, 0x93, 0x12, 0xe8, 0x2d, 0xd6, 0xca, 0x44, 0x5b, - 0x56, 0xcf, 0xdc, 0xf2, 0x54, 0x88, 0x95, 0x24, 0xf5, 0xa9, 0xe9, 0x1a, 0xd6, 0xfe, 0xd8, 0x3b, - 0x08, 0x45, 0x5a, 0x71, 0x8e, 0xd3, 0xb6, 0x38, 0x09, 0xbd, 0x0a, 0x8b, 0x36, 0x71, 0xcd, 0x23, - 0x62, 0x47, 0x42, 0x69, 0x2e, 0x54, 0x90, 0x54, 0x21, 0x56, 0xfe, 0x73, 0x02, 0x74, 0x2c, 0x1b, - 0x7e, 0xf2, 0xec, 0x25, 0x5a, 0x01, 0xf6, 0xe6, 0x1b, 0xf9, 0xa1, 0xe9, 0x3e, 0x61, 0xa3, 0x13, - 0x99, 0x93, 0x5b, 0xcd, 0x3c, 0xcd, 0x56, 0xd7, 0x20, 0x67, 0x5a, 0x07, 0x9e, 0xff, 0x99, 0x4b, - 0xec, 0x01, 0x91, 0xa8, 0x16, 0x27, 0xa1, 0x9b, 0x80, 0x6c, 0x32, 0x0a, 0x08, 0xdb, 0x81, 0x6d, - 0x3c, 0xe1, 0xc4, 0x2c, 0x4f, 0xc5, 0x24, 0xe9, 0xec, 0x9a, 0x61, 0x78, 0x2a, 0x87, 0x86, 0x4d, - 0x5c, 0x6a, 0xca, 0x18, 0xe7, 0x25, 0xb1, 0xce, 0x68, 0xe5, 0xbf, 0x2a, 0xb0, 0x1c, 0x8b, 0xde, - 0x05, 0x62, 0x60, 0x1c, 0xb4, 0x92, 0x4f, 0x01, 0x5a, 0xcf, 0x5c, 0x53, 0xe5, 0x3e, 0xe4, 0x5a, - 0x4e, 0x48, 0xa3, 0x1a, 0xf8, 0x21, 0x68, 0xa1, 0x3c, 0xe9, 0xb2, 0x0a, 0xce, 0x02, 0x02, 0x59, - 0xf9, 0x13, 0xf1, 0x3b, 0xaa, 0x96, 0xd0, 0x93, 0x77, 0x54, 0x2d, 0xa9, 0xab, 0xe5, 0xff, 0x24, - 0x20, 0x2f, 0xcc, 0x5e, 0xf8, 0x91, 0xfb, 0x10, 0x34, 0x99, 0x7c, 0xf1, 0x90, 0x39, 0xf1, 0xb2, - 0x8c, 0xfb, 0x10, 0x3d, 0x33, 0x23, 0xc7, 0x23, 0xad, 0xe2, 0x1f, 0x15, 0x88, 0x8a, 0x05, 0x6d, - 0x80, 0x3a, 0xbf, 0x55, 0x8c, 0x3d, 0x20, 0xa5, 0x01, 0x2e, 0xc8, 0xce, 0x24, 0xbb, 0x2a, 0x03, - 0xf2, 0xc0, 0x09, 0xa3, 0x47, 0x76, 0x12, 0xe7, 0x86, 0xbe, 0x8d, 0x25, 0x09, 0xbd, 0x0e, 0xa9, - 0xc0, 0x1f, 0x53, 0x22, 0x33, 0x18, 0xfb, 0x32, 0x81, 0x19, 0x59, 0x9a, 0x13, 0x32, 0xe8, 0xff, - 0x60, 0xc9, 0x0a, 0x88, 0x49, 0xc9, 0xd4, 0x24, 0xbf, 0xd6, 0xf0, 0xa2, 0x20, 0x47, 0x56, 0xef, - 0xa8, 0x9a, 0xaa, 0xa7, 0xca, 0xff, 0x50, 0x20, 0x5f, 0x1d, 0x8d, 0xdc, 0xa3, 0x28, 0x81, 0x1f, - 0x40, 0xc6, 0xda, 0x37, 0xbd, 0x01, 0x89, 0x3e, 0x84, 0x5c, 0x9b, 0x2e, 0x17, 0x17, 0xac, 0x6c, - 0x71, 0xa9, 0xe8, 0x13, 0x84, 0xd4, 0x29, 0x7e, 0xa5, 0x40, 0x5a, 0x70, 0x50, 0x05, 0x9e, 0x23, - 0x87, 0x23, 0x62, 0x51, 0xe3, 0xc4, 0x06, 0xf9, 0x63, 0x16, 0x2f, 0x0b, 0xd6, 0xdd, 0xd8, 0x36, - 0xdf, 0x84, 0xf4, 0x78, 0x14, 0x92, 0x80, 0xca, 0xc4, 0xcd, 0x0f, 0x1e, 0x96, 0x42, 0xe8, 0x65, - 0x48, 0xdb, 0xc4, 0x25, 0x32, 0x2c, 0x33, 0x67, 0x56, 0xb2, 0xca, 0x0e, 0xbf, 0x46, 0x99, 0xd3, - 0x17, 0x5d, 0x47, 0xe5, 0x7f, 0x26, 0x40, 0x8f, 0x4e, 0x54, 0x78, 0x61, 0xb7, 0xf6, 0xe9, 0xfe, - 0x2a, 0x79, 0xba, 0xbf, 0x62, 0x77, 0x3b, 0x6b, 0xd8, 0x26, 0x32, 0xa2, 0x02, 0x58, 0x13, 0x17, - 0x49, 0xbc, 0x06, 0x4b, 0x1e, 0x39, 0xa4, 0xc6, 0xc8, 0x1c, 0x10, 0x83, 0xfa, 0x07, 0xc4, 0x93, - 0x48, 0x55, 0x60, 0xe4, 0xae, 0x39, 0x20, 0x7d, 0x46, 0x44, 0xd7, 0x00, 0xb8, 0x88, 0x78, 0xa9, - 0x30, 0x18, 0x4d, 0xe1, 0x2c, 0xa3, 0xf0, 0x67, 0x0a, 0xba, 0x05, 0xf9, 0xd0, 0x19, 0x78, 0x26, - 0x1d, 0x07, 0xa4, 0xdf, 0x6f, 0x49, 0x6c, 0x7e, 0xc2, 0xa3, 0x57, 0x7b, 0x78, 0x5c, 0x52, 0xf8, - 0x8b, 0xf6, 0x84, 0xe2, 0xa9, 0x6e, 0x44, 0x9b, 0xed, 0x46, 0xca, 0x7f, 0x4a, 0xc0, 0x72, 0x2c, - 0xbe, 0x17, 0x8e, 0x0b, 0x4d, 0xc8, 0x46, 0xb0, 0x18, 0x01, 0xc3, 0xab, 0xa7, 0xb1, 0x73, 0xe2, - 0x49, 0xc5, 0x98, 0x7c, 0x79, 0x12, 0x76, 0xa6, 0xda, 0xf3, 0x82, 0xad, 0xce, 0x09, 0x76, 0xf1, - 0x13, 0xc8, 0x4e, 0xac, 0xa0, 0x37, 0x4e, 0x20, 0xc9, 0x1c, 0xd8, 0x3e, 0x01, 0x23, 0xd7, 0x00, - 0x58, 0x3c, 0x89, 0xcd, 0x7b, 0x4d, 0xf1, 0xc2, 0xcd, 0x0a, 0xca, 0x4e, 0xe0, 0x96, 0x7f, 0xa5, - 0x40, 0x8a, 0x83, 0x05, 0x7a, 0x1f, 0x32, 0x43, 0x32, 0xdc, 0x65, 0xf8, 0x2e, 0xce, 0xf7, 0x79, - 0xef, 0xef, 0x48, 0x9c, 0x5d, 0x7a, 0xa3, 0xc0, 0x19, 0x9a, 0xc1, 0x91, 0xf8, 0x12, 0x88, 0xa3, - 0x29, 0xba, 0x01, 0xd9, 0xe8, 0x01, 0x1e, 0x7d, 0x0c, 0x3a, 0xf9, 0x3e, 0x9f, 0xb2, 0x65, 0x53, - 0xf5, 0xfb, 0x04, 0xa4, 0x45, 0xd4, 0xd1, 0x07, 0x00, 0xd1, 0x23, 0xfb, 0xa9, 0xbf, 0x09, 0x64, - 0xa5, 0x46, 0xd3, 0x9e, 0x82, 0x63, 0xe2, 0x29, 0xc0, 0x71, 0x03, 0x54, 0x42, 0x2d, 0x5b, 0x02, - 0xe9, 0xa5, 0xd9, 0x0a, 0xa8, 0x34, 0xa8, 0x65, 0x47, 0x61, 0x65, 0x82, 0xc5, 0x2f, 0x14, 0x50, - 0x19, 0x91, 0xc5, 0xd7, 0x72, 0xc7, 0xec, 0xca, 0x8b, 0xbc, 0x54, 0x71, 0x56, 0x52, 0x9a, 0x36, - 0x7a, 0x01, 0xb2, 0x22, 0x4c, 0x8c, 0x9b, 0xe0, 0x5c, 0x4d, 0x10, 0x9a, 0x36, 0x2a, 0x82, 0x36, - 0x41, 0x3f, 0x71, 0x5a, 0x27, 0x73, 0xa6, 0x18, 0x98, 0x7b, 0xd4, 0xa0, 0x24, 0x10, 0x2f, 0x6f, - 0x15, 0x6b, 0x8c, 0xd0, 0x27, 0xc1, 0x30, 0xfa, 0x34, 0xc1, 0x7e, 0x6f, 0x7c, 0x97, 0x80, 0xb4, - 0xa8, 0x68, 0x94, 0x86, 0x44, 0xe7, 0x23, 0x7d, 0x01, 0x5d, 0x82, 0xe5, 0x3b, 0x9d, 0x1d, 0xdc, - 0xae, 0xb6, 0x8c, 0x76, 0xa7, 0x6f, 0x6c, 0x77, 0x76, 0xda, 0x75, 0x5d, 0x41, 0xd7, 0xe0, 0x6a, - 0xbb, 0x63, 0x44, 0x9c, 0x2e, 0x6e, 0xde, 0xad, 0xe2, 0xfb, 0x46, 0x0d, 0x77, 0x3e, 0x6a, 0x60, - 0x3d, 0x81, 0x56, 0xa1, 0xc8, 0xa4, 0xcf, 0xe0, 0x27, 0xd1, 0x65, 0x40, 0x71, 0xbe, 0xa4, 0xa7, - 0xd0, 0x1a, 0xbc, 0xd8, 0x6c, 0xf7, 0x76, 0xb6, 0xb7, 0x9b, 0x5b, 0xcd, 0x46, 0x7b, 0x56, 0xa0, - 0xa7, 0xab, 0xe8, 0x45, 0x58, 0xe9, 0x6c, 0x6f, 0xf7, 0x1a, 0x7d, 0xee, 0xce, 0xfd, 0x46, 0xdf, - 0xa8, 0x7e, 0x5c, 0x6d, 0xb6, 0xaa, 0xb5, 0x56, 0x43, 0x4f, 0xa3, 0x25, 0xc8, 0xdd, 0xc3, 0x9d, - 0xf6, 0x2d, 0x03, 0x77, 0x76, 0xfa, 0x0d, 0x3d, 0xc3, 0xdc, 0xef, 0xe2, 0x4e, 0xb7, 0xd3, 0xab, - 0xb6, 0x8c, 0xbb, 0xcd, 0xde, 0xdd, 0x6a, 0x7f, 0xeb, 0xb6, 0xae, 0xa1, 0x17, 0xe0, 0x4a, 0xa3, - 0xbf, 0x55, 0x37, 0xfa, 0xb8, 0xda, 0xee, 0x55, 0xb7, 0xfa, 0xcd, 0x4e, 0xdb, 0xd8, 0xae, 0x36, - 0x5b, 0x8d, 0xba, 0x9e, 0x65, 0x46, 0x98, 0xed, 0x6a, 0xab, 0xd5, 0xb9, 0xd7, 0xa8, 0xeb, 0x80, - 0xae, 0xc0, 0x73, 0xc2, 0x6a, 0xb5, 0xdb, 0x6d, 0xb4, 0xeb, 0x86, 0x70, 0x40, 0xcf, 0x31, 0x67, - 0x9a, 0xed, 0x7a, 0xe3, 0x13, 0xe3, 0x76, 0xb5, 0x67, 0xdc, 0xc2, 0x8d, 0x6a, 0xbf, 0x81, 0x23, - 0x6e, 0x9e, 0xad, 0x8d, 0x1b, 0xb7, 0x9a, 0x3d, 0x46, 0x9c, 0xac, 0x5d, 0xb8, 0xe1, 0x81, 0x3e, - 0xfb, 0xcd, 0x02, 0xe5, 0x20, 0xd3, 0x6c, 0x7f, 0x5c, 0x6d, 0x35, 0xeb, 0xfa, 0x02, 0xd2, 0x40, - 0x6d, 0x77, 0xda, 0x0d, 0x5d, 0x61, 0xa3, 0x5b, 0x9f, 0x36, 0xbb, 0x7a, 0x02, 0x15, 0x20, 0xfb, - 0x69, 0xaf, 0x5f, 0x6d, 0xd7, 0xab, 0xb8, 0xae, 0x27, 0x11, 0x40, 0xba, 0xd7, 0xae, 0x76, 0xbb, - 0xf7, 0x75, 0x95, 0xc5, 0x9a, 0x09, 0xb1, 0x75, 0x5b, 0x9d, 0x6a, 0xdd, 0xa8, 0x37, 0xb6, 0x3a, - 0x77, 0xbb, 0xb8, 0xd1, 0xeb, 0x35, 0x3b, 0x6d, 0x3d, 0xb5, 0xf9, 0x8b, 0xe4, 0xb4, 0x73, 0xf8, - 0x01, 0xa8, 0xac, 0xdb, 0x40, 0x97, 0x66, 0xbb, 0x0f, 0x7e, 0x93, 0x14, 0x2f, 0xcf, 0x6f, 0x4a, - 0xd0, 0xfb, 0x90, 0xe2, 0x37, 0x1c, 0xba, 0x3c, 0xff, 0x9e, 0x2e, 0x5e, 0x39, 0x45, 0x97, 0x9a, - 0xef, 0x81, 0xca, 0xde, 0xe0, 0xf1, 0x05, 0x63, 0x9f, 0x31, 0xe2, 0x0b, 0xc6, 0x9f, 0xea, 0x6f, - 0x29, 0xe8, 0x03, 0x48, 0x8b, 0x07, 0x11, 0x3a, 0x69, 0x7b, 0xfa, 0x5a, 0x2d, 0xae, 0x9c, 0x66, - 0x08, 0xf5, 0x75, 0x05, 0xdd, 0x86, 0xec, 0xa4, 0xf9, 0x45, 0xc5, 0xf8, 0x2a, 0x27, 0xdf, 0x13, - 0xc5, 0x17, 0xe6, 0xf2, 0x22, 0x3b, 0x6f, 0x31, 0x4b, 0x05, 0x16, 0x8b, 0x09, 0x16, 0xc7, 0xad, - 0xcd, 0x5e, 0xc5, 0x71, 0x6b, 0xa7, 0xc0, 0xbb, 0xd6, 0x78, 0xf8, 0xef, 0xd5, 0x85, 0x87, 0xdf, - 0xad, 0x2a, 0xdf, 0x7e, 0xb7, 0xaa, 0xfc, 0xfa, 0xd1, 0xea, 0xc2, 0x37, 0x8f, 0x56, 0x95, 0xbf, - 0x3c, 0x5a, 0x55, 0xbe, 0x7d, 0xb4, 0xba, 0xf0, 0xb7, 0x47, 0xab, 0x0b, 0x9f, 0xbe, 0x3c, 0xf0, - 0x2b, 0x03, 0xf3, 0x73, 0x42, 0x29, 0xa9, 0xd8, 0xe4, 0xc1, 0x86, 0xe5, 0x07, 0x64, 0x63, 0xe6, - 0x8f, 0xad, 0xdd, 0x34, 0x1f, 0xbd, 0xf3, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x49, 0xe2, 0x70, - 0x4f, 0xf2, 0x1a, 0x00, 0x00, + // 2612 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4d, 0x70, 0xdb, 0xc6, + 0xf5, 0x17, 0x48, 0x90, 0x04, 0x1f, 0x49, 0x09, 0xda, 0xc4, 0x36, 0xcd, 0xc4, 0xa2, 0xc2, 0x7c, + 0xfc, 0x65, 0x27, 0xa1, 0x12, 0xe5, 0xdf, 0x24, 0x75, 0x27, 0x6d, 0x48, 0x91, 0xb2, 0xe9, 0xd0, + 0x24, 0x67, 0x49, 0xc5, 0x71, 0x0e, 0xc5, 0x40, 0xc0, 0x8a, 0x42, 0x05, 0x02, 0x28, 0x00, 0x3a, + 0x52, 0x6e, 0xb9, 0xb4, 0x39, 0xa4, 0x33, 0x9d, 0x9e, 0x72, 0xea, 0xe4, 0xd2, 0x73, 0x3b, 0xd3, + 0x5b, 0x7b, 0xe9, 0xd1, 0xbd, 0xe5, 0xd8, 0x99, 0xb6, 0xea, 0x34, 0x9e, 0xe9, 0xf4, 0xec, 0x63, + 0x4e, 0x9d, 0xfd, 0x00, 0x09, 0x51, 0xa4, 0x65, 0x1f, 0x74, 0xe1, 0x60, 0xdf, 0xd7, 0xbe, 0x7d, + 0xef, 0xe1, 0xb7, 0xef, 0x81, 0xb0, 0xb6, 0xe7, 0xbb, 0x87, 0xc4, 0xdf, 0xf4, 0x7c, 0x37, 0x74, + 0x0d, 0xd7, 0x9e, 0x3c, 0x54, 0xd9, 0x03, 0x52, 0xa2, 0x75, 0xe9, 0xf9, 0xa1, 0x3b, 0x74, 0xd9, + 0x6a, 0x93, 0x3e, 0x71, 0x7e, 0x69, 0x6d, 0xe8, 0xba, 0x43, 0x9b, 0x70, 0xb5, 0xbd, 0xf1, 0xfe, + 0xa6, 0x39, 0xf6, 0xf5, 0xd0, 0x72, 0x1d, 0xce, 0xaf, 0xec, 0x43, 0xaa, 0xad, 0xef, 0x11, 0x1b, + 0x21, 0x90, 0x1d, 0x7d, 0x44, 0x8a, 0xd2, 0xba, 0xb4, 0x91, 0xc5, 0xec, 0x19, 0x3d, 0x0f, 0xa9, + 0x07, 0xba, 0x3d, 0x26, 0xc5, 0x04, 0x23, 0xf2, 0x05, 0x7a, 0x13, 0xd2, 0x9e, 0x4f, 0xf6, 0xad, + 0xa3, 0x62, 0x72, 0x5d, 0xda, 0x50, 0xea, 0x97, 0x1e, 0x9f, 0x94, 0x57, 0x8f, 0xf5, 0x91, 0x7d, + 0xb3, 0xf2, 0x86, 0x3b, 0xb2, 0x42, 0x32, 0xf2, 0xc2, 0xe3, 0x0a, 0x16, 0x42, 0x37, 0xe5, 0xff, + 0x7e, 0x53, 0x96, 0x2a, 0x03, 0x50, 0xd8, 0x3e, 0x7d, 0x12, 0xa2, 0x3a, 0xa4, 0x6d, 0xfa, 0x1c, + 0x14, 0xa5, 0xf5, 0xe4, 0x46, 0x6e, 0x6b, 0xa5, 0x3a, 0x39, 0x14, 0x93, 0xa9, 0x5f, 0x7d, 0x78, + 0x52, 0x5e, 0x5a, 0x60, 0x95, 0x6b, 0x0a, 0xab, 0x5f, 0x48, 0x50, 0x10, 0x66, 0x6d, 0x62, 0x84, + 0xae, 0x8f, 0xb6, 0x20, 0x63, 0x39, 0x86, 0x3d, 0x36, 0xf9, 0x49, 0x72, 0x5b, 0x68, 0xc6, 0x78, + 0x9f, 0x84, 0x75, 0x99, 0xda, 0xc7, 0x91, 0x20, 0xd5, 0x21, 0x47, 0x5c, 0x27, 0x71, 0x9e, 0x8e, + 0x10, 0xbc, 0xa9, 0x7c, 0xfd, 0x4d, 0x79, 0x89, 0xf9, 0xf0, 0x55, 0x16, 0x72, 0x77, 0xdc, 0xb1, + 0xef, 0xe8, 0x76, 0xdf, 0x23, 0x06, 0xfa, 0xff, 0x78, 0x20, 0xeb, 0xeb, 0x73, 0x8f, 0xf1, 0xfd, + 0x49, 0x39, 0x23, 0x74, 0x44, 0xa8, 0xdf, 0x83, 0x9c, 0x4f, 0x3c, 0xdb, 0x32, 0x58, 0x72, 0x98, + 0x1f, 0xa9, 0x45, 0x91, 0x8d, 0x4b, 0xa2, 0xde, 0x24, 0x98, 0xc9, 0x85, 0xbe, 0xbf, 0x42, 0x7d, + 0xff, 0xf6, 0xa4, 0x2c, 0x3d, 0x3e, 0x29, 0x17, 0x67, 0xed, 0xbd, 0x61, 0x39, 0xb6, 0xe5, 0x90, + 0x49, 0x68, 0xd1, 0x2e, 0x28, 0xfb, 0xbe, 0x3e, 0x1c, 0x11, 0x27, 0x2c, 0xca, 0xcc, 0xe6, 0xda, + 0xd4, 0x66, 0xec, 0xa4, 0xd5, 0x1d, 0x21, 0xf5, 0xa4, 0x7c, 0x4d, 0x4c, 0xa1, 0x9f, 0x40, 0x6a, + 0xdf, 0xd6, 0x87, 0x41, 0x31, 0xbd, 0x2e, 0x6d, 0x14, 0xea, 0xd7, 0x17, 0x05, 0x46, 0x8d, 0x6d, + 0xa1, 0xed, 0xd8, 0xfa, 0x10, 0x73, 0x3d, 0xd4, 0x86, 0x95, 0x91, 0x7e, 0xa4, 0xe9, 0x9e, 0x47, + 0x1c, 0x53, 0xf3, 0xf5, 0x90, 0x14, 0x33, 0xeb, 0xd2, 0x46, 0xb2, 0xfe, 0xca, 0xe3, 0x93, 0xf2, + 0x3a, 0x37, 0x35, 0x23, 0x10, 0xf7, 0xa4, 0x30, 0xd2, 0x8f, 0x6a, 0x8c, 0x85, 0xf5, 0x90, 0x94, + 0xbe, 0x4a, 0x81, 0x12, 0x1d, 0x80, 0x96, 0xb4, 0x4d, 0x9c, 0x61, 0x78, 0xc0, 0xb2, 0x96, 0x5c, + 0x58, 0xd2, 0x5c, 0x08, 0xb9, 0xb0, 0x6a, 0xb8, 0x23, 0xcf, 0x27, 0x41, 0x60, 0xb9, 0x8e, 0x66, + 0xb8, 0x26, 0x31, 0x58, 0xca, 0x96, 0xb7, 0x4a, 0xd3, 0x50, 0x6d, 0x4f, 0x45, 0xb6, 0xa9, 0x44, + 0xfd, 0xb5, 0xc7, 0x27, 0xe5, 0x0a, 0xb7, 0x7a, 0x46, 0x3d, 0xbe, 0x8d, 0x6a, 0xcc, 0x68, 0xa2, + 0x1f, 0x43, 0x3a, 0x08, 0x5d, 0x9f, 0xd0, 0x24, 0x27, 0x37, 0xb2, 0xcc, 0xd2, 0xdc, 0xe0, 0x15, + 0xa2, 0x23, 0xf5, 0xa9, 0x38, 0x16, 0x5a, 0x28, 0x00, 0xd5, 0x27, 0xfb, 0x3e, 0x09, 0x0e, 0x34, + 0xcb, 0x09, 0x89, 0xff, 0x40, 0xb7, 0x45, 0x6a, 0xaf, 0x56, 0x39, 0x40, 0x54, 0x23, 0x80, 0xa8, + 0x36, 0x04, 0x40, 0xd4, 0xdf, 0x14, 0x59, 0x7d, 0x89, 0x6f, 0x34, 0x6b, 0x20, 0xb6, 0xf1, 0xd7, + 0xff, 0x2a, 0x4b, 0x78, 0x45, 0x08, 0xb4, 0x04, 0x1f, 0x7d, 0x0c, 0x59, 0x9f, 0x84, 0xc4, 0x61, + 0x05, 0x9d, 0x3a, 0x6f, 0xb7, 0x6b, 0x0b, 0x6b, 0x88, 0x59, 0x9f, 0x9a, 0x42, 0x23, 0x58, 0xde, + 0xb7, 0xc7, 0xf1, 0xa3, 0xa4, 0xcf, 0x33, 0xfe, 0xba, 0x30, 0x5e, 0xe6, 0xc6, 0x4f, 0xab, 0xcf, + 0x6e, 0x55, 0x60, 0xec, 0xc9, 0x31, 0x7e, 0x0a, 0x97, 0x3c, 0x3d, 0x3c, 0xd0, 0x3c, 0x37, 0x08, + 0xf7, 0xad, 0x23, 0x8d, 0x8a, 0xda, 0x51, 0xf1, 0x65, 0xeb, 0x37, 0x1e, 0x9f, 0x94, 0x5f, 0xe3, + 0x66, 0xe7, 0x8a, 0xc5, 0x13, 0xfb, 0x1c, 0x95, 0xe8, 0x71, 0x81, 0x81, 0xe0, 0x0b, 0x24, 0xab, + 0x81, 0x4c, 0x6b, 0x1d, 0xad, 0x42, 0xa1, 0xd3, 0x1d, 0x68, 0xfd, 0x5e, 0x73, 0xbb, 0xb5, 0xd3, + 0x6a, 0x36, 0xd4, 0x25, 0x94, 0x07, 0xa5, 0xab, 0xe1, 0x46, 0xb7, 0xd3, 0xbe, 0xaf, 0x4a, 0x7c, + 0x75, 0x0f, 0xb3, 0x55, 0x02, 0x01, 0xa4, 0x29, 0xef, 0x1e, 0x56, 0x65, 0x61, 0xe8, 0x77, 0x12, + 0xe4, 0x7a, 0xbe, 0x6b, 0x90, 0x20, 0x60, 0x70, 0x54, 0x85, 0x84, 0x65, 0x0a, 0x2c, 0x2c, 0x4e, + 0x8b, 0x33, 0x26, 0x52, 0x6d, 0x35, 0x04, 0xba, 0x25, 0x2c, 0x13, 0x6d, 0x80, 0x42, 0x1c, 0xd3, + 0x73, 0x2d, 0x27, 0xe4, 0xb0, 0x5f, 0xcf, 0x7f, 0x7f, 0x52, 0x56, 0x9a, 0x82, 0x86, 0x27, 0xdc, + 0xd2, 0xbb, 0x90, 0x68, 0x35, 0xe8, 0xbd, 0xf1, 0xb9, 0xeb, 0x4c, 0xee, 0x0d, 0xfa, 0x8c, 0x2e, + 0x43, 0x3a, 0x18, 0xef, 0xd3, 0x1b, 0x82, 0x5f, 0x1c, 0x62, 0xc5, 0x3d, 0xbc, 0x29, 0x7f, 0x49, + 0xfd, 0xfc, 0xa5, 0x04, 0x50, 0x67, 0x77, 0x1b, 0x73, 0x73, 0x00, 0x79, 0x8f, 0xbb, 0xa4, 0x05, + 0x1e, 0x31, 0x84, 0xc3, 0x97, 0xe6, 0x3a, 0x5c, 0x2f, 0xc5, 0xf0, 0x6c, 0x59, 0xd4, 0x4b, 0x84, + 0x62, 0x39, 0x2f, 0x76, 0xf8, 0x97, 0xa1, 0xf0, 0x33, 0x8e, 0x26, 0x9a, 0x6d, 0x8d, 0x2c, 0x7e, + 0xa2, 0x02, 0xce, 0x0b, 0x62, 0x9b, 0xd2, 0x2a, 0x7f, 0x4f, 0xc4, 0x90, 0xe0, 0x55, 0xc8, 0x08, + 0xa6, 0x00, 0xf0, 0x5c, 0x1c, 0xab, 0x23, 0x1e, 0x5a, 0x87, 0xd4, 0x1e, 0x19, 0x5a, 0x1c, 0xa8, + 0x93, 0x75, 0xf8, 0xfe, 0xa4, 0x9c, 0xee, 0xee, 0xef, 0x07, 0x24, 0xc4, 0x9c, 0x81, 0x5e, 0x84, + 0x24, 0x71, 0x4c, 0x06, 0xca, 0xa7, 0xf9, 0x94, 0x8c, 0xae, 0x43, 0x32, 0x18, 0x8f, 0xc4, 0x3b, + 0xb8, 0x3a, 0x3d, 0x65, 0xff, 0x76, 0xed, 0xed, 0xfe, 0x78, 0x24, 0xf2, 0x41, 0x65, 0xd0, 0xad, + 0x79, 0x60, 0x93, 0x3a, 0x0f, 0x6c, 0xe6, 0x80, 0xc8, 0xbb, 0x50, 0xd8, 0xd3, 0x8d, 0x43, 0xcb, + 0x19, 0x6a, 0x0c, 0x16, 0xd8, 0x6b, 0x93, 0xad, 0xaf, 0x9e, 0x85, 0x8d, 0xbc, 0x90, 0x63, 0x2b, + 0x74, 0x15, 0x94, 0x91, 0x6b, 0x6a, 0xa1, 0x35, 0x12, 0x80, 0x8b, 0x33, 0x23, 0xd7, 0x1c, 0x58, + 0x23, 0x82, 0x5e, 0x82, 0x7c, 0xbc, 0xe8, 0x8b, 0x0a, 0x4b, 0x77, 0x2e, 0x56, 0xe6, 0x95, 0x8f, + 0x20, 0x23, 0x0e, 0x45, 0xdb, 0x09, 0x4f, 0xf7, 0xc3, 0xb7, 0x59, 0x64, 0xd3, 0x98, 0x2f, 0x22, + 0xea, 0x16, 0x0b, 0xa5, 0xa0, 0x6e, 0x45, 0xd4, 0x77, 0x58, 0x00, 0x33, 0x9c, 0xfa, 0x4e, 0xe5, + 0x0f, 0x09, 0xc8, 0x61, 0xa2, 0x9b, 0x98, 0xfc, 0x7c, 0x4c, 0x82, 0x10, 0x6d, 0x40, 0xfa, 0x80, + 0xe8, 0x26, 0xf1, 0x45, 0xbd, 0xa8, 0xd3, 0x80, 0xdc, 0x66, 0x74, 0x2c, 0xf8, 0xf1, 0xbc, 0x26, + 0x9e, 0x90, 0xd7, 0x0a, 0xa4, 0x5d, 0x96, 0xa6, 0x39, 0x89, 0x13, 0x1c, 0xea, 0xda, 0x9e, 0xed, + 0x1a, 0x87, 0x2c, 0x7b, 0x0a, 0xe6, 0x0b, 0xb4, 0x0e, 0x79, 0xd3, 0xd5, 0x1c, 0x37, 0xd4, 0x3c, + 0xdf, 0x3d, 0x3a, 0x66, 0x19, 0x52, 0x30, 0x98, 0x6e, 0xc7, 0x0d, 0x7b, 0x94, 0x42, 0x8b, 0x71, + 0x44, 0x42, 0xdd, 0xd4, 0x43, 0x5d, 0x73, 0x1d, 0xfb, 0x98, 0xc5, 0x5f, 0xc1, 0xf9, 0x88, 0xd8, + 0x75, 0xec, 0x63, 0x74, 0x1d, 0x80, 0x5e, 0x5e, 0xc2, 0x89, 0xcc, 0x19, 0x27, 0xb2, 0xc4, 0x31, + 0xf9, 0x23, 0x7a, 0x05, 0x96, 0x59, 0xa9, 0x69, 0x93, 0xec, 0x28, 0x2c, 0x3b, 0x79, 0x46, 0xbd, + 0xcb, 0x53, 0x54, 0xf9, 0x6d, 0x02, 0xf2, 0x3c, 0x64, 0x81, 0xe7, 0x3a, 0x01, 0xa1, 0x31, 0x0b, + 0x42, 0x3d, 0x1c, 0x07, 0x2c, 0x66, 0xcb, 0xf1, 0x98, 0xf5, 0x19, 0x1d, 0x0b, 0x7e, 0x2c, 0xba, + 0x89, 0x73, 0xa2, 0xfb, 0x34, 0x61, 0xbb, 0x0e, 0xf0, 0x99, 0x6f, 0x85, 0x44, 0xa3, 0x3a, 0x2c, + 0x76, 0x33, 0x27, 0x63, 0x5c, 0x6a, 0x18, 0x55, 0x63, 0x1d, 0x48, 0x6a, 0xb6, 0xab, 0x89, 0x4a, + 0x35, 0xd6, 0x5a, 0xbc, 0x04, 0xf9, 0xe8, 0x59, 0x1b, 0xfb, 0xfc, 0x3e, 0xc8, 0xe2, 0x5c, 0x44, + 0xdb, 0xf5, 0x6d, 0x54, 0x84, 0x8c, 0xe1, 0x3a, 0xf4, 0x0a, 0x61, 0x41, 0xcd, 0xe3, 0x68, 0x59, + 0xf9, 0x32, 0x09, 0x05, 0xd1, 0x17, 0x5c, 0x54, 0x55, 0xcd, 0xd6, 0x46, 0xf2, 0x4c, 0x6d, 0x4c, + 0x03, 0x98, 0x5a, 0x18, 0xc0, 0x0f, 0x61, 0xc5, 0x38, 0x20, 0xc6, 0xa1, 0xe6, 0x93, 0xa1, 0x15, + 0x84, 0xc4, 0x0f, 0xc4, 0xc5, 0x77, 0xe5, 0x4c, 0xcb, 0xc7, 0x9b, 0x61, 0xbc, 0xcc, 0xe4, 0x71, + 0x24, 0x8e, 0x7e, 0x04, 0x2b, 0x63, 0x87, 0x82, 0xc8, 0xd4, 0x42, 0x66, 0x51, 0xd3, 0x88, 0x97, + 0x99, 0xe8, 0x54, 0xb9, 0x06, 0x28, 0x18, 0xef, 0x85, 0xbe, 0x6e, 0x84, 0x31, 0x7d, 0x65, 0xa1, + 0xfe, 0x6a, 0x24, 0x3d, 0x35, 0x11, 0x4b, 0x82, 0x7c, 0x2a, 0x09, 0xe2, 0xee, 0xfa, 0x4d, 0x02, + 0x96, 0xa3, 0x54, 0x3c, 0x73, 0xb5, 0x56, 0xcf, 0xab, 0x56, 0x01, 0xaa, 0x51, 0xee, 0x6e, 0x40, + 0xda, 0x70, 0x47, 0xf4, 0x52, 0x48, 0x2e, 0x2c, 0x31, 0x21, 0x81, 0xde, 0xa2, 0xad, 0x4c, 0x74, + 0x64, 0x79, 0xe1, 0x91, 0xa7, 0x42, 0xb4, 0x24, 0x43, 0x37, 0xd4, 0x6d, 0xcd, 0x38, 0x18, 0x3b, + 0x87, 0x01, 0x4f, 0x2b, 0xce, 0x31, 0xda, 0x36, 0x23, 0xa1, 0x57, 0x61, 0xd9, 0x24, 0xb6, 0x7e, + 0x4c, 0xcc, 0x48, 0x28, 0xcd, 0x84, 0x0a, 0x82, 0xca, 0xc5, 0x2a, 0x7f, 0x4e, 0x80, 0x8a, 0x45, + 0xc3, 0x4f, 0x9e, 0xbd, 0x44, 0xab, 0x40, 0x47, 0x44, 0xcf, 0x0d, 0x74, 0xfb, 0x09, 0x07, 0x9d, + 0xc8, 0x9c, 0x3e, 0x6a, 0xe6, 0x69, 0x8e, 0xba, 0x0e, 0x39, 0xdd, 0x38, 0x74, 0xdc, 0xcf, 0x6c, + 0x62, 0x0e, 0x89, 0x40, 0xb5, 0x38, 0x09, 0xdd, 0x04, 0x64, 0x12, 0xcf, 0x27, 0xf4, 0x04, 0xa6, + 0xf6, 0x84, 0x37, 0x66, 0x75, 0x2a, 0x26, 0x48, 0x8b, 0x6b, 0x86, 0xe2, 0xa9, 0x78, 0xd4, 0x4c, + 0x62, 0x87, 0xba, 0x88, 0x71, 0x5e, 0x10, 0x1b, 0x94, 0x56, 0xf9, 0xab, 0x04, 0xab, 0xb1, 0xe8, + 0x5d, 0x20, 0x06, 0xc6, 0x41, 0x2b, 0xf9, 0x14, 0xa0, 0xf5, 0xcc, 0x35, 0x55, 0x19, 0x40, 0xae, + 0x6d, 0x05, 0x61, 0x54, 0x03, 0x3f, 0x04, 0x25, 0x10, 0x6f, 0xba, 0xa8, 0x82, 0x45, 0x40, 0x20, + 0x2a, 0x7f, 0x22, 0x7e, 0x47, 0x56, 0x12, 0x6a, 0xf2, 0x8e, 0xac, 0x24, 0x55, 0xb9, 0xf2, 0x9f, + 0x04, 0xe4, 0xb9, 0xd9, 0x0b, 0x7f, 0xe5, 0x3e, 0x04, 0x45, 0x24, 0x9f, 0x0f, 0x32, 0xa7, 0x26, + 0xcb, 0xb8, 0x0f, 0xd1, 0x98, 0x19, 0x39, 0x1e, 0x69, 0x95, 0xfe, 0x28, 0x41, 0x54, 0x2c, 0x68, + 0x13, 0xe4, 0xf9, 0xad, 0x62, 0x6c, 0x80, 0x14, 0x06, 0x98, 0x20, 0x7d, 0x27, 0xe9, 0x55, 0xe9, + 0x93, 0x07, 0x56, 0x10, 0x0d, 0xd9, 0x49, 0x9c, 0x1b, 0xb9, 0x26, 0x16, 0x24, 0xf4, 0x3a, 0xa4, + 0x7c, 0x77, 0x1c, 0x12, 0x91, 0xc1, 0xd8, 0x97, 0x09, 0x4c, 0xc9, 0xc2, 0x1c, 0x97, 0x41, 0xff, + 0x07, 0x2b, 0x86, 0x4f, 0xf4, 0x90, 0x4c, 0x4d, 0xb2, 0x6b, 0x0d, 0x2f, 0x73, 0x72, 0x64, 0xf5, + 0x8e, 0xac, 0xc8, 0x6a, 0xaa, 0xf2, 0x0f, 0x09, 0xf2, 0x35, 0xcf, 0xb3, 0x8f, 0xa3, 0x04, 0x7e, + 0x00, 0x19, 0xe3, 0x40, 0x77, 0x86, 0x24, 0xfa, 0x10, 0x72, 0x6d, 0xba, 0x5d, 0x5c, 0xb0, 0xba, + 0xcd, 0xa4, 0xa2, 0x4f, 0x10, 0x42, 0xa7, 0xf4, 0x95, 0x04, 0x69, 0xce, 0x41, 0x55, 0x78, 0x8e, + 0x1c, 0x79, 0xc4, 0x08, 0xb5, 0x53, 0x07, 0x64, 0xc3, 0x2c, 0x5e, 0xe5, 0xac, 0xbb, 0xb1, 0x63, + 0xbe, 0x09, 0xe9, 0xb1, 0x17, 0x10, 0x3f, 0x14, 0x89, 0x9b, 0x1f, 0x3c, 0x2c, 0x84, 0xd0, 0xcb, + 0x90, 0x36, 0x89, 0x4d, 0x44, 0x58, 0x66, 0xde, 0x59, 0xc1, 0xaa, 0x58, 0xec, 0x1a, 0xa5, 0x4e, + 0x5f, 0x74, 0x1d, 0x55, 0xfe, 0x99, 0x00, 0x35, 0x7a, 0xa3, 0x82, 0x0b, 0xbb, 0xb5, 0xcf, 0xf6, + 0x57, 0xc9, 0xb3, 0xfd, 0x15, 0xbd, 0xdb, 0x69, 0xc3, 0x36, 0x91, 0xe1, 0x15, 0x40, 0x9b, 0xb8, + 0x48, 0xe2, 0x35, 0x58, 0x71, 0xc8, 0x51, 0xa8, 0x79, 0xfa, 0x90, 0x68, 0xa1, 0x7b, 0x48, 0x1c, + 0x81, 0x54, 0x05, 0x4a, 0xee, 0xe9, 0x43, 0x32, 0xa0, 0x44, 0x74, 0x0d, 0x80, 0x89, 0xf0, 0x49, + 0x85, 0xc2, 0x68, 0x0a, 0x67, 0x29, 0x85, 0x8d, 0x29, 0xe8, 0x16, 0xe4, 0x03, 0x6b, 0xe8, 0xe8, + 0xe1, 0xd8, 0x27, 0x83, 0x41, 0x5b, 0x60, 0xf3, 0x13, 0x86, 0x5e, 0xe5, 0xe1, 0x49, 0x59, 0x62, + 0x13, 0xed, 0x29, 0xc5, 0x33, 0xdd, 0x88, 0x32, 0xdb, 0x8d, 0x54, 0xfe, 0x94, 0x80, 0xd5, 0x58, + 0x7c, 0x2f, 0x1c, 0x17, 0x5a, 0x90, 0x8d, 0x60, 0x31, 0x02, 0x86, 0x57, 0xcf, 0x62, 0xe7, 0xc4, + 0x93, 0xaa, 0x36, 0xf9, 0xf2, 0xc4, 0xed, 0x4c, 0xb5, 0xe7, 0x05, 0x5b, 0x9e, 0x13, 0xec, 0xd2, + 0x27, 0x90, 0x9d, 0x58, 0x41, 0x6f, 0x9c, 0x42, 0x92, 0x39, 0xb0, 0x7d, 0x0a, 0x46, 0xae, 0x01, + 0xd0, 0x78, 0x12, 0x93, 0xf5, 0x9a, 0x7c, 0xc2, 0xcd, 0x72, 0xca, 0xae, 0x6f, 0x57, 0x7e, 0x25, + 0x41, 0x8a, 0x81, 0x05, 0x7a, 0x1f, 0x32, 0x23, 0x32, 0xda, 0xa3, 0xf8, 0xce, 0xdf, 0xef, 0xf3, + 0xe6, 0xef, 0x48, 0x9c, 0x5e, 0x7a, 0x9e, 0x6f, 0x8d, 0x74, 0xff, 0x98, 0x7f, 0x09, 0xc4, 0xd1, + 0x12, 0xdd, 0x80, 0x6c, 0x34, 0x80, 0x47, 0x1f, 0x83, 0x4e, 0xcf, 0xe7, 0x53, 0xb6, 0x68, 0xaa, + 0x7e, 0x9f, 0x80, 0x34, 0x8f, 0x3a, 0xfa, 0x00, 0x20, 0x1a, 0xb2, 0x9f, 0xfa, 0x9b, 0x40, 0x56, + 0x68, 0xb4, 0xcc, 0x29, 0x38, 0x26, 0x9e, 0x02, 0x1c, 0x37, 0x41, 0x26, 0xa1, 0x61, 0x0a, 0x20, + 0xbd, 0x34, 0x5b, 0x01, 0xd5, 0x66, 0x68, 0x98, 0x51, 0x58, 0xa9, 0x60, 0xe9, 0x0b, 0x09, 0x64, + 0x4a, 0xa4, 0xf1, 0x35, 0xec, 0x31, 0xbd, 0xf2, 0x22, 0x2f, 0x65, 0x9c, 0x15, 0x94, 0x96, 0x89, + 0x5e, 0x80, 0x2c, 0x0f, 0x13, 0xe5, 0x26, 0x18, 0x57, 0xe1, 0x84, 0x96, 0x89, 0x4a, 0xa0, 0x4c, + 0xd0, 0x8f, 0xbf, 0xad, 0x93, 0x35, 0x55, 0xf4, 0xf5, 0xfd, 0x50, 0x0b, 0x89, 0xcf, 0x27, 0x6f, + 0x19, 0x2b, 0x94, 0x30, 0x20, 0xfe, 0x28, 0xfa, 0x34, 0x41, 0x7f, 0x6f, 0x7c, 0x97, 0x80, 0x34, + 0xaf, 0x68, 0x94, 0x86, 0x44, 0xf7, 0x23, 0x75, 0x09, 0x5d, 0x82, 0xd5, 0x3b, 0xdd, 0x5d, 0xdc, + 0xa9, 0xb5, 0xb5, 0x4e, 0x77, 0xa0, 0xed, 0x74, 0x77, 0x3b, 0x0d, 0x55, 0x42, 0xd7, 0xe0, 0x6a, + 0xa7, 0xab, 0x45, 0x9c, 0x1e, 0x6e, 0xdd, 0xad, 0xe1, 0xfb, 0x5a, 0x1d, 0x77, 0x3f, 0x6a, 0x62, + 0x35, 0x81, 0xd6, 0xa0, 0x44, 0xa5, 0x17, 0xf0, 0x93, 0xe8, 0x32, 0xa0, 0x38, 0x5f, 0xd0, 0x53, + 0x68, 0x1d, 0x5e, 0x6c, 0x75, 0xfa, 0xbb, 0x3b, 0x3b, 0xad, 0xed, 0x56, 0xb3, 0x33, 0x2b, 0xd0, + 0x57, 0x65, 0xf4, 0x22, 0x14, 0xbb, 0x3b, 0x3b, 0xfd, 0xe6, 0x80, 0xb9, 0x73, 0xbf, 0x39, 0xd0, + 0x6a, 0x1f, 0xd7, 0x5a, 0xed, 0x5a, 0xbd, 0xdd, 0x54, 0xd3, 0x68, 0x05, 0x72, 0xf7, 0x70, 0xb7, + 0x73, 0x4b, 0xc3, 0xdd, 0xdd, 0x41, 0x53, 0xcd, 0x50, 0xf7, 0x7b, 0xb8, 0xdb, 0xeb, 0xf6, 0x6b, + 0x6d, 0xed, 0x6e, 0xab, 0x7f, 0xb7, 0x36, 0xd8, 0xbe, 0xad, 0x2a, 0xe8, 0x05, 0xb8, 0xd2, 0x1c, + 0x6c, 0x37, 0xb4, 0x01, 0xae, 0x75, 0xfa, 0xb5, 0xed, 0x41, 0xab, 0xdb, 0xd1, 0x76, 0x6a, 0xad, + 0x76, 0xb3, 0xa1, 0x66, 0xa9, 0x11, 0x6a, 0xbb, 0xd6, 0x6e, 0x77, 0xef, 0x35, 0x1b, 0x2a, 0xa0, + 0x2b, 0xf0, 0x1c, 0xb7, 0x5a, 0xeb, 0xf5, 0x9a, 0x9d, 0x86, 0xc6, 0x1d, 0x50, 0x73, 0xd4, 0x99, + 0x56, 0xa7, 0xd1, 0xfc, 0x44, 0xbb, 0x5d, 0xeb, 0x6b, 0xb7, 0x70, 0xb3, 0x36, 0x68, 0xe2, 0x88, + 0x9b, 0xa7, 0x7b, 0xe3, 0xe6, 0xad, 0x56, 0x9f, 0x12, 0x27, 0x7b, 0x17, 0x6e, 0x38, 0xa0, 0xce, + 0x7e, 0xb3, 0x40, 0x39, 0xc8, 0xb4, 0x3a, 0x1f, 0xd7, 0xda, 0xad, 0x86, 0xba, 0x84, 0x14, 0x90, + 0x3b, 0xdd, 0x4e, 0x53, 0x95, 0xe8, 0xd3, 0xad, 0x4f, 0x5b, 0x3d, 0x35, 0x81, 0x0a, 0x90, 0xfd, + 0xb4, 0x3f, 0xa8, 0x75, 0x1a, 0x35, 0xdc, 0x50, 0x93, 0x08, 0x20, 0xdd, 0xef, 0xd4, 0x7a, 0xbd, + 0xfb, 0xaa, 0x4c, 0x63, 0x4d, 0x85, 0xe8, 0xbe, 0xed, 0x6e, 0xad, 0xa1, 0x35, 0x9a, 0xdb, 0xdd, + 0xbb, 0x3d, 0xdc, 0xec, 0xf7, 0x5b, 0xdd, 0x8e, 0x9a, 0xda, 0xfa, 0x45, 0x72, 0xda, 0x39, 0xfc, + 0x00, 0x64, 0xda, 0x6d, 0xa0, 0x4b, 0xb3, 0xdd, 0x07, 0xbb, 0x49, 0x4a, 0x97, 0xe7, 0x37, 0x25, + 0xe8, 0x7d, 0x48, 0xb1, 0x1b, 0x0e, 0x5d, 0x9e, 0x7f, 0x4f, 0x97, 0xae, 0x9c, 0xa1, 0x0b, 0xcd, + 0xf7, 0x40, 0xa6, 0x33, 0x78, 0x7c, 0xc3, 0xd8, 0x67, 0x8c, 0xf8, 0x86, 0xf1, 0x51, 0xfd, 0x2d, + 0x09, 0x7d, 0x00, 0x69, 0x3e, 0x10, 0xa1, 0xd3, 0xb6, 0xa7, 0xd3, 0x6a, 0xa9, 0x78, 0x96, 0xc1, + 0xd5, 0x37, 0x24, 0x74, 0x1b, 0xb2, 0x93, 0xe6, 0x17, 0x95, 0xe2, 0xbb, 0x9c, 0x9e, 0x27, 0x4a, + 0x2f, 0xcc, 0xe5, 0x45, 0x76, 0xde, 0xa2, 0x96, 0x0a, 0x34, 0x16, 0x13, 0x2c, 0x8e, 0x5b, 0x9b, + 0xbd, 0x8a, 0xe3, 0xd6, 0xce, 0x80, 0x77, 0xbd, 0xf9, 0xf0, 0xdf, 0x6b, 0x4b, 0x0f, 0xbf, 0x5b, + 0x93, 0xbe, 0xfd, 0x6e, 0x4d, 0xfa, 0xf5, 0xa3, 0xb5, 0xa5, 0x6f, 0x1e, 0xad, 0x49, 0x7f, 0x79, + 0xb4, 0x26, 0x7d, 0xfb, 0x68, 0x6d, 0xe9, 0x6f, 0x8f, 0xd6, 0x96, 0x3e, 0x7d, 0x79, 0xe8, 0x56, + 0x87, 0xfa, 0xe7, 0x24, 0x0c, 0x49, 0xd5, 0x24, 0x0f, 0x36, 0x0d, 0xd7, 0x27, 0x9b, 0x33, 0xff, + 0x83, 0xed, 0xa5, 0xd9, 0xd3, 0x3b, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x13, 0xd4, 0xf1, 0x26, + 0x21, 0x1b, 0x00, 0x00, } func (this *Label) Equal(that interface{}) bool { @@ -1958,6 +1964,9 @@ func (this *Label) Equal(that interface{}) bool { if this.Value != that1.Value { return false } + if this.Prefix != that1.Prefix { + return false + } return true } func (this *LabelSet) Equal(that interface{}) bool { @@ -2668,6 +2677,16 @@ func (m *Label) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Prefix { + i-- + if m.Prefix { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x18 + } if len(m.Value) > 0 { i -= len(m.Value) copy(dAtA[i:], m.Value) @@ -4231,6 +4250,9 @@ func (m *Label) ProtoSize() (n int) { if l > 0 { n += 1 + l + sovProtocol(uint64(l)) } + if m.Prefix { + n += 2 + } return n } @@ -4945,6 +4967,26 @@ func (m *Label) Unmarshal(dAtA []byte) error { } m.Value = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Prefix", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Prefix = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProtocol(dAtA[iNdEx:]) diff --git a/broker/protocol/protocol.proto b/broker/protocol/protocol.proto index 44fea2dd..d60c2cb1 100644 --- a/broker/protocol/protocol.proto +++ b/broker/protocol/protocol.proto @@ -97,8 +97,15 @@ enum CompressionCodec { message Label { option (gogoproto.equal) = true; + // Name of this label. string name = 1; + // Value of this label. string value = 2; + // Does this label match on value prefix? + // May only be set true within a LabelSelector. + bool prefix = 3 [ + (gogoproto.moretags) = "yaml:\",omitempty\"" + ]; } // LabelSet is a collection of labels and their values. From a5f9cc61a5a940c205060b7d53debd2667e06941 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 22 Jun 2024 13:21:34 -0500 Subject: [PATCH 3/8] broker: List RPC is now server-streaming and supports long-lived watches The List RPC was originally a unary API, which has two issues: * In large deployments, unary responses bump against gRPC maximum messages sizes. * Unary responses can't represent a long-lived watch of changes. Often callers want to react to changes in journals, such as reading a newly created journal or spreading appended messages across the current set of journals. client.NewPolledList _somewhat_ enabled this functionality, but is inefficient and requires selection of a polling interval. To address both issues, List is updated to become a server-streaming API while retaining backward comatibility with MOST clients expecting a unary response, so long as the response size is under one thousand journals. The List RPC is also extended with a Watch mode. When enabled, brokers will continue to stream real-time updates upon every change to the set of journals which match the selector. This lets clients react instantly to changes in journal topology. client.NewPolledList becomes client.NewWatchedList, with a nearly identical API (the polling interval is removed). The shard List RPC is not modified by this change, but could follow a similar path in the future. Refactor pbx.NewUnroutedHeader, which is a foot-gun because it obtains a re-entrant lock (which is not supported). Instead, make it the caller's responsibility to obtain an RLock prior to the call. --- allocator/allocator_key_space.go | 2 +- broker/client/list.go | 179 ++++-- broker/client/list_test.go | 20 +- broker/list_apply_api.go | 175 +++++- broker/list_apply_api_test.go | 103 +++- broker/protocol/ext/header_extensions.go | 5 +- broker/protocol/journal_spec_extensions.go | 14 +- .../protocol/journal_spec_extensions_test.go | 5 +- broker/protocol/protocol.pb.go | 519 +++++++++++------- broker/protocol/protocol.proto | 9 +- broker/protocol/rpc_extensions.go | 20 + broker/read_api.go | 21 +- broker/teststub/broker.go | 32 +- brokertest/broker_test.go | 29 +- cmd/gazctl/gazctlcmd/journals_append.go | 8 +- cmd/gazctl/gazctlcmd/journals_list.go | 24 +- cmd/gazctl/gazctlcmd/journals_read.go | 10 +- consumer/protocol/shard_spec_extensions.go | 13 +- .../protocol/shard_spec_extensions_test.go | 5 +- consumer/recovery_test.go | 3 +- consumer/shard_api.go | 52 +- examples/bike-share/application.go | 11 +- examples/stream-sum/stream_sum.go | 15 +- examples/word-count/word_count.go | 11 +- 24 files changed, 885 insertions(+), 400 deletions(-) diff --git a/allocator/allocator_key_space.go b/allocator/allocator_key_space.go index 3c86eebc..9096efea 100644 --- a/allocator/allocator_key_space.go +++ b/allocator/allocator_key_space.go @@ -16,7 +16,7 @@ const ( ItemsPrefix = "/items/" // MembersPrefix prefixes Member keys, eg "root/members/zone#suffix" MembersPrefix = "/members/" - // AssignmentsPrefix prefixes Assignment keys, eg "prefix/assign/item-id#zone#member-suffix#slot" + // AssignmentsPrefix prefixes Assignment keys, eg "root/assign/item-id#zone#member-suffix#slot" AssignmentsPrefix = "/assign/" // '#' is selected as separator, because it's the first visual ASCII character // which is not interpreted by shells (preceding visual characters are " and !). diff --git a/broker/client/list.go b/broker/client/list.go index 79d9f828..6f54125d 100644 --- a/broker/client/list.go +++ b/broker/client/list.go @@ -2,6 +2,8 @@ package client import ( "context" + "fmt" + "io" "sync/atomic" "time" @@ -11,97 +13,168 @@ import ( "google.golang.org/grpc" ) -// PolledList periodically polls the List RPC with a given ListRequest, making +// WatchedList drives ongoing List RPCs with a given ListRequest, making // its most recent result available via List. It's a building block for // applications which interact with dynamic journal sets and wish to react // to changes in their set membership over time. // -// var partitions, _ = protocol.ParseLabelSelector("logs=clicks, source=mobile") -// var pl, err = NewPolledList(ctx, client, time.Minute, protocol.ListRequest{ -// Selector: partitions, -// }) -// -type PolledList struct { +// var partitions, _ = protocol.ParseLabelSelector("logs=clicks, source=mobile") +// var pl, err = NewPolledList(ctx, client, protocol.ListRequest{ +// Selector: partitions, +// }) +type WatchedList struct { ctx context.Context client pb.JournalClient req pb.ListRequest resp atomic.Value - updateCh chan struct{} + updateCh chan error } -// NewPolledList returns a PolledList of the ListRequest which is initialized and -// ready for immediate use, and which will regularly refresh with the given Duration. -// An error encountered in the first List RPC is returned. Subsequent RPC errors -// will be logged as warnings and retried as part of regular refreshes. -func NewPolledList(ctx context.Context, client pb.JournalClient, dur time.Duration, req pb.ListRequest) (*PolledList, error) { - var resp, err = ListAllJournals(ctx, client, req) - if err != nil { - return nil, err +// NewWatchedList returns a WatchedList of the ListRequest which will keep +// up-to-date with watched changes from brokers. +// +// Upon return, the WatchedList is not yet ready, but the caller may await +// UpdateCh() to be notified when the very first listing (or error) is available. +// +// If `updateCh` is non-nil, then it's used as UpdateCh() instead of allocating a +// new channel. This is recommended if you have a dynamic set of WatchedList +// instances and wish to take action if any one of them update. Otherwise, +// a new channel with a single buffered item is allocated. +func NewWatchedList(ctx context.Context, client pb.JournalClient, req pb.ListRequest, updateCh chan error) *WatchedList { + if updateCh == nil { + updateCh = make(chan error, 1) } - var pl = &PolledList{ + + req = pb.ListRequest{ + Selector: req.Selector, + Watch: true, + } + + var pl = &WatchedList{ ctx: ctx, client: client, req: req, - updateCh: make(chan struct{}, 1), + updateCh: updateCh, } - pl.resp.Store(resp) - pl.updateCh <- struct{}{} - go pl.periodicRefresh(dur) - return pl, nil + go pl.watch() + return pl +} + +// List returns the most recent ListResponse snapshot, or nil if a snapshot has not been received yet. +func (pl *WatchedList) List() (out *pb.ListResponse) { + out, _ = pl.resp.Load().(*pb.ListResponse) + return } -// List returns the most recent polled & merged ListResponse (see ListAllJournals). -func (pl *PolledList) List() *pb.ListResponse { return pl.resp.Load().(*pb.ListResponse) } +// UpdateCh returns a channel which is signaled with each update or error of the +// PolledList. Errors are informational only: WatchedList will retry on all errors, +// but the caller may wish to examine errors and cancel its context. +// Only one signal is sent per-update, so if multiple goroutines +// select from UpdateCh only one will wake. +func (pl *WatchedList) UpdateCh() <-chan error { return pl.updateCh } -// UpdateCh returns a channel which is signaled with each update of the -// PolledList. Only one channel is allocated and one signal sent per-update, -// so if multiple goroutines select from UpdateCh only one will wake. -func (pl *PolledList) UpdateCh() <-chan struct{} { return pl.updateCh } +func (pl *WatchedList) watch() { + var ( + attempt int + err error + resp *pb.ListResponse + stream pb.Journal_ListClient + ) -func (pl *PolledList) periodicRefresh(dur time.Duration) { - var ticker = time.NewTicker(dur) for { - select { - case <-ticker.C: - var resp, err = ListAllJournals(pl.ctx, pl.client, pl.req) - if err != nil { - log.WithFields(log.Fields{"err": err, "req": pl.req.String()}). - Warn("periodic List refresh failed (will retry)") - } else { - pl.resp.Store(resp) + if stream == nil { + stream, err = pl.client.List(pb.WithDispatchDefault(pl.ctx), &pl.req, grpc.WaitForReady(true)) + } + if err == nil { + resp, err = ReadListResponse(stream, pl.req) + } + + if err == nil { + pl.resp.Store(resp) + pl.req.WatchResume = &resp.Header + attempt = 0 + } else { + stream = nil // Must restart. + + // Wait for back-off timer or context cancellation. + select { + case <-pl.ctx.Done(): + return + case <-time.After(backoff(attempt)): + } - select { - case pl.updateCh <- struct{}{}: - default: // Don't block if nobody's reading. - } + if attempt != 0 { + log.WithFields(log.Fields{"err": err, "attempt": attempt, "req": pl.req.String()}). + Warn("watched journal listing failed (will retry)") } - case <-pl.ctx.Done(): - ticker.Stop() - return + attempt += 1 + } + + select { + case pl.updateCh <- err: + default: // Don't block if nobody's reading. } } } -// ListAllJournals performs a broker journal listing. +// ListAllJournals performs a unary broker journal listing. // Any encountered error is returned. func ListAllJournals(ctx context.Context, client pb.JournalClient, req pb.ListRequest) (*pb.ListResponse, error) { - // List RPCs may be dispatched to any broker. - var resp, err = client.List(pb.WithDispatchDefault(ctx), &req, grpc.FailFast(false)) + if req.Watch { + panic("ListAllJournals cannot be used to Watch a listing") + } + var stream, err = client.List(pb.WithDispatchDefault(ctx), &req, grpc.WaitForReady(true)) if err != nil { - return resp, mapGRPCCtxErr(ctx, err) + return nil, mapGRPCCtxErr(ctx, err) + } + + if resp, err := ReadListResponse(stream, req); err != nil { + return nil, err + } else { + if dr, ok := client.(pb.DispatchRouter); ok { + for _, j := range resp.Journals { + dr.UpdateRoute(j.Spec.Name.String(), &j.Route) + } + } + return resp, nil + } +} + +// ReadListResponse reads a complete ListResponse snapshot from the stream. +func ReadListResponse(stream pb.Journal_ListClient, req pb.ListRequest) (*pb.ListResponse, error) { + var resp, err = stream.Recv() + + if err != nil { + return nil, mapGRPCCtxErr(stream.Context(), err) } else if err = resp.Validate(); err != nil { return resp, err } else if resp.Status != pb.Status_OK { return resp, errors.New(resp.Status.String()) + } else if req.WatchResume != nil && req.WatchResume.Etcd.Revision >= resp.Header.Etcd.Revision { + return nil, fmt.Errorf("server sent unexpected ListResponse revision (%d; expected > %d)", + resp.Header.Etcd.Revision, req.WatchResume.Etcd.Revision) } - if dr, ok := client.(pb.DispatchRouter); ok { - for _, j := range resp.Journals { - dr.UpdateRoute(j.Spec.Name.String(), &j.Route) + for { + if next, err := stream.Recv(); err == io.EOF { + if req.Watch { + return nil, io.ErrUnexpectedEOF // Unexpected EOF of long-lived watch. + } else { + return resp, nil // Expected EOF of unary listing. + } + } else if err != nil { + return nil, mapGRPCCtxErr(stream.Context(), err) + } else if len(next.Journals) == 0 { + if req.Watch { + return resp, nil + } else { + return nil, fmt.Errorf("unexpected empty ListResponse.Journals in unary listing") + } + } else { + resp.Journals = append(resp.Journals, next.Journals...) } } - return resp, nil } // GetJournal retrieves the JournalSpec of the named Journal, or returns an error. diff --git a/broker/client/list_test.go b/broker/client/list_test.go index dab5d10f..3c2b7877 100644 --- a/broker/client/list_test.go +++ b/broker/client/list_test.go @@ -118,24 +118,26 @@ func (s *ListSuite) TestPolledList(c *gc.C) { return &fixture, nil } - // Expect NewPolledList calls ListAllJournals once, and List is prepared before return. + // Expect NewWatchedList asynchronously lists journals. callCh <- struct{}{} var ctx, cancel = context.WithCancel(context.Background()) defer cancel() - var pl, err = NewPolledList(ctx, broker.Client(), 5*time.Millisecond, pb.ListRequest{}) - c.Check(err, gc.IsNil) + var pl = NewWatchedList(ctx, broker.Client(), pb.ListRequest{}, nil) + c.Check(<-pl.UpdateCh(), gc.IsNil) // Initial load. c.Check(pl.List(), gc.DeepEquals, &fixture) - <-pl.UpdateCh() // Expect UpdateCh is initially ready to select. + + c.Check(<-pl.UpdateCh(), gc.Equals, context.Canceled) // Alter the fixture. List will eventually reflect it, after being given a chance to refresh. fixture.Journals = mk("part-one", "part-two", "part-three") + fixture.Header.Etcd.Revision += 1 c.Check(pl.List(), gc.Not(gc.DeepEquals), &fixture) - // Expect another poll is done, and the PolledList updates. + // Expect another listing is done, and the PolledList updates. callCh <- struct{}{} - <-pl.UpdateCh() + c.Check(<-pl.UpdateCh(), gc.IsNil) c.Check(pl.List(), gc.DeepEquals, &fixture) } @@ -264,7 +266,7 @@ func (s *ListSuite) TestApplyJournalsInBatches(c *gc.C) { broker.ApplyFunc = func(ctx context.Context, req *pb.ApplyRequest) (*pb.ApplyResponse, error) { return nil, errors.New("something has gone wrong") } - resp, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) + _, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) c.Check(err, gc.ErrorMatches, "rpc error: code = Unknown desc = something has gone wrong") // Case: Status !OK mapped as an error. @@ -274,7 +276,7 @@ func (s *ListSuite) TestApplyJournalsInBatches(c *gc.C) { Header: *hdr, }, nil } - resp, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) + _, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) require.EqualError(c, err, pb.Status_ETCD_TRANSACTION_FAILED.String()) // Case: Validation error mapped as error. @@ -283,7 +285,7 @@ func (s *ListSuite) TestApplyJournalsInBatches(c *gc.C) { Status: pb.Status_ETCD_TRANSACTION_FAILED, }, nil } - resp, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) + _, err = ApplyJournalsInBatches(ctx, rjc, fixture, 1) c.Check(err, gc.ErrorMatches, `Header.Route: invalid Primary \(0; expected -1 <= Primary < 0\)`) } diff --git a/broker/list_apply_api.go b/broker/list_apply_api.go index 85f1fcc7..6d2e2fa9 100644 --- a/broker/list_apply_api.go +++ b/broker/list_apply_api.go @@ -10,17 +10,18 @@ import ( "go.gazette.dev/core/allocator" pb "go.gazette.dev/core/broker/protocol" pbx "go.gazette.dev/core/broker/protocol/ext" + "go.gazette.dev/core/keyspace" "google.golang.org/grpc/peer" ) // List dispatches the JournalServer.List API. -func (svc *Service) List(ctx context.Context, req *pb.ListRequest) (resp *pb.ListResponse, err error) { +func (svc *Service) List(req *pb.ListRequest, stream pb.Journal_ListServer) (err error) { defer instrumentJournalServerRPC("List", &err, nil)() defer func() { if err != nil { var addr net.Addr - if p, ok := peer.FromContext(ctx); ok { + if p, ok := peer.FromContext(stream.Context()); ok { addr = p.Addr } log.WithFields(log.Fields{"err": err, "req": req, "client": addr}). @@ -28,16 +29,9 @@ func (svc *Service) List(ctx context.Context, req *pb.ListRequest) (resp *pb.Lis } }() - var s = svc.resolver.state - - resp = &pb.ListResponse{ - Status: pb.Status_OK, - Header: pbx.NewUnroutedHeader(s), - } if err = req.Validate(); err != nil { - return resp, err + return err } - var metaLabels, allLabels pb.LabelSet // Gazette has historically offered a special "prefix" label which matches // slash-terminated prefixes of a journal name. Today, it's implemented in @@ -51,34 +45,148 @@ func (svc *Service) List(ctx context.Context, req *pb.ListRequest) (resp *pb.Lis } } - defer s.KS.Mu.RUnlock() - s.KS.Mu.RLock() + var ( + // Etcd revision to read through before responding. + minEtcdRevision int64 = 0 + // Number of journals in the last list snapshot. + lastMatchedItems = -1 + // Max Etcd mod revision in the last list snapshot. + lastMaxModRevision int64 = -1 + ) + if req.WatchResume != nil { + // Resume from the next Etcd revision. + minEtcdRevision = req.WatchResume.Etcd.Revision + 1 + } + + for { + var resp, err = listRound( + stream.Context(), + &req.Selector, + svc.resolver.state, + &minEtcdRevision, + &lastMatchedItems, + &lastMaxModRevision, + ) + + if err != nil { + return pb.SuppressCancellationError(err) + } else if resp == nil { + continue // Next snapshot is not ready. + } + + // Note that listWatchRound() holds an RLock over our service state, + // but we do *not* hold a lock below while streaming the response. + // We cannot allow client back-pressure to cause us to hold our lock. + + for len(resp.Journals) > maxJournalsPerListResponse { + var tail = resp.Journals[maxJournalsPerListResponse:] + resp.Journals = resp.Journals[:maxJournalsPerListResponse] + + if err = stream.Send(resp); err != nil { + return err + } + *resp = pb.ListResponse{Journals: tail} + } + if err = stream.Send(resp); err != nil { + return err + } + + if !req.Watch { + return nil // Unary snapshot is complete. + } else if err = stream.Send(&pb.ListResponse{}); err != nil { + return err + } + } +} + +func listRound( + ctx context.Context, + selector *pb.LabelSelector, + state *allocator.State, + minEtcdRevision *int64, + lastMatchedItems *int, + lastMaxModRevision *int64, +) (*pb.ListResponse, error) { + defer state.KS.Mu.RUnlock() + state.KS.Mu.RLock() + + if *minEtcdRevision > state.KS.Header.Revision { + addTrace(ctx, " ... at revision %d, but want at least %d", + state.KS.Header.Revision, *minEtcdRevision) + + if err := state.KS.WaitForRevision(ctx, *minEtcdRevision); err != nil { + return nil, err + } + addTrace(ctx, "WaitForRevision(%d) => %d", + minEtcdRevision, state.KS.Header.Revision) + } + // Next iteration, we'll block until we observe a still-larger revision. + *minEtcdRevision = state.KS.Header.Revision + 1 + + var ( + allItems = state.Items + assignments = state.Assignments + matchedItems keyspace.KeyValues + maxModRevision int64 = -1 + scratch pb.LabelSet + ) + + // If we're matching over a single name (or name prefix), then leverage + // the fact that Items and Assignments are naturally ordered on name. + if name := selector.Include.ValuesOf("name"); len(name) == 1 { + allItems = allItems.Prefixed(state.KS.Root + allocator.ItemsPrefix + name[0]) + assignments = assignments.Prefixed(state.KS.Root + allocator.AssignmentsPrefix + name[0]) + } + + for _, item := range allItems { + var spec = item.Decoded.(allocator.Item).ItemValue.(*pb.JournalSpec) + + // LabelSetExt() truncates `scratch` while re-using its storage. + scratch = spec.LabelSetExt(scratch) + + if selector.Matches(scratch) { + if item.Raw.ModRevision > maxModRevision { + maxModRevision = item.Raw.ModRevision + } + matchedItems = append(matchedItems, item) + } + } + + // If the number of items is unchanged AND the maximum mod revision is unchanged, + // then the snapshot cannot have changed. This follows because a create or update + // will increase the mod revision, and a deletion will decrease the number of items. + if len(matchedItems) == *lastMatchedItems && maxModRevision == *lastMaxModRevision { + return nil, nil + } + + // The listing has changed and we must build a new snapshot. + *lastMatchedItems, *lastMaxModRevision = len(matchedItems), maxModRevision + + var resp = &pb.ListResponse{ + Status: pb.Status_OK, + Header: pbx.NewUnroutedHeader(state), + } var it = allocator.LeftJoin{ - LenL: len(s.Items), - LenR: len(s.Assignments), + LenL: len(matchedItems), + LenR: len(assignments), Compare: func(l, r int) int { - var lID = s.Items[l].Decoded.(allocator.Item).ID - var rID = s.Assignments[r].Decoded.(allocator.Assignment).ItemID + var lID = matchedItems[l].Decoded.(allocator.Item).ID + var rID = assignments[r].Decoded.(allocator.Assignment).ItemID return strings.Compare(lID, rID) }, } for cur, ok := it.Next(); ok; cur, ok = it.Next() { var journal = pb.ListResponse_Journal{ - Spec: *s.Items[cur.Left].Decoded.(allocator.Item).ItemValue.(*pb.JournalSpec)} - metaLabels = pb.ExtractJournalSpecMetaLabels(&journal.Spec, metaLabels) - allLabels = pb.UnionLabelSets(metaLabels, journal.Spec.LabelSet, allLabels) - - if !req.Selector.Matches(allLabels) { - continue + Spec: *matchedItems[cur.Left].Decoded.(allocator.Item).ItemValue.(*pb.JournalSpec), + ModRevision: matchedItems[cur.Left].Raw.ModRevision, + CreateRevision: matchedItems[cur.Left].Raw.CreateRevision, } - journal.ModRevision = s.Items[cur.Left].Raw.ModRevision - journal.CreateRevision = s.Items[cur.Left].Raw.CreateRevision - pbx.Init(&journal.Route, s.Assignments[cur.RightBegin:cur.RightEnd]) - pbx.AttachEndpoints(&journal.Route, s.KS) - + pbx.Init(&journal.Route, assignments[cur.RightBegin:cur.RightEnd]) + pbx.AttachEndpoints(&journal.Route, state.KS) resp.Journals = append(resp.Journals, journal) } + return resp, nil } @@ -123,10 +231,12 @@ func (svc *Service) Apply(ctx context.Context, req *pb.ApplyRequest) (resp *pb.A } } + s.KS.Mu.RLock() resp = &pb.ApplyResponse{ Status: pb.Status_OK, Header: pbx.NewUnroutedHeader(s), } + s.KS.Mu.RUnlock() var txnResp clientv3.OpResponse if txnResp, err = svc.etcd.Do(ctx, clientv3.OpTxn(cmp, ops, nil)); err != nil { @@ -142,3 +252,14 @@ func (svc *Service) Apply(ctx context.Context, req *pb.ApplyRequest) (resp *pb.A resp.Header.Etcd.Revision = txnResp.Txn().Header.Revision return resp, err } + +// NOTE(johnny): List was originally a unary API, which had two issues: +// * In large deployments, responses would bump against gRPC maximum messages sizes. +// * Unary responses can't represent a long-lived watch. +// +// List was updated to become a server-streaming API which addresses both issues, +// while remaining compatible with older clients that expect a unary API, +// so long as the entire response set can be sent as a single message. +// This value is large to allow older deployments to continue to function, +// without being SO large that we bump against gRPC message limits. +var maxJournalsPerListResponse = 1000 diff --git a/broker/list_apply_api_test.go b/broker/list_apply_api_test.go index 56813adc..d4e93384 100644 --- a/broker/list_apply_api_test.go +++ b/broker/list_apply_api_test.go @@ -2,6 +2,7 @@ package broker import ( "context" + "io" "testing" "time" @@ -15,6 +16,9 @@ func TestListCases(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + defer func(v int) { maxJournalsPerListResponse = v }(maxJournalsPerListResponse) + maxJournalsPerListResponse = 2 + // Create a fixture of JournalSpecs which we'll list. var fragSpec = pb.JournalSpec_Fragment{ Length: 1024, @@ -67,8 +71,20 @@ func TestListCases(t *testing.T) { require.NoError(t, broker.ks.WaitForRevision(ctx, rev)) broker.ks.Mu.RUnlock() - var verify = func(resp *pb.ListResponse, expect ...*pb.JournalSpec) { + var verify = func(stream pb.Journal_ListClient, expect ...*pb.JournalSpec) { + var resp, err = stream.Recv() + require.NoError(t, err) require.Equal(t, pb.Status_OK, resp.Status) + + // Accumulate all streamed responses. + for { + var next, err = stream.Recv() + if err == io.EOF || err == nil && len(next.Journals) == 0 { + break // End of stream or snapshot. + } + require.NoError(t, err) + resp.Journals = append(resp.Journals, next.Journals...) + } require.Len(t, resp.Journals, len(expect)) for i, exp := range expect { @@ -89,47 +105,103 @@ func TestListCases(t *testing.T) { } } - // Case: Empty selector returns all shards. - var resp, err = broker.client().List(ctx, &pb.ListRequest{ + // Case: Empty selector returns all journals. + var stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{}, }) require.NoError(t, err) - verify(resp, specA, specC, specB) + verify(stream, specA, specC, specB) + + // Case: Unmatched include selector returns no journals. + stream, err = broker.client().List(ctx, &pb.ListRequest{ + Selector: pb.LabelSelector{Include: pb.MustLabelSet("missing", "label")}, + }) + require.NoError(t, err) + verify(stream) // Case: Exclude on label. - resp, err = broker.client().List(ctx, &pb.ListRequest{ + stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Exclude: pb.MustLabelSet("foo", "")}, }) require.NoError(t, err) - verify(resp, specC, specB) + verify(stream, specC, specB) // Case: Meta-label "name" selects journals by name. - resp, err = broker.client().List(ctx, &pb.ListRequest{ + stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("name", "journal/2/B")}, }) require.NoError(t, err) - verify(resp, specB) + verify(stream, specB) // Case: Meta-label "name:prefix" selects journals by name prefix. - resp, err = broker.client().List(ctx, &pb.ListRequest{ + stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("name:prefix", "journal/1")}, }) require.NoError(t, err) - verify(resp, specA, specC) + verify(stream, specA, specC) // Case: legacy meta-label "prefix" also selects journals by name prefix. - resp, err = broker.client().List(ctx, &pb.ListRequest{ + stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("prefix", "journal/1/")}, }) require.NoError(t, err) - verify(resp, specA, specC) + verify(stream, specA, specC) // Case: Errors on request validation error. - _, err = broker.client().List(ctx, &pb.ListRequest{ + stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("prefix", "invalid/because/missing/trailing/slash")}, }) + require.NoError(t, err) + _, err = stream.Recv() require.Regexp(t, `.* Selector.Include.Labels\["prefix"\]: expected trailing '/' (.*)`, err) + // Case: streaming watch of a prefix. + var cancelCtx, cancel = context.WithCancel(ctx) + stream, err = broker.client().List(cancelCtx, &pb.ListRequest{ + Selector: pb.LabelSelector{Include: pb.MustLabelSet("name:prefix", "journal/1/")}, + Watch: true, + }) + require.NoError(t, err) + verify(stream, specA, specC) + + // Delete a journal that's not part of the listing. + resp, err := broker.client().Apply(ctx, &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{ + {Delete: specB.Name, ExpectModRevision: fixtureRevision}, + }, + }) + require.NoError(t, err) + require.Equal(t, pb.Status_OK, resp.Status) + + // Delete a journal that IS part of the listing. + resp, err = broker.client().Apply(ctx, &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{ + {Delete: specA.Name, ExpectModRevision: fixtureRevision}, + }, + }) + require.NoError(t, err) + require.Equal(t, pb.Status_OK, resp.Status) + + // Expect we see a new snapshot, this time having only C. + verify(stream, specC) + + // Delete final listed journal. + resp, err = broker.client().Apply(ctx, &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{ + {Delete: specC.Name, ExpectModRevision: fixtureRevision}, + }, + }) + require.NoError(t, err) + require.Equal(t, pb.Status_OK, resp.Status) + + // Expect we see an empty snapshot. + verify(stream) + + // Expect to read no further responses upon cancellation. + cancel() + _, err = stream.Recv() + require.Equal(t, "rpc error: code = Canceled desc = context canceled", err.Error()) + broker.cleanup() } @@ -156,10 +228,13 @@ func TestApplyCases(t *testing.T) { var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) var verifyAndFetchRev = func(name pb.Journal, expect pb.JournalSpec) int64 { - var resp, err = broker.client().List(ctx, &pb.ListRequest{ + var stream, err = broker.client().List(ctx, &pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("name", name.String())}, }) require.NoError(t, err) + resp, err := stream.Recv() + require.NoError(t, err) + require.Equal(t, pb.Status_OK, resp.Status) require.Equal(t, expect, resp.Journals[0].Spec) return resp.Journals[0].ModRevision diff --git a/broker/protocol/ext/header_extensions.go b/broker/protocol/ext/header_extensions.go index 4ee1db8c..37889e03 100644 --- a/broker/protocol/ext/header_extensions.go +++ b/broker/protocol/ext/header_extensions.go @@ -20,10 +20,9 @@ func FromEtcdResponseHeader(h epb.ResponseHeader) pb.Header_Etcd { // from the v3_allocator.State, and Route left as zero-valued. It is a helper for // APIs which do not utilize item resolution but still return Headers (eg, List // and Update). +// +// You MUST hold a lock over State.KS.Mu when calling this function! func NewUnroutedHeader(s *allocator.State) (hdr pb.Header) { - defer s.KS.Mu.RUnlock() - s.KS.Mu.RLock() - if s.LocalMemberInd != -1 { var member = s.Members[s.LocalMemberInd].Decoded.(allocator.Member) hdr.ProcessId = pb.ProcessSpec_ID{Zone: member.Zone, Suffix: member.Suffix} diff --git a/broker/protocol/journal_spec_extensions.go b/broker/protocol/journal_spec_extensions.go index 82a1ea3d..8e4cd147 100644 --- a/broker/protocol/journal_spec_extensions.go +++ b/broker/protocol/journal_spec_extensions.go @@ -358,11 +358,15 @@ func SubtractJournalSpecs(a, b JournalSpec) JournalSpec { return a } -// ExtractJournalSpecMetaLabels adds to the LabelSet a singular label "name", -// with value of the JournalSpec Name. -func ExtractJournalSpecMetaLabels(spec *JournalSpec, out LabelSet) LabelSet { - out.Labels = append(out.Labels[:0], Label{Name: "name", Value: spec.Name.String()}) - return out +// LabelSetExt adds additional metadata labels to the LabelSet of the JournalSpec, +// returning the result. The result is built by truncating `buf` and then appending +// the merged LabelSet. +func (m *JournalSpec) LabelSetExt(buf LabelSet) LabelSet { + return UnionLabelSets(LabelSet{ + Labels: []Label{ + {Name: "name", Value: m.Name.String()}, + }, + }, m.LabelSet, buf) } // validateJournalLabelConstraints asserts expected invariants of MessageType, diff --git a/broker/protocol/journal_spec_extensions_test.go b/broker/protocol/journal_spec_extensions_test.go index 526f3f60..27a99835 100644 --- a/broker/protocol/journal_spec_extensions_test.go +++ b/broker/protocol/journal_spec_extensions_test.go @@ -178,8 +178,9 @@ func (s *JournalSuite) TestSpecValidationCases(c *gc.C) { } func (s *JournalSuite) TestMetaLabelExtraction(c *gc.C) { - c.Check(ExtractJournalSpecMetaLabels(&JournalSpec{Name: "path/to/my/journal"}, MustLabelSet("label", "buffer")), - gc.DeepEquals, MustLabelSet("name", "path/to/my/journal")) + var spec = JournalSpec{Name: "path/to/my/journal", LabelSet: MustLabelSet("hello", "world")} + c.Check(spec.LabelSetExt(MustLabelSet("label", "buffer")), + gc.DeepEquals, MustLabelSet("name", "path/to/my/journal", "hello", "world")) } func (s *JournalSuite) TestFlagYAMLRoundTrip(c *gc.C) { diff --git a/broker/protocol/protocol.pb.go b/broker/protocol/protocol.pb.go index 819d3de1..b69dcbe1 100644 --- a/broker/protocol/protocol.pb.go +++ b/broker/protocol/protocol.pb.go @@ -1160,6 +1160,11 @@ type ListRequest struct { // - prefix=examples/ will match any JournalSpec having prefix "examples/". // The prefix Label value must end in '/'. Selector LabelSelector `protobuf:"bytes,1,opt,name=selector,proto3" json:"selector"` + // Watch the requested selector and send an updated snapshot upon every + // change to matched journal specifications. + Watch bool `protobuf:"varint,4,opt,name=watch,proto3" json:"watch,omitempty"` + // Header of the last ListResponse processed by a resumed watch of a listing. + WatchResume *Header `protobuf:"bytes,5,opt,name=watch_resume,json=watchResume,proto3" json:"watch_resume,omitempty"` } func (m *ListRequest) Reset() { *m = ListRequest{} } @@ -1195,7 +1200,7 @@ func (m *ListRequest) XXX_DiscardUnknown() { var xxx_messageInfo_ListRequest proto.InternalMessageInfo -// ListResponse is the unary response message of the broker List RPC. +// ListResponse is the streamed response message of the broker List RPC. type ListResponse struct { // Status of the List RPC. Status Status `protobuf:"varint,1,opt,name=status,proto3,enum=protocol.Status" json:"status,omitempty"` @@ -1772,171 +1777,173 @@ func init() { } var fileDescriptor_0c0999e5af553218 = []byte{ - // 2612 bytes of a gzipped FileDescriptorProto + // 2646 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4d, 0x70, 0xdb, 0xc6, - 0xf5, 0x17, 0x48, 0x90, 0x04, 0x1f, 0x49, 0x09, 0xda, 0xc4, 0x36, 0xcd, 0xc4, 0xa2, 0xc2, 0x7c, - 0xfc, 0x65, 0x27, 0xa1, 0x12, 0xe5, 0xdf, 0x24, 0x75, 0x27, 0x6d, 0x48, 0x91, 0xb2, 0xe9, 0xd0, - 0x24, 0x67, 0x49, 0xc5, 0x71, 0x0e, 0xc5, 0x40, 0xc0, 0x8a, 0x42, 0x05, 0x02, 0x28, 0x00, 0x3a, - 0x52, 0x6e, 0xb9, 0xb4, 0x39, 0xa4, 0x33, 0x9d, 0x9e, 0x72, 0xea, 0xe4, 0xd2, 0x73, 0x3b, 0xd3, - 0x5b, 0x7b, 0xe9, 0xd1, 0xbd, 0xe5, 0xd8, 0x99, 0xb6, 0xea, 0x34, 0x9e, 0xe9, 0xf4, 0xec, 0x63, - 0x4e, 0x9d, 0xfd, 0x00, 0x09, 0x51, 0xa4, 0x65, 0x1f, 0x74, 0xe1, 0x60, 0xdf, 0xd7, 0xbe, 0x7d, - 0xef, 0xe1, 0xb7, 0xef, 0x81, 0xb0, 0xb6, 0xe7, 0xbb, 0x87, 0xc4, 0xdf, 0xf4, 0x7c, 0x37, 0x74, - 0x0d, 0xd7, 0x9e, 0x3c, 0x54, 0xd9, 0x03, 0x52, 0xa2, 0x75, 0xe9, 0xf9, 0xa1, 0x3b, 0x74, 0xd9, - 0x6a, 0x93, 0x3e, 0x71, 0x7e, 0x69, 0x6d, 0xe8, 0xba, 0x43, 0x9b, 0x70, 0xb5, 0xbd, 0xf1, 0xfe, - 0xa6, 0x39, 0xf6, 0xf5, 0xd0, 0x72, 0x1d, 0xce, 0xaf, 0xec, 0x43, 0xaa, 0xad, 0xef, 0x11, 0x1b, - 0x21, 0x90, 0x1d, 0x7d, 0x44, 0x8a, 0xd2, 0xba, 0xb4, 0x91, 0xc5, 0xec, 0x19, 0x3d, 0x0f, 0xa9, - 0x07, 0xba, 0x3d, 0x26, 0xc5, 0x04, 0x23, 0xf2, 0x05, 0x7a, 0x13, 0xd2, 0x9e, 0x4f, 0xf6, 0xad, - 0xa3, 0x62, 0x72, 0x5d, 0xda, 0x50, 0xea, 0x97, 0x1e, 0x9f, 0x94, 0x57, 0x8f, 0xf5, 0x91, 0x7d, - 0xb3, 0xf2, 0x86, 0x3b, 0xb2, 0x42, 0x32, 0xf2, 0xc2, 0xe3, 0x0a, 0x16, 0x42, 0x37, 0xe5, 0xff, - 0x7e, 0x53, 0x96, 0x2a, 0x03, 0x50, 0xd8, 0x3e, 0x7d, 0x12, 0xa2, 0x3a, 0xa4, 0x6d, 0xfa, 0x1c, - 0x14, 0xa5, 0xf5, 0xe4, 0x46, 0x6e, 0x6b, 0xa5, 0x3a, 0x39, 0x14, 0x93, 0xa9, 0x5f, 0x7d, 0x78, - 0x52, 0x5e, 0x5a, 0x60, 0x95, 0x6b, 0x0a, 0xab, 0x5f, 0x48, 0x50, 0x10, 0x66, 0x6d, 0x62, 0x84, - 0xae, 0x8f, 0xb6, 0x20, 0x63, 0x39, 0x86, 0x3d, 0x36, 0xf9, 0x49, 0x72, 0x5b, 0x68, 0xc6, 0x78, - 0x9f, 0x84, 0x75, 0x99, 0xda, 0xc7, 0x91, 0x20, 0xd5, 0x21, 0x47, 0x5c, 0x27, 0x71, 0x9e, 0x8e, - 0x10, 0xbc, 0xa9, 0x7c, 0xfd, 0x4d, 0x79, 0x89, 0xf9, 0xf0, 0x55, 0x16, 0x72, 0x77, 0xdc, 0xb1, - 0xef, 0xe8, 0x76, 0xdf, 0x23, 0x06, 0xfa, 0xff, 0x78, 0x20, 0xeb, 0xeb, 0x73, 0x8f, 0xf1, 0xfd, - 0x49, 0x39, 0x23, 0x74, 0x44, 0xa8, 0xdf, 0x83, 0x9c, 0x4f, 0x3c, 0xdb, 0x32, 0x58, 0x72, 0x98, - 0x1f, 0xa9, 0x45, 0x91, 0x8d, 0x4b, 0xa2, 0xde, 0x24, 0x98, 0xc9, 0x85, 0xbe, 0xbf, 0x42, 0x7d, - 0xff, 0xf6, 0xa4, 0x2c, 0x3d, 0x3e, 0x29, 0x17, 0x67, 0xed, 0xbd, 0x61, 0x39, 0xb6, 0xe5, 0x90, - 0x49, 0x68, 0xd1, 0x2e, 0x28, 0xfb, 0xbe, 0x3e, 0x1c, 0x11, 0x27, 0x2c, 0xca, 0xcc, 0xe6, 0xda, - 0xd4, 0x66, 0xec, 0xa4, 0xd5, 0x1d, 0x21, 0xf5, 0xa4, 0x7c, 0x4d, 0x4c, 0xa1, 0x9f, 0x40, 0x6a, - 0xdf, 0xd6, 0x87, 0x41, 0x31, 0xbd, 0x2e, 0x6d, 0x14, 0xea, 0xd7, 0x17, 0x05, 0x46, 0x8d, 0x6d, - 0xa1, 0xed, 0xd8, 0xfa, 0x10, 0x73, 0x3d, 0xd4, 0x86, 0x95, 0x91, 0x7e, 0xa4, 0xe9, 0x9e, 0x47, - 0x1c, 0x53, 0xf3, 0xf5, 0x90, 0x14, 0x33, 0xeb, 0xd2, 0x46, 0xb2, 0xfe, 0xca, 0xe3, 0x93, 0xf2, - 0x3a, 0x37, 0x35, 0x23, 0x10, 0xf7, 0xa4, 0x30, 0xd2, 0x8f, 0x6a, 0x8c, 0x85, 0xf5, 0x90, 0x94, - 0xbe, 0x4a, 0x81, 0x12, 0x1d, 0x80, 0x96, 0xb4, 0x4d, 0x9c, 0x61, 0x78, 0xc0, 0xb2, 0x96, 0x5c, - 0x58, 0xd2, 0x5c, 0x08, 0xb9, 0xb0, 0x6a, 0xb8, 0x23, 0xcf, 0x27, 0x41, 0x60, 0xb9, 0x8e, 0x66, - 0xb8, 0x26, 0x31, 0x58, 0xca, 0x96, 0xb7, 0x4a, 0xd3, 0x50, 0x6d, 0x4f, 0x45, 0xb6, 0xa9, 0x44, - 0xfd, 0xb5, 0xc7, 0x27, 0xe5, 0x0a, 0xb7, 0x7a, 0x46, 0x3d, 0xbe, 0x8d, 0x6a, 0xcc, 0x68, 0xa2, - 0x1f, 0x43, 0x3a, 0x08, 0x5d, 0x9f, 0xd0, 0x24, 0x27, 0x37, 0xb2, 0xcc, 0xd2, 0xdc, 0xe0, 0x15, - 0xa2, 0x23, 0xf5, 0xa9, 0x38, 0x16, 0x5a, 0x28, 0x00, 0xd5, 0x27, 0xfb, 0x3e, 0x09, 0x0e, 0x34, - 0xcb, 0x09, 0x89, 0xff, 0x40, 0xb7, 0x45, 0x6a, 0xaf, 0x56, 0x39, 0x40, 0x54, 0x23, 0x80, 0xa8, - 0x36, 0x04, 0x40, 0xd4, 0xdf, 0x14, 0x59, 0x7d, 0x89, 0x6f, 0x34, 0x6b, 0x20, 0xb6, 0xf1, 0xd7, - 0xff, 0x2a, 0x4b, 0x78, 0x45, 0x08, 0xb4, 0x04, 0x1f, 0x7d, 0x0c, 0x59, 0x9f, 0x84, 0xc4, 0x61, - 0x05, 0x9d, 0x3a, 0x6f, 0xb7, 0x6b, 0x0b, 0x6b, 0x88, 0x59, 0x9f, 0x9a, 0x42, 0x23, 0x58, 0xde, - 0xb7, 0xc7, 0xf1, 0xa3, 0xa4, 0xcf, 0x33, 0xfe, 0xba, 0x30, 0x5e, 0xe6, 0xc6, 0x4f, 0xab, 0xcf, - 0x6e, 0x55, 0x60, 0xec, 0xc9, 0x31, 0x7e, 0x0a, 0x97, 0x3c, 0x3d, 0x3c, 0xd0, 0x3c, 0x37, 0x08, - 0xf7, 0xad, 0x23, 0x8d, 0x8a, 0xda, 0x51, 0xf1, 0x65, 0xeb, 0x37, 0x1e, 0x9f, 0x94, 0x5f, 0xe3, - 0x66, 0xe7, 0x8a, 0xc5, 0x13, 0xfb, 0x1c, 0x95, 0xe8, 0x71, 0x81, 0x81, 0xe0, 0x0b, 0x24, 0xab, - 0x81, 0x4c, 0x6b, 0x1d, 0xad, 0x42, 0xa1, 0xd3, 0x1d, 0x68, 0xfd, 0x5e, 0x73, 0xbb, 0xb5, 0xd3, - 0x6a, 0x36, 0xd4, 0x25, 0x94, 0x07, 0xa5, 0xab, 0xe1, 0x46, 0xb7, 0xd3, 0xbe, 0xaf, 0x4a, 0x7c, - 0x75, 0x0f, 0xb3, 0x55, 0x02, 0x01, 0xa4, 0x29, 0xef, 0x1e, 0x56, 0x65, 0x61, 0xe8, 0x77, 0x12, - 0xe4, 0x7a, 0xbe, 0x6b, 0x90, 0x20, 0x60, 0x70, 0x54, 0x85, 0x84, 0x65, 0x0a, 0x2c, 0x2c, 0x4e, - 0x8b, 0x33, 0x26, 0x52, 0x6d, 0x35, 0x04, 0xba, 0x25, 0x2c, 0x13, 0x6d, 0x80, 0x42, 0x1c, 0xd3, - 0x73, 0x2d, 0x27, 0xe4, 0xb0, 0x5f, 0xcf, 0x7f, 0x7f, 0x52, 0x56, 0x9a, 0x82, 0x86, 0x27, 0xdc, - 0xd2, 0xbb, 0x90, 0x68, 0x35, 0xe8, 0xbd, 0xf1, 0xb9, 0xeb, 0x4c, 0xee, 0x0d, 0xfa, 0x8c, 0x2e, - 0x43, 0x3a, 0x18, 0xef, 0xd3, 0x1b, 0x82, 0x5f, 0x1c, 0x62, 0xc5, 0x3d, 0xbc, 0x29, 0x7f, 0x49, - 0xfd, 0xfc, 0xa5, 0x04, 0x50, 0x67, 0x77, 0x1b, 0x73, 0x73, 0x00, 0x79, 0x8f, 0xbb, 0xa4, 0x05, - 0x1e, 0x31, 0x84, 0xc3, 0x97, 0xe6, 0x3a, 0x5c, 0x2f, 0xc5, 0xf0, 0x6c, 0x59, 0xd4, 0x4b, 0x84, - 0x62, 0x39, 0x2f, 0x76, 0xf8, 0x97, 0xa1, 0xf0, 0x33, 0x8e, 0x26, 0x9a, 0x6d, 0x8d, 0x2c, 0x7e, - 0xa2, 0x02, 0xce, 0x0b, 0x62, 0x9b, 0xd2, 0x2a, 0x7f, 0x4f, 0xc4, 0x90, 0xe0, 0x55, 0xc8, 0x08, - 0xa6, 0x00, 0xf0, 0x5c, 0x1c, 0xab, 0x23, 0x1e, 0x5a, 0x87, 0xd4, 0x1e, 0x19, 0x5a, 0x1c, 0xa8, - 0x93, 0x75, 0xf8, 0xfe, 0xa4, 0x9c, 0xee, 0xee, 0xef, 0x07, 0x24, 0xc4, 0x9c, 0x81, 0x5e, 0x84, - 0x24, 0x71, 0x4c, 0x06, 0xca, 0xa7, 0xf9, 0x94, 0x8c, 0xae, 0x43, 0x32, 0x18, 0x8f, 0xc4, 0x3b, - 0xb8, 0x3a, 0x3d, 0x65, 0xff, 0x76, 0xed, 0xed, 0xfe, 0x78, 0x24, 0xf2, 0x41, 0x65, 0xd0, 0xad, - 0x79, 0x60, 0x93, 0x3a, 0x0f, 0x6c, 0xe6, 0x80, 0xc8, 0xbb, 0x50, 0xd8, 0xd3, 0x8d, 0x43, 0xcb, - 0x19, 0x6a, 0x0c, 0x16, 0xd8, 0x6b, 0x93, 0xad, 0xaf, 0x9e, 0x85, 0x8d, 0xbc, 0x90, 0x63, 0x2b, - 0x74, 0x15, 0x94, 0x91, 0x6b, 0x6a, 0xa1, 0x35, 0x12, 0x80, 0x8b, 0x33, 0x23, 0xd7, 0x1c, 0x58, - 0x23, 0x82, 0x5e, 0x82, 0x7c, 0xbc, 0xe8, 0x8b, 0x0a, 0x4b, 0x77, 0x2e, 0x56, 0xe6, 0x95, 0x8f, - 0x20, 0x23, 0x0e, 0x45, 0xdb, 0x09, 0x4f, 0xf7, 0xc3, 0xb7, 0x59, 0x64, 0xd3, 0x98, 0x2f, 0x22, - 0xea, 0x16, 0x0b, 0xa5, 0xa0, 0x6e, 0x45, 0xd4, 0x77, 0x58, 0x00, 0x33, 0x9c, 0xfa, 0x4e, 0xe5, - 0x0f, 0x09, 0xc8, 0x61, 0xa2, 0x9b, 0x98, 0xfc, 0x7c, 0x4c, 0x82, 0x10, 0x6d, 0x40, 0xfa, 0x80, - 0xe8, 0x26, 0xf1, 0x45, 0xbd, 0xa8, 0xd3, 0x80, 0xdc, 0x66, 0x74, 0x2c, 0xf8, 0xf1, 0xbc, 0x26, - 0x9e, 0x90, 0xd7, 0x0a, 0xa4, 0x5d, 0x96, 0xa6, 0x39, 0x89, 0x13, 0x1c, 0xea, 0xda, 0x9e, 0xed, - 0x1a, 0x87, 0x2c, 0x7b, 0x0a, 0xe6, 0x0b, 0xb4, 0x0e, 0x79, 0xd3, 0xd5, 0x1c, 0x37, 0xd4, 0x3c, - 0xdf, 0x3d, 0x3a, 0x66, 0x19, 0x52, 0x30, 0x98, 0x6e, 0xc7, 0x0d, 0x7b, 0x94, 0x42, 0x8b, 0x71, - 0x44, 0x42, 0xdd, 0xd4, 0x43, 0x5d, 0x73, 0x1d, 0xfb, 0x98, 0xc5, 0x5f, 0xc1, 0xf9, 0x88, 0xd8, - 0x75, 0xec, 0x63, 0x74, 0x1d, 0x80, 0x5e, 0x5e, 0xc2, 0x89, 0xcc, 0x19, 0x27, 0xb2, 0xc4, 0x31, - 0xf9, 0x23, 0x7a, 0x05, 0x96, 0x59, 0xa9, 0x69, 0x93, 0xec, 0x28, 0x2c, 0x3b, 0x79, 0x46, 0xbd, - 0xcb, 0x53, 0x54, 0xf9, 0x6d, 0x02, 0xf2, 0x3c, 0x64, 0x81, 0xe7, 0x3a, 0x01, 0xa1, 0x31, 0x0b, - 0x42, 0x3d, 0x1c, 0x07, 0x2c, 0x66, 0xcb, 0xf1, 0x98, 0xf5, 0x19, 0x1d, 0x0b, 0x7e, 0x2c, 0xba, - 0x89, 0x73, 0xa2, 0xfb, 0x34, 0x61, 0xbb, 0x0e, 0xf0, 0x99, 0x6f, 0x85, 0x44, 0xa3, 0x3a, 0x2c, - 0x76, 0x33, 0x27, 0x63, 0x5c, 0x6a, 0x18, 0x55, 0x63, 0x1d, 0x48, 0x6a, 0xb6, 0xab, 0x89, 0x4a, - 0x35, 0xd6, 0x5a, 0xbc, 0x04, 0xf9, 0xe8, 0x59, 0x1b, 0xfb, 0xfc, 0x3e, 0xc8, 0xe2, 0x5c, 0x44, - 0xdb, 0xf5, 0x6d, 0x54, 0x84, 0x8c, 0xe1, 0x3a, 0xf4, 0x0a, 0x61, 0x41, 0xcd, 0xe3, 0x68, 0x59, - 0xf9, 0x32, 0x09, 0x05, 0xd1, 0x17, 0x5c, 0x54, 0x55, 0xcd, 0xd6, 0x46, 0xf2, 0x4c, 0x6d, 0x4c, - 0x03, 0x98, 0x5a, 0x18, 0xc0, 0x0f, 0x61, 0xc5, 0x38, 0x20, 0xc6, 0xa1, 0xe6, 0x93, 0xa1, 0x15, - 0x84, 0xc4, 0x0f, 0xc4, 0xc5, 0x77, 0xe5, 0x4c, 0xcb, 0xc7, 0x9b, 0x61, 0xbc, 0xcc, 0xe4, 0x71, - 0x24, 0x8e, 0x7e, 0x04, 0x2b, 0x63, 0x87, 0x82, 0xc8, 0xd4, 0x42, 0x66, 0x51, 0xd3, 0x88, 0x97, - 0x99, 0xe8, 0x54, 0xb9, 0x06, 0x28, 0x18, 0xef, 0x85, 0xbe, 0x6e, 0x84, 0x31, 0x7d, 0x65, 0xa1, - 0xfe, 0x6a, 0x24, 0x3d, 0x35, 0x11, 0x4b, 0x82, 0x7c, 0x2a, 0x09, 0xe2, 0xee, 0xfa, 0x4d, 0x02, - 0x96, 0xa3, 0x54, 0x3c, 0x73, 0xb5, 0x56, 0xcf, 0xab, 0x56, 0x01, 0xaa, 0x51, 0xee, 0x6e, 0x40, - 0xda, 0x70, 0x47, 0xf4, 0x52, 0x48, 0x2e, 0x2c, 0x31, 0x21, 0x81, 0xde, 0xa2, 0xad, 0x4c, 0x74, - 0x64, 0x79, 0xe1, 0x91, 0xa7, 0x42, 0xb4, 0x24, 0x43, 0x37, 0xd4, 0x6d, 0xcd, 0x38, 0x18, 0x3b, - 0x87, 0x01, 0x4f, 0x2b, 0xce, 0x31, 0xda, 0x36, 0x23, 0xa1, 0x57, 0x61, 0xd9, 0x24, 0xb6, 0x7e, - 0x4c, 0xcc, 0x48, 0x28, 0xcd, 0x84, 0x0a, 0x82, 0xca, 0xc5, 0x2a, 0x7f, 0x4e, 0x80, 0x8a, 0x45, - 0xc3, 0x4f, 0x9e, 0xbd, 0x44, 0xab, 0x40, 0x47, 0x44, 0xcf, 0x0d, 0x74, 0xfb, 0x09, 0x07, 0x9d, - 0xc8, 0x9c, 0x3e, 0x6a, 0xe6, 0x69, 0x8e, 0xba, 0x0e, 0x39, 0xdd, 0x38, 0x74, 0xdc, 0xcf, 0x6c, - 0x62, 0x0e, 0x89, 0x40, 0xb5, 0x38, 0x09, 0xdd, 0x04, 0x64, 0x12, 0xcf, 0x27, 0xf4, 0x04, 0xa6, - 0xf6, 0x84, 0x37, 0x66, 0x75, 0x2a, 0x26, 0x48, 0x8b, 0x6b, 0x86, 0xe2, 0xa9, 0x78, 0xd4, 0x4c, - 0x62, 0x87, 0xba, 0x88, 0x71, 0x5e, 0x10, 0x1b, 0x94, 0x56, 0xf9, 0xab, 0x04, 0xab, 0xb1, 0xe8, - 0x5d, 0x20, 0x06, 0xc6, 0x41, 0x2b, 0xf9, 0x14, 0xa0, 0xf5, 0xcc, 0x35, 0x55, 0x19, 0x40, 0xae, - 0x6d, 0x05, 0x61, 0x54, 0x03, 0x3f, 0x04, 0x25, 0x10, 0x6f, 0xba, 0xa8, 0x82, 0x45, 0x40, 0x20, - 0x2a, 0x7f, 0x22, 0x7e, 0x47, 0x56, 0x12, 0x6a, 0xf2, 0x8e, 0xac, 0x24, 0x55, 0xb9, 0xf2, 0x9f, - 0x04, 0xe4, 0xb9, 0xd9, 0x0b, 0x7f, 0xe5, 0x3e, 0x04, 0x45, 0x24, 0x9f, 0x0f, 0x32, 0xa7, 0x26, - 0xcb, 0xb8, 0x0f, 0xd1, 0x98, 0x19, 0x39, 0x1e, 0x69, 0x95, 0xfe, 0x28, 0x41, 0x54, 0x2c, 0x68, - 0x13, 0xe4, 0xf9, 0xad, 0x62, 0x6c, 0x80, 0x14, 0x06, 0x98, 0x20, 0x7d, 0x27, 0xe9, 0x55, 0xe9, - 0x93, 0x07, 0x56, 0x10, 0x0d, 0xd9, 0x49, 0x9c, 0x1b, 0xb9, 0x26, 0x16, 0x24, 0xf4, 0x3a, 0xa4, - 0x7c, 0x77, 0x1c, 0x12, 0x91, 0xc1, 0xd8, 0x97, 0x09, 0x4c, 0xc9, 0xc2, 0x1c, 0x97, 0x41, 0xff, - 0x07, 0x2b, 0x86, 0x4f, 0xf4, 0x90, 0x4c, 0x4d, 0xb2, 0x6b, 0x0d, 0x2f, 0x73, 0x72, 0x64, 0xf5, - 0x8e, 0xac, 0xc8, 0x6a, 0xaa, 0xf2, 0x0f, 0x09, 0xf2, 0x35, 0xcf, 0xb3, 0x8f, 0xa3, 0x04, 0x7e, - 0x00, 0x19, 0xe3, 0x40, 0x77, 0x86, 0x24, 0xfa, 0x10, 0x72, 0x6d, 0xba, 0x5d, 0x5c, 0xb0, 0xba, - 0xcd, 0xa4, 0xa2, 0x4f, 0x10, 0x42, 0xa7, 0xf4, 0x95, 0x04, 0x69, 0xce, 0x41, 0x55, 0x78, 0x8e, - 0x1c, 0x79, 0xc4, 0x08, 0xb5, 0x53, 0x07, 0x64, 0xc3, 0x2c, 0x5e, 0xe5, 0xac, 0xbb, 0xb1, 0x63, - 0xbe, 0x09, 0xe9, 0xb1, 0x17, 0x10, 0x3f, 0x14, 0x89, 0x9b, 0x1f, 0x3c, 0x2c, 0x84, 0xd0, 0xcb, - 0x90, 0x36, 0x89, 0x4d, 0x44, 0x58, 0x66, 0xde, 0x59, 0xc1, 0xaa, 0x58, 0xec, 0x1a, 0xa5, 0x4e, - 0x5f, 0x74, 0x1d, 0x55, 0xfe, 0x99, 0x00, 0x35, 0x7a, 0xa3, 0x82, 0x0b, 0xbb, 0xb5, 0xcf, 0xf6, - 0x57, 0xc9, 0xb3, 0xfd, 0x15, 0xbd, 0xdb, 0x69, 0xc3, 0x36, 0x91, 0xe1, 0x15, 0x40, 0x9b, 0xb8, - 0x48, 0xe2, 0x35, 0x58, 0x71, 0xc8, 0x51, 0xa8, 0x79, 0xfa, 0x90, 0x68, 0xa1, 0x7b, 0x48, 0x1c, - 0x81, 0x54, 0x05, 0x4a, 0xee, 0xe9, 0x43, 0x32, 0xa0, 0x44, 0x74, 0x0d, 0x80, 0x89, 0xf0, 0x49, - 0x85, 0xc2, 0x68, 0x0a, 0x67, 0x29, 0x85, 0x8d, 0x29, 0xe8, 0x16, 0xe4, 0x03, 0x6b, 0xe8, 0xe8, - 0xe1, 0xd8, 0x27, 0x83, 0x41, 0x5b, 0x60, 0xf3, 0x13, 0x86, 0x5e, 0xe5, 0xe1, 0x49, 0x59, 0x62, - 0x13, 0xed, 0x29, 0xc5, 0x33, 0xdd, 0x88, 0x32, 0xdb, 0x8d, 0x54, 0xfe, 0x94, 0x80, 0xd5, 0x58, - 0x7c, 0x2f, 0x1c, 0x17, 0x5a, 0x90, 0x8d, 0x60, 0x31, 0x02, 0x86, 0x57, 0xcf, 0x62, 0xe7, 0xc4, - 0x93, 0xaa, 0x36, 0xf9, 0xf2, 0xc4, 0xed, 0x4c, 0xb5, 0xe7, 0x05, 0x5b, 0x9e, 0x13, 0xec, 0xd2, - 0x27, 0x90, 0x9d, 0x58, 0x41, 0x6f, 0x9c, 0x42, 0x92, 0x39, 0xb0, 0x7d, 0x0a, 0x46, 0xae, 0x01, - 0xd0, 0x78, 0x12, 0x93, 0xf5, 0x9a, 0x7c, 0xc2, 0xcd, 0x72, 0xca, 0xae, 0x6f, 0x57, 0x7e, 0x25, - 0x41, 0x8a, 0x81, 0x05, 0x7a, 0x1f, 0x32, 0x23, 0x32, 0xda, 0xa3, 0xf8, 0xce, 0xdf, 0xef, 0xf3, - 0xe6, 0xef, 0x48, 0x9c, 0x5e, 0x7a, 0x9e, 0x6f, 0x8d, 0x74, 0xff, 0x98, 0x7f, 0x09, 0xc4, 0xd1, - 0x12, 0xdd, 0x80, 0x6c, 0x34, 0x80, 0x47, 0x1f, 0x83, 0x4e, 0xcf, 0xe7, 0x53, 0xb6, 0x68, 0xaa, - 0x7e, 0x9f, 0x80, 0x34, 0x8f, 0x3a, 0xfa, 0x00, 0x20, 0x1a, 0xb2, 0x9f, 0xfa, 0x9b, 0x40, 0x56, - 0x68, 0xb4, 0xcc, 0x29, 0x38, 0x26, 0x9e, 0x02, 0x1c, 0x37, 0x41, 0x26, 0xa1, 0x61, 0x0a, 0x20, - 0xbd, 0x34, 0x5b, 0x01, 0xd5, 0x66, 0x68, 0x98, 0x51, 0x58, 0xa9, 0x60, 0xe9, 0x0b, 0x09, 0x64, - 0x4a, 0xa4, 0xf1, 0x35, 0xec, 0x31, 0xbd, 0xf2, 0x22, 0x2f, 0x65, 0x9c, 0x15, 0x94, 0x96, 0x89, - 0x5e, 0x80, 0x2c, 0x0f, 0x13, 0xe5, 0x26, 0x18, 0x57, 0xe1, 0x84, 0x96, 0x89, 0x4a, 0xa0, 0x4c, - 0xd0, 0x8f, 0xbf, 0xad, 0x93, 0x35, 0x55, 0xf4, 0xf5, 0xfd, 0x50, 0x0b, 0x89, 0xcf, 0x27, 0x6f, - 0x19, 0x2b, 0x94, 0x30, 0x20, 0xfe, 0x28, 0xfa, 0x34, 0x41, 0x7f, 0x6f, 0x7c, 0x97, 0x80, 0x34, - 0xaf, 0x68, 0x94, 0x86, 0x44, 0xf7, 0x23, 0x75, 0x09, 0x5d, 0x82, 0xd5, 0x3b, 0xdd, 0x5d, 0xdc, - 0xa9, 0xb5, 0xb5, 0x4e, 0x77, 0xa0, 0xed, 0x74, 0x77, 0x3b, 0x0d, 0x55, 0x42, 0xd7, 0xe0, 0x6a, - 0xa7, 0xab, 0x45, 0x9c, 0x1e, 0x6e, 0xdd, 0xad, 0xe1, 0xfb, 0x5a, 0x1d, 0x77, 0x3f, 0x6a, 0x62, - 0x35, 0x81, 0xd6, 0xa0, 0x44, 0xa5, 0x17, 0xf0, 0x93, 0xe8, 0x32, 0xa0, 0x38, 0x5f, 0xd0, 0x53, - 0x68, 0x1d, 0x5e, 0x6c, 0x75, 0xfa, 0xbb, 0x3b, 0x3b, 0xad, 0xed, 0x56, 0xb3, 0x33, 0x2b, 0xd0, - 0x57, 0x65, 0xf4, 0x22, 0x14, 0xbb, 0x3b, 0x3b, 0xfd, 0xe6, 0x80, 0xb9, 0x73, 0xbf, 0x39, 0xd0, - 0x6a, 0x1f, 0xd7, 0x5a, 0xed, 0x5a, 0xbd, 0xdd, 0x54, 0xd3, 0x68, 0x05, 0x72, 0xf7, 0x70, 0xb7, - 0x73, 0x4b, 0xc3, 0xdd, 0xdd, 0x41, 0x53, 0xcd, 0x50, 0xf7, 0x7b, 0xb8, 0xdb, 0xeb, 0xf6, 0x6b, - 0x6d, 0xed, 0x6e, 0xab, 0x7f, 0xb7, 0x36, 0xd8, 0xbe, 0xad, 0x2a, 0xe8, 0x05, 0xb8, 0xd2, 0x1c, - 0x6c, 0x37, 0xb4, 0x01, 0xae, 0x75, 0xfa, 0xb5, 0xed, 0x41, 0xab, 0xdb, 0xd1, 0x76, 0x6a, 0xad, - 0x76, 0xb3, 0xa1, 0x66, 0xa9, 0x11, 0x6a, 0xbb, 0xd6, 0x6e, 0x77, 0xef, 0x35, 0x1b, 0x2a, 0xa0, - 0x2b, 0xf0, 0x1c, 0xb7, 0x5a, 0xeb, 0xf5, 0x9a, 0x9d, 0x86, 0xc6, 0x1d, 0x50, 0x73, 0xd4, 0x99, - 0x56, 0xa7, 0xd1, 0xfc, 0x44, 0xbb, 0x5d, 0xeb, 0x6b, 0xb7, 0x70, 0xb3, 0x36, 0x68, 0xe2, 0x88, - 0x9b, 0xa7, 0x7b, 0xe3, 0xe6, 0xad, 0x56, 0x9f, 0x12, 0x27, 0x7b, 0x17, 0x6e, 0x38, 0xa0, 0xce, - 0x7e, 0xb3, 0x40, 0x39, 0xc8, 0xb4, 0x3a, 0x1f, 0xd7, 0xda, 0xad, 0x86, 0xba, 0x84, 0x14, 0x90, - 0x3b, 0xdd, 0x4e, 0x53, 0x95, 0xe8, 0xd3, 0xad, 0x4f, 0x5b, 0x3d, 0x35, 0x81, 0x0a, 0x90, 0xfd, - 0xb4, 0x3f, 0xa8, 0x75, 0x1a, 0x35, 0xdc, 0x50, 0x93, 0x08, 0x20, 0xdd, 0xef, 0xd4, 0x7a, 0xbd, - 0xfb, 0xaa, 0x4c, 0x63, 0x4d, 0x85, 0xe8, 0xbe, 0xed, 0x6e, 0xad, 0xa1, 0x35, 0x9a, 0xdb, 0xdd, - 0xbb, 0x3d, 0xdc, 0xec, 0xf7, 0x5b, 0xdd, 0x8e, 0x9a, 0xda, 0xfa, 0x45, 0x72, 0xda, 0x39, 0xfc, - 0x00, 0x64, 0xda, 0x6d, 0xa0, 0x4b, 0xb3, 0xdd, 0x07, 0xbb, 0x49, 0x4a, 0x97, 0xe7, 0x37, 0x25, - 0xe8, 0x7d, 0x48, 0xb1, 0x1b, 0x0e, 0x5d, 0x9e, 0x7f, 0x4f, 0x97, 0xae, 0x9c, 0xa1, 0x0b, 0xcd, - 0xf7, 0x40, 0xa6, 0x33, 0x78, 0x7c, 0xc3, 0xd8, 0x67, 0x8c, 0xf8, 0x86, 0xf1, 0x51, 0xfd, 0x2d, - 0x09, 0x7d, 0x00, 0x69, 0x3e, 0x10, 0xa1, 0xd3, 0xb6, 0xa7, 0xd3, 0x6a, 0xa9, 0x78, 0x96, 0xc1, - 0xd5, 0x37, 0x24, 0x74, 0x1b, 0xb2, 0x93, 0xe6, 0x17, 0x95, 0xe2, 0xbb, 0x9c, 0x9e, 0x27, 0x4a, - 0x2f, 0xcc, 0xe5, 0x45, 0x76, 0xde, 0xa2, 0x96, 0x0a, 0x34, 0x16, 0x13, 0x2c, 0x8e, 0x5b, 0x9b, - 0xbd, 0x8a, 0xe3, 0xd6, 0xce, 0x80, 0x77, 0xbd, 0xf9, 0xf0, 0xdf, 0x6b, 0x4b, 0x0f, 0xbf, 0x5b, - 0x93, 0xbe, 0xfd, 0x6e, 0x4d, 0xfa, 0xf5, 0xa3, 0xb5, 0xa5, 0x6f, 0x1e, 0xad, 0x49, 0x7f, 0x79, - 0xb4, 0x26, 0x7d, 0xfb, 0x68, 0x6d, 0xe9, 0x6f, 0x8f, 0xd6, 0x96, 0x3e, 0x7d, 0x79, 0xe8, 0x56, - 0x87, 0xfa, 0xe7, 0x24, 0x0c, 0x49, 0xd5, 0x24, 0x0f, 0x36, 0x0d, 0xd7, 0x27, 0x9b, 0x33, 0xff, - 0x83, 0xed, 0xa5, 0xd9, 0xd3, 0x3b, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x13, 0xd4, 0xf1, 0x26, - 0x21, 0x1b, 0x00, 0x00, + 0x15, 0x16, 0x48, 0x90, 0x04, 0x1f, 0x49, 0x09, 0xda, 0xc4, 0xb6, 0xcc, 0xc4, 0xa2, 0xc2, 0xfc, + 0x54, 0x76, 0x12, 0xca, 0x91, 0x3b, 0x49, 0xea, 0x4e, 0xda, 0x90, 0x22, 0x65, 0xd3, 0xa1, 0x49, + 0xce, 0x92, 0x8a, 0xe3, 0x1c, 0x8a, 0x81, 0x80, 0x15, 0x85, 0x0a, 0x04, 0x50, 0x00, 0x74, 0xa4, + 0xdc, 0x72, 0x6a, 0x0e, 0xe9, 0x4c, 0xa7, 0xa7, 0x9c, 0xda, 0x5c, 0x7a, 0x6e, 0x67, 0x7a, 0x6b, + 0x2f, 0x3d, 0xba, 0xb7, 0x1c, 0x3b, 0xd3, 0x56, 0x9d, 0xc6, 0x33, 0x9d, 0x9e, 0x7d, 0xcc, 0xa9, + 0xb3, 0x3f, 0x20, 0x21, 0xfe, 0x58, 0xf6, 0xc1, 0x17, 0xcd, 0xee, 0x7b, 0xdf, 0xbe, 0xdd, 0x7d, + 0xef, 0xf1, 0x7b, 0x6f, 0x21, 0x58, 0xdf, 0xf7, 0xdd, 0x23, 0xe2, 0x6f, 0x79, 0xbe, 0x1b, 0xba, + 0x86, 0x6b, 0x8f, 0x07, 0x15, 0x36, 0x40, 0x4a, 0x34, 0x2f, 0xbe, 0x38, 0x70, 0x07, 0x2e, 0x9b, + 0x6d, 0xd1, 0x11, 0xd7, 0x17, 0xd7, 0x07, 0xae, 0x3b, 0xb0, 0x09, 0x5f, 0xb6, 0x3f, 0x3a, 0xd8, + 0x32, 0x47, 0xbe, 0x1e, 0x5a, 0xae, 0xc3, 0xf5, 0xe5, 0x03, 0x48, 0xb5, 0xf4, 0x7d, 0x62, 0x23, + 0x04, 0xb2, 0xa3, 0x0f, 0xc9, 0x9a, 0xb4, 0x21, 0x6d, 0x66, 0x31, 0x1b, 0xa3, 0x17, 0x21, 0xf5, + 0x40, 0xb7, 0x47, 0x64, 0x2d, 0xc1, 0x84, 0x7c, 0x82, 0xde, 0x86, 0xb4, 0xe7, 0x93, 0x03, 0xeb, + 0x78, 0x2d, 0xb9, 0x21, 0x6d, 0x2a, 0xb5, 0x0b, 0x8f, 0x4f, 0x4b, 0xab, 0x27, 0xfa, 0xd0, 0xbe, + 0x59, 0x7e, 0xcb, 0x1d, 0x5a, 0x21, 0x19, 0x7a, 0xe1, 0x49, 0x19, 0x0b, 0xd0, 0x4d, 0xf9, 0x7f, + 0xdf, 0x94, 0xa4, 0x72, 0x1f, 0x14, 0xb6, 0x4f, 0x8f, 0x84, 0xa8, 0x06, 0x69, 0x9b, 0x8e, 0x83, + 0x35, 0x69, 0x23, 0xb9, 0x99, 0xdb, 0x5e, 0xa9, 0x8c, 0x2f, 0xc5, 0x30, 0xb5, 0xcb, 0x0f, 0x4f, + 0x4b, 0x4b, 0x0b, 0xac, 0xf2, 0x95, 0xc2, 0xea, 0x17, 0x12, 0x14, 0x84, 0x59, 0x9b, 0x18, 0xa1, + 0xeb, 0xa3, 0x6d, 0xc8, 0x58, 0x8e, 0x61, 0x8f, 0x4c, 0x7e, 0x93, 0xdc, 0x36, 0x9a, 0x32, 0xde, + 0x23, 0x61, 0x4d, 0xa6, 0xf6, 0x71, 0x04, 0xa4, 0x6b, 0xc8, 0x31, 0x5f, 0x93, 0x38, 0x6f, 0x8d, + 0x00, 0xde, 0x54, 0xbe, 0xfe, 0xa6, 0xb4, 0xc4, 0xce, 0xf0, 0x55, 0x16, 0x72, 0x77, 0xdc, 0x91, + 0xef, 0xe8, 0x76, 0xcf, 0x23, 0x06, 0xfa, 0x61, 0xdc, 0x91, 0xb5, 0x8d, 0xb9, 0xd7, 0xf8, 0xfe, + 0xb4, 0x94, 0x11, 0x6b, 0x84, 0xab, 0xdf, 0x83, 0x9c, 0x4f, 0x3c, 0xdb, 0x32, 0x58, 0x70, 0xd8, + 0x39, 0x52, 0x8b, 0x3c, 0x1b, 0x47, 0xa2, 0xee, 0xd8, 0x99, 0xc9, 0x85, 0x67, 0x7f, 0x8d, 0x9e, + 0xfd, 0xdb, 0xd3, 0x92, 0xf4, 0xf8, 0xb4, 0xb4, 0x36, 0x6d, 0xef, 0x2d, 0xcb, 0xb1, 0x2d, 0x87, + 0x8c, 0x5d, 0x8b, 0xf6, 0x40, 0x39, 0xf0, 0xf5, 0xc1, 0x90, 0x38, 0xe1, 0x9a, 0xcc, 0x6c, 0xae, + 0x4f, 0x6c, 0xc6, 0x6e, 0x5a, 0xd9, 0x15, 0xa8, 0x27, 0xc5, 0x6b, 0x6c, 0x0a, 0xfd, 0x14, 0x52, + 0x07, 0xb6, 0x3e, 0x08, 0xd6, 0xd2, 0x1b, 0xd2, 0x66, 0xa1, 0x76, 0x75, 0x91, 0x63, 0xd4, 0xd8, + 0x16, 0xda, 0xae, 0xad, 0x0f, 0x30, 0x5f, 0x87, 0x5a, 0xb0, 0x32, 0xd4, 0x8f, 0x35, 0xdd, 0xf3, + 0x88, 0x63, 0x6a, 0xbe, 0x1e, 0x92, 0xb5, 0xcc, 0x86, 0xb4, 0x99, 0xac, 0xbd, 0xf6, 0xf8, 0xb4, + 0xb4, 0xc1, 0x4d, 0x4d, 0x01, 0xe2, 0x27, 0x29, 0x0c, 0xf5, 0xe3, 0x2a, 0x53, 0x61, 0x3d, 0x24, + 0xc5, 0xaf, 0x52, 0xa0, 0x44, 0x17, 0xa0, 0x29, 0x6d, 0x13, 0x67, 0x10, 0x1e, 0xb2, 0xa8, 0x25, + 0x17, 0xa6, 0x34, 0x07, 0x21, 0x17, 0x56, 0x0d, 0x77, 0xe8, 0xf9, 0x24, 0x08, 0x2c, 0xd7, 0xd1, + 0x0c, 0xd7, 0x24, 0x06, 0x0b, 0xd9, 0xf2, 0x76, 0x71, 0xe2, 0xaa, 0x9d, 0x09, 0x64, 0x87, 0x22, + 0x6a, 0x6f, 0x3c, 0x3e, 0x2d, 0x95, 0xb9, 0xd5, 0x99, 0xe5, 0xf1, 0x6d, 0x54, 0x63, 0x6a, 0x25, + 0xfa, 0x09, 0xa4, 0x83, 0xd0, 0xf5, 0x09, 0x0d, 0x72, 0x72, 0x33, 0xcb, 0x2c, 0xcd, 0x75, 0x5e, + 0x21, 0xba, 0x52, 0x8f, 0xc2, 0xb1, 0x58, 0x85, 0x02, 0x50, 0x7d, 0x72, 0xe0, 0x93, 0xe0, 0x50, + 0xb3, 0x9c, 0x90, 0xf8, 0x0f, 0x74, 0x5b, 0x84, 0xf6, 0x72, 0x85, 0x13, 0x44, 0x25, 0x22, 0x88, + 0x4a, 0x5d, 0x10, 0x44, 0xed, 0x6d, 0x11, 0xd5, 0x57, 0xf8, 0x46, 0xd3, 0x06, 0x62, 0x1b, 0x7f, + 0xfd, 0xef, 0x92, 0x84, 0x57, 0x04, 0xa0, 0x29, 0xf4, 0xe8, 0x63, 0xc8, 0xfa, 0x24, 0x24, 0x0e, + 0x4b, 0xe8, 0xd4, 0x79, 0xbb, 0x5d, 0x59, 0x98, 0x43, 0xcc, 0xfa, 0xc4, 0x14, 0x1a, 0xc2, 0xf2, + 0x81, 0x3d, 0x8a, 0x5f, 0x25, 0x7d, 0x9e, 0xf1, 0x37, 0x85, 0xf1, 0x12, 0x37, 0x7e, 0x76, 0xf9, + 0xf4, 0x56, 0x05, 0xa6, 0x1e, 0x5f, 0xe3, 0x67, 0x70, 0xc1, 0xd3, 0xc3, 0x43, 0xcd, 0x73, 0x83, + 0xf0, 0xc0, 0x3a, 0xd6, 0x28, 0xd4, 0x8e, 0x92, 0x2f, 0x5b, 0xbb, 0xf6, 0xf8, 0xb4, 0xf4, 0x06, + 0x37, 0x3b, 0x17, 0x16, 0x0f, 0xec, 0x0b, 0x14, 0xd1, 0xe5, 0x80, 0xbe, 0xd0, 0x0b, 0x26, 0xab, + 0x82, 0x4c, 0x73, 0x1d, 0xad, 0x42, 0xa1, 0xdd, 0xe9, 0x6b, 0xbd, 0x6e, 0x63, 0xa7, 0xb9, 0xdb, + 0x6c, 0xd4, 0xd5, 0x25, 0x94, 0x07, 0xa5, 0xa3, 0xe1, 0x7a, 0xa7, 0xdd, 0xba, 0xaf, 0x4a, 0x7c, + 0x76, 0x0f, 0xb3, 0x59, 0x02, 0x01, 0xa4, 0xa9, 0xee, 0x1e, 0x56, 0x65, 0x61, 0xe8, 0xf7, 0x12, + 0xe4, 0xba, 0xbe, 0x6b, 0x90, 0x20, 0x60, 0x74, 0x54, 0x81, 0x84, 0x65, 0x0a, 0x2e, 0x5c, 0x9b, + 0x24, 0x67, 0x0c, 0x52, 0x69, 0xd6, 0x05, 0xbb, 0x25, 0x2c, 0x13, 0x6d, 0x82, 0x42, 0x1c, 0xd3, + 0x73, 0x2d, 0x27, 0xe4, 0xb4, 0x5f, 0xcb, 0x7f, 0x7f, 0x5a, 0x52, 0x1a, 0x42, 0x86, 0xc7, 0xda, + 0xe2, 0xbb, 0x90, 0x68, 0xd6, 0x69, 0xdd, 0xf8, 0xdc, 0x75, 0xc6, 0x75, 0x83, 0x8e, 0xd1, 0x45, + 0x48, 0x07, 0xa3, 0x03, 0x5a, 0x21, 0x78, 0xe1, 0x10, 0x33, 0x7e, 0xc2, 0x9b, 0xf2, 0x97, 0xf4, + 0x9c, 0xbf, 0x94, 0x00, 0x6a, 0xac, 0xb6, 0xb1, 0x63, 0xf6, 0x21, 0xef, 0xf1, 0x23, 0x69, 0x81, + 0x47, 0x0c, 0x71, 0xe0, 0x0b, 0x73, 0x0f, 0x5c, 0x2b, 0xc6, 0xf8, 0x6c, 0x59, 0xe4, 0x4b, 0xc4, + 0x62, 0x39, 0x2f, 0x76, 0xf9, 0x57, 0xa1, 0xf0, 0x73, 0xce, 0x26, 0x9a, 0x6d, 0x0d, 0x2d, 0x7e, + 0xa3, 0x02, 0xce, 0x0b, 0x61, 0x8b, 0xca, 0xca, 0xff, 0x48, 0xc4, 0x98, 0xe0, 0x75, 0xc8, 0x08, + 0xa5, 0x20, 0xf0, 0x5c, 0x9c, 0xab, 0x23, 0x1d, 0xda, 0x80, 0xd4, 0x3e, 0x19, 0x58, 0x9c, 0xa8, + 0x93, 0x35, 0xf8, 0xfe, 0xb4, 0x94, 0xee, 0x1c, 0x1c, 0x04, 0x24, 0xc4, 0x5c, 0x81, 0x5e, 0x86, + 0x24, 0x71, 0x4c, 0x46, 0xca, 0x67, 0xf5, 0x54, 0x8c, 0xae, 0x42, 0x32, 0x18, 0x0d, 0xc5, 0x6f, + 0x70, 0x75, 0x72, 0xcb, 0xde, 0xed, 0xea, 0x3b, 0xbd, 0xd1, 0x50, 0xc4, 0x83, 0x62, 0xd0, 0xad, + 0x79, 0x64, 0x93, 0x3a, 0x8f, 0x6c, 0xe6, 0x90, 0xc8, 0xbb, 0x50, 0xd8, 0xd7, 0x8d, 0x23, 0xcb, + 0x19, 0x68, 0x8c, 0x16, 0xd8, 0xcf, 0x26, 0x5b, 0x5b, 0x9d, 0xa5, 0x8d, 0xbc, 0xc0, 0xb1, 0x19, + 0xba, 0x0c, 0xca, 0xd0, 0x35, 0xb5, 0xd0, 0x1a, 0x0a, 0xc2, 0xc5, 0x99, 0xa1, 0x6b, 0xf6, 0xad, + 0x21, 0x41, 0xaf, 0x40, 0x3e, 0x9e, 0xf4, 0x6b, 0x0a, 0x0b, 0x77, 0x2e, 0x96, 0xe6, 0xe5, 0x8f, + 0x20, 0x23, 0x2e, 0x45, 0xdb, 0x09, 0x4f, 0xf7, 0xc3, 0x77, 0x98, 0x67, 0xd3, 0x98, 0x4f, 0x22, + 0xe9, 0x36, 0x73, 0xa5, 0x90, 0x6e, 0x47, 0xd2, 0x1b, 0xcc, 0x81, 0x19, 0x2e, 0xbd, 0x51, 0xfe, + 0x63, 0x02, 0x72, 0x98, 0xe8, 0x26, 0x26, 0xbf, 0x18, 0x91, 0x20, 0x44, 0x9b, 0x90, 0x3e, 0x24, + 0xba, 0x49, 0x7c, 0x91, 0x2f, 0xea, 0xc4, 0x21, 0xb7, 0x99, 0x1c, 0x0b, 0x7d, 0x3c, 0xae, 0x89, + 0x27, 0xc4, 0xb5, 0x0c, 0x69, 0x97, 0x85, 0x69, 0x4e, 0xe0, 0x84, 0x86, 0x1e, 0x6d, 0xdf, 0x76, + 0x8d, 0x23, 0x16, 0x3d, 0x05, 0xf3, 0x09, 0xda, 0x80, 0xbc, 0xe9, 0x6a, 0x8e, 0x1b, 0x6a, 0x9e, + 0xef, 0x1e, 0x9f, 0xb0, 0x08, 0x29, 0x18, 0x4c, 0xb7, 0xed, 0x86, 0x5d, 0x2a, 0xa1, 0xc9, 0x38, + 0x24, 0xa1, 0x6e, 0xea, 0xa1, 0xae, 0xb9, 0x8e, 0x7d, 0xc2, 0xfc, 0xaf, 0xe0, 0x7c, 0x24, 0xec, + 0x38, 0xf6, 0x09, 0xba, 0x0a, 0x40, 0x8b, 0x97, 0x38, 0x44, 0x66, 0xe6, 0x10, 0x59, 0xe2, 0x98, + 0x7c, 0x88, 0x5e, 0x83, 0x65, 0x96, 0x6a, 0xda, 0x38, 0x3a, 0x0a, 0x8b, 0x4e, 0x9e, 0x49, 0xef, + 0xf2, 0x10, 0x95, 0x7f, 0x9b, 0x80, 0x3c, 0x77, 0x59, 0xe0, 0xb9, 0x4e, 0x40, 0xa8, 0xcf, 0x82, + 0x50, 0x0f, 0x47, 0x01, 0xf3, 0xd9, 0x72, 0xdc, 0x67, 0x3d, 0x26, 0xc7, 0x42, 0x1f, 0xf3, 0x6e, + 0xe2, 0x1c, 0xef, 0x3e, 0x8d, 0xdb, 0xae, 0x02, 0x7c, 0xe6, 0x5b, 0x21, 0xd1, 0xe8, 0x1a, 0xe6, + 0xbb, 0xa9, 0x9b, 0x31, 0x2d, 0x35, 0x8c, 0x2a, 0xb1, 0x0e, 0x24, 0x35, 0xdd, 0xd5, 0x44, 0xa9, + 0x1a, 0x6b, 0x2d, 0x5e, 0x81, 0x7c, 0x34, 0xd6, 0x46, 0x3e, 0xaf, 0x07, 0x59, 0x9c, 0x8b, 0x64, + 0x7b, 0xbe, 0x8d, 0xd6, 0x20, 0x63, 0xb8, 0x0e, 0x2d, 0x21, 0xcc, 0xa9, 0x79, 0x1c, 0x4d, 0xcb, + 0x5f, 0x26, 0xa1, 0x20, 0xfa, 0x82, 0xe7, 0x95, 0x55, 0xd3, 0xb9, 0x91, 0x9c, 0xc9, 0x8d, 0x89, + 0x03, 0x53, 0x0b, 0x1d, 0xf8, 0x21, 0xac, 0x18, 0x87, 0xc4, 0x38, 0xd2, 0x7c, 0x32, 0xb0, 0x82, + 0x90, 0xf8, 0x81, 0x28, 0x7c, 0x97, 0x66, 0x5a, 0x3e, 0xde, 0x0c, 0xe3, 0x65, 0x86, 0xc7, 0x11, + 0x1c, 0xfd, 0x18, 0x56, 0x46, 0x0e, 0x25, 0x91, 0x89, 0x85, 0xcc, 0xa2, 0xa6, 0x11, 0x2f, 0x33, + 0xe8, 0x64, 0x71, 0x15, 0x50, 0x30, 0xda, 0x0f, 0x7d, 0xdd, 0x08, 0x63, 0xeb, 0x95, 0x85, 0xeb, + 0x57, 0x23, 0xf4, 0xc4, 0x44, 0x2c, 0x08, 0xf2, 0x99, 0x20, 0x88, 0xda, 0xf5, 0x9b, 0x04, 0x2c, + 0x47, 0xa1, 0x78, 0xe6, 0x6c, 0xad, 0x9c, 0x97, 0xad, 0x82, 0x54, 0xa3, 0xd8, 0x5d, 0x83, 0xb4, + 0xe1, 0x0e, 0x69, 0x51, 0x48, 0x2e, 0x4c, 0x31, 0x81, 0x40, 0xd7, 0x69, 0x2b, 0x13, 0x5d, 0x59, + 0x5e, 0x78, 0xe5, 0x09, 0x88, 0xa6, 0x64, 0xe8, 0x86, 0xba, 0xad, 0x19, 0x87, 0x23, 0xe7, 0x28, + 0xe0, 0x61, 0xc5, 0x39, 0x26, 0xdb, 0x61, 0x22, 0xf4, 0x3a, 0x2c, 0x9b, 0xc4, 0xd6, 0x4f, 0x88, + 0x19, 0x81, 0xd2, 0x0c, 0x54, 0x10, 0x52, 0x0e, 0x2b, 0xff, 0x25, 0x01, 0x2a, 0x16, 0x0d, 0x3f, + 0x79, 0xf6, 0x14, 0xad, 0x00, 0x7d, 0x22, 0x7a, 0x6e, 0xa0, 0xdb, 0x4f, 0xb8, 0xe8, 0x18, 0x73, + 0xf6, 0xaa, 0x99, 0xa7, 0xb9, 0xea, 0x06, 0xe4, 0x74, 0xe3, 0xc8, 0x71, 0x3f, 0xb3, 0x89, 0x39, + 0x20, 0x82, 0xd5, 0xe2, 0x22, 0x74, 0x13, 0x90, 0x49, 0x3c, 0x9f, 0xd0, 0x1b, 0x98, 0xda, 0x13, + 0x7e, 0x31, 0xab, 0x13, 0x98, 0x10, 0x2d, 0xce, 0x19, 0xca, 0xa7, 0x62, 0xa8, 0x99, 0xc4, 0x0e, + 0x75, 0xe1, 0xe3, 0xbc, 0x10, 0xd6, 0xa9, 0xac, 0xfc, 0x37, 0x09, 0x56, 0x63, 0xde, 0x7b, 0x8e, + 0x1c, 0x18, 0x27, 0xad, 0xe4, 0x53, 0x90, 0xd6, 0x33, 0xe7, 0x54, 0xf9, 0x77, 0x12, 0xe4, 0x5a, + 0x56, 0x10, 0x46, 0x49, 0xf0, 0x23, 0x50, 0x02, 0xf1, 0x53, 0x17, 0x69, 0xb0, 0x88, 0x09, 0x44, + 0xea, 0x8f, 0xe1, 0xb4, 0x86, 0x7d, 0xa6, 0x87, 0xc6, 0x61, 0x54, 0xc3, 0xd8, 0x04, 0xdd, 0x80, + 0x3c, 0x1b, 0x68, 0x3e, 0x09, 0x46, 0x43, 0x22, 0xb8, 0x77, 0xf6, 0xca, 0x39, 0x86, 0xc2, 0x0c, + 0x74, 0x47, 0x56, 0x12, 0x6a, 0xf2, 0x8e, 0xac, 0x24, 0x55, 0xb9, 0xfc, 0xdf, 0x04, 0xe4, 0xf9, + 0x09, 0x9f, 0xfb, 0xcf, 0xf7, 0x43, 0x50, 0x44, 0x22, 0xf1, 0x47, 0xd1, 0x99, 0x57, 0x6a, 0xfc, + 0x0c, 0xd1, 0x93, 0x35, 0xf2, 0x41, 0xb4, 0xaa, 0xf8, 0x27, 0x09, 0xa2, 0xc4, 0x43, 0x5b, 0x20, + 0xcf, 0x6f, 0x3b, 0x63, 0x8f, 0x51, 0x61, 0x80, 0x01, 0xe9, 0xef, 0x9b, 0x96, 0x5d, 0x9f, 0x3c, + 0xb0, 0x82, 0xe8, 0xc1, 0x9e, 0xc4, 0xb9, 0xa1, 0x6b, 0x62, 0x21, 0x42, 0x6f, 0x42, 0xca, 0x77, + 0x47, 0x21, 0x11, 0xd9, 0x10, 0xfb, 0xca, 0x81, 0xa9, 0x58, 0x98, 0xe3, 0x18, 0xf4, 0x03, 0x58, + 0x31, 0x7c, 0xa2, 0x87, 0x64, 0x62, 0x92, 0x95, 0x48, 0xbc, 0xcc, 0xc5, 0x91, 0xd5, 0x3b, 0xb2, + 0x22, 0xab, 0xa9, 0xf2, 0x3f, 0x25, 0xc8, 0x57, 0x3d, 0xcf, 0x3e, 0x89, 0x72, 0xe1, 0x03, 0xc8, + 0x18, 0x87, 0xba, 0x33, 0x20, 0xd1, 0x47, 0x95, 0x2b, 0x93, 0xed, 0xe2, 0xc0, 0xca, 0x0e, 0x43, + 0x45, 0x9f, 0x33, 0xc4, 0x9a, 0xe2, 0x57, 0x12, 0xa4, 0xb9, 0x06, 0x55, 0xe0, 0x05, 0x72, 0xec, + 0x11, 0x23, 0xd4, 0xce, 0x5c, 0x90, 0x3d, 0x8c, 0xf1, 0x2a, 0x57, 0xdd, 0x8d, 0x5d, 0xf3, 0x6d, + 0x48, 0x8f, 0xbc, 0x80, 0xf8, 0xa1, 0x08, 0xdc, 0x7c, 0xe7, 0x61, 0x01, 0x42, 0xaf, 0x42, 0xda, + 0x24, 0x36, 0x11, 0x6e, 0x99, 0xfa, 0xfd, 0x0b, 0x55, 0xd9, 0x62, 0x25, 0x99, 0x1e, 0xfa, 0x79, + 0xe7, 0x51, 0xf9, 0x5f, 0x09, 0x50, 0xa3, 0x5f, 0x67, 0xf0, 0xdc, 0x3a, 0x80, 0xd9, 0x5e, 0x2d, + 0x39, 0xdb, 0xab, 0xd1, 0x3e, 0x81, 0x36, 0x7f, 0x63, 0x0c, 0xcf, 0x00, 0xda, 0x10, 0x46, 0x88, + 0x37, 0x60, 0xc5, 0x21, 0xc7, 0xa1, 0xe6, 0xe9, 0x03, 0xa2, 0x85, 0xee, 0x11, 0x71, 0x04, 0xeb, + 0x15, 0xa8, 0xb8, 0xab, 0x0f, 0x48, 0x9f, 0x0a, 0xd1, 0x15, 0x00, 0x06, 0xe1, 0xaf, 0x1e, 0x4a, + 0xc9, 0x29, 0x9c, 0xa5, 0x12, 0xf6, 0xe4, 0x41, 0xb7, 0x20, 0x1f, 0x58, 0x03, 0x47, 0x0f, 0x47, + 0x3e, 0xe9, 0xf7, 0x5b, 0x82, 0xe7, 0x9f, 0xf0, 0x80, 0x56, 0x1e, 0x9e, 0x96, 0x24, 0xf6, 0x3a, + 0x3e, 0xb3, 0x70, 0xa6, 0xb3, 0x51, 0xa6, 0x3b, 0x9b, 0xf2, 0x9f, 0x13, 0xb0, 0x1a, 0xf3, 0xef, + 0x73, 0xe7, 0x85, 0x26, 0x64, 0x23, 0x8a, 0x8d, 0x88, 0xe1, 0xf5, 0x59, 0x1e, 0x1e, 0x9f, 0xa4, + 0xa2, 0x8d, 0xbf, 0x62, 0x71, 0x3b, 0x93, 0xd5, 0xf3, 0x9c, 0x2d, 0xcf, 0x71, 0x76, 0xf1, 0x13, + 0xc8, 0x8e, 0xad, 0xa0, 0xb7, 0xce, 0x30, 0xc9, 0x9c, 0x12, 0x70, 0x86, 0x46, 0xae, 0x00, 0x50, + 0x7f, 0x12, 0x93, 0xf5, 0xad, 0xfc, 0xb5, 0x9c, 0xe5, 0x92, 0x3d, 0xdf, 0x2e, 0xff, 0x4a, 0x82, + 0x14, 0x23, 0x0b, 0xf4, 0x3e, 0x64, 0x86, 0x64, 0xb8, 0x4f, 0x6b, 0x05, 0xff, 0x7d, 0x9f, 0xf7, + 0x96, 0x8f, 0xe0, 0xb4, 0x80, 0x7a, 0xbe, 0x35, 0xd4, 0xfd, 0x13, 0xfe, 0x55, 0x11, 0x47, 0x53, + 0x74, 0x0d, 0xb2, 0xd1, 0x63, 0x3e, 0xfa, 0xb0, 0x74, 0xf6, 0xad, 0x3f, 0x51, 0x8b, 0x06, 0xed, + 0x0f, 0x09, 0x48, 0x73, 0xaf, 0xa3, 0x0f, 0x00, 0xa2, 0x07, 0xfb, 0x53, 0x7f, 0x5f, 0xc8, 0x8a, + 0x15, 0x4d, 0x73, 0x42, 0x8e, 0x89, 0xa7, 0x20, 0xc7, 0x2d, 0x90, 0x49, 0x68, 0x98, 0x82, 0x48, + 0x2f, 0x4c, 0x67, 0x40, 0xa5, 0x11, 0x1a, 0x66, 0xe4, 0x56, 0x0a, 0x2c, 0x7e, 0x21, 0x81, 0x4c, + 0x85, 0xd4, 0xbf, 0x86, 0x3d, 0xa2, 0xe5, 0x33, 0x3a, 0xa5, 0x8c, 0xb3, 0x42, 0xd2, 0x34, 0xd1, + 0x4b, 0x90, 0xe5, 0x6e, 0xa2, 0xda, 0x04, 0xd3, 0x2a, 0x5c, 0xd0, 0x34, 0x51, 0x11, 0x94, 0x31, + 0xfb, 0xf1, 0x5f, 0xeb, 0x78, 0x4e, 0x17, 0xfa, 0xfa, 0x41, 0xa8, 0x85, 0xc4, 0xe7, 0xaf, 0x78, + 0x19, 0x2b, 0x54, 0xd0, 0x27, 0xfe, 0x30, 0xfa, 0xcc, 0x41, 0xff, 0x5e, 0xfb, 0x2e, 0x01, 0x69, + 0x9e, 0xd1, 0x28, 0x0d, 0x89, 0xce, 0x47, 0xea, 0x12, 0xba, 0x00, 0xab, 0x77, 0x3a, 0x7b, 0xb8, + 0x5d, 0x6d, 0x69, 0xed, 0x4e, 0x5f, 0xdb, 0xed, 0xec, 0xb5, 0xeb, 0xaa, 0x84, 0xae, 0xc0, 0xe5, + 0x76, 0x47, 0x8b, 0x34, 0x5d, 0xdc, 0xbc, 0x5b, 0xc5, 0xf7, 0xb5, 0x1a, 0xee, 0x7c, 0xd4, 0xc0, + 0x6a, 0x02, 0xad, 0x43, 0x91, 0xa2, 0x17, 0xe8, 0x93, 0xe8, 0x22, 0xa0, 0xb8, 0x5e, 0xc8, 0x53, + 0x68, 0x03, 0x5e, 0x6e, 0xb6, 0x7b, 0x7b, 0xbb, 0xbb, 0xcd, 0x9d, 0x66, 0xa3, 0x3d, 0x0d, 0xe8, + 0xa9, 0x32, 0x7a, 0x19, 0xd6, 0x3a, 0xbb, 0xbb, 0xbd, 0x46, 0x9f, 0x1d, 0xe7, 0x7e, 0xa3, 0xaf, + 0x55, 0x3f, 0xae, 0x36, 0x5b, 0xd5, 0x5a, 0xab, 0xa1, 0xa6, 0xd1, 0x0a, 0xe4, 0xee, 0xe1, 0x4e, + 0xfb, 0x96, 0x86, 0x3b, 0x7b, 0xfd, 0x86, 0x9a, 0xa1, 0xc7, 0xef, 0xe2, 0x4e, 0xb7, 0xd3, 0xab, + 0xb6, 0xb4, 0xbb, 0xcd, 0xde, 0xdd, 0x6a, 0x7f, 0xe7, 0xb6, 0xaa, 0xa0, 0x97, 0xe0, 0x52, 0xa3, + 0xbf, 0x53, 0xd7, 0xfa, 0xb8, 0xda, 0xee, 0x55, 0x77, 0xfa, 0xcd, 0x4e, 0x5b, 0xdb, 0xad, 0x36, + 0x5b, 0x8d, 0xba, 0x9a, 0xa5, 0x46, 0xa8, 0xed, 0x6a, 0xab, 0xd5, 0xb9, 0xd7, 0xa8, 0xab, 0x80, + 0x2e, 0xc1, 0x0b, 0xdc, 0x6a, 0xb5, 0xdb, 0x6d, 0xb4, 0xeb, 0x1a, 0x3f, 0x80, 0x9a, 0xa3, 0x87, + 0x69, 0xb6, 0xeb, 0x8d, 0x4f, 0xb4, 0xdb, 0xd5, 0x9e, 0x76, 0x0b, 0x37, 0xaa, 0xfd, 0x06, 0x8e, + 0xb4, 0x79, 0xba, 0x37, 0x6e, 0xdc, 0x6a, 0xf6, 0xa8, 0x70, 0xbc, 0x77, 0xe1, 0x9a, 0x03, 0xea, + 0xf4, 0xf7, 0x0f, 0x94, 0x83, 0x4c, 0xb3, 0xfd, 0x71, 0xb5, 0xd5, 0xac, 0xab, 0x4b, 0x48, 0x01, + 0xb9, 0xdd, 0x69, 0x37, 0x54, 0x89, 0x8e, 0x6e, 0x7d, 0xda, 0xec, 0xaa, 0x09, 0x54, 0x80, 0xec, + 0xa7, 0xbd, 0x7e, 0xb5, 0x5d, 0xaf, 0xe2, 0xba, 0x9a, 0x44, 0x00, 0xe9, 0x5e, 0xbb, 0xda, 0xed, + 0xde, 0x57, 0x65, 0xea, 0x6b, 0x0a, 0xa2, 0xfb, 0xb6, 0x3a, 0xd5, 0xba, 0x56, 0x6f, 0xec, 0x74, + 0xee, 0x76, 0x71, 0xa3, 0xd7, 0x6b, 0x76, 0xda, 0x6a, 0x6a, 0xfb, 0xcb, 0xe4, 0xa4, 0x73, 0x78, + 0x0f, 0x64, 0xda, 0x6d, 0xa0, 0x0b, 0xd3, 0xdd, 0x07, 0xab, 0x24, 0xc5, 0x8b, 0xf3, 0x9b, 0x92, + 0xeb, 0x12, 0x7a, 0x1f, 0x52, 0xac, 0xc6, 0xa1, 0x8b, 0xf3, 0x2b, 0x75, 0xf1, 0xd2, 0x8c, 0x5c, + 0x90, 0xe7, 0x7b, 0x20, 0xd3, 0x17, 0x7d, 0x7c, 0xcb, 0xd8, 0x47, 0x91, 0xf8, 0x96, 0xf1, 0x87, + 0xff, 0x75, 0x09, 0x7d, 0x00, 0x69, 0xfe, 0xbc, 0x42, 0x67, 0x6d, 0x4f, 0xde, 0xbe, 0xc5, 0xb5, + 0x59, 0x05, 0x5f, 0xbe, 0x29, 0xa1, 0xdb, 0x90, 0x1d, 0xb7, 0xd2, 0xa8, 0x18, 0xdf, 0xe5, 0xec, + 0xeb, 0xa4, 0xf8, 0xd2, 0x5c, 0x5d, 0x64, 0xe7, 0x3a, 0xb5, 0x54, 0xa0, 0xde, 0x18, 0xb3, 0x71, + 0xdc, 0xda, 0x74, 0x31, 0x8e, 0x5b, 0x9b, 0xa1, 0xef, 0x5a, 0xe3, 0xe1, 0x7f, 0xd6, 0x97, 0x1e, + 0x7e, 0xb7, 0x2e, 0x7d, 0xfb, 0xdd, 0xba, 0xf4, 0xeb, 0x47, 0xeb, 0x4b, 0xdf, 0x3c, 0x5a, 0x97, + 0xfe, 0xfa, 0x68, 0x5d, 0xfa, 0xf6, 0xd1, 0xfa, 0xd2, 0xdf, 0x1f, 0xad, 0x2f, 0x7d, 0xfa, 0xea, + 0xc0, 0xad, 0x0c, 0xf4, 0xcf, 0x49, 0x18, 0x92, 0x8a, 0x49, 0x1e, 0x6c, 0x19, 0xae, 0x4f, 0xb6, + 0xa6, 0xfe, 0xab, 0xb6, 0x9f, 0x66, 0xa3, 0x1b, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x7b, 0xe9, + 0xe8, 0x1d, 0x6f, 0x1b, 0x00, 0x00, } func (this *Label) Equal(that interface{}) bool { @@ -2300,7 +2307,7 @@ const _ = grpc.SupportPackageIsVersion4 // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type JournalClient interface { // List Journals, their JournalSpecs and current Routes. - List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (*ListResponse, error) + List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (Journal_ListClient, error) // Apply changes to the collection of Journals managed by the brokers. Apply(ctx context.Context, in *ApplyRequest, opts ...grpc.CallOption) (*ApplyResponse, error) // Read from a specific Journal. @@ -2323,13 +2330,36 @@ func NewJournalClient(cc *grpc.ClientConn) JournalClient { return &journalClient{cc} } -func (c *journalClient) List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (*ListResponse, error) { - out := new(ListResponse) - err := c.cc.Invoke(ctx, "/protocol.Journal/List", in, out, opts...) +func (c *journalClient) List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (Journal_ListClient, error) { + stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[0], "/protocol.Journal/List", opts...) if err != nil { return nil, err } - return out, nil + x := &journalListClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Journal_ListClient interface { + Recv() (*ListResponse, error) + grpc.ClientStream +} + +type journalListClient struct { + grpc.ClientStream +} + +func (x *journalListClient) Recv() (*ListResponse, error) { + m := new(ListResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil } func (c *journalClient) Apply(ctx context.Context, in *ApplyRequest, opts ...grpc.CallOption) (*ApplyResponse, error) { @@ -2342,7 +2372,7 @@ func (c *journalClient) Apply(ctx context.Context, in *ApplyRequest, opts ...grp } func (c *journalClient) Read(ctx context.Context, in *ReadRequest, opts ...grpc.CallOption) (Journal_ReadClient, error) { - stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[0], "/protocol.Journal/Read", opts...) + stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[1], "/protocol.Journal/Read", opts...) if err != nil { return nil, err } @@ -2374,7 +2404,7 @@ func (x *journalReadClient) Recv() (*ReadResponse, error) { } func (c *journalClient) Append(ctx context.Context, opts ...grpc.CallOption) (Journal_AppendClient, error) { - stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[1], "/protocol.Journal/Append", opts...) + stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[2], "/protocol.Journal/Append", opts...) if err != nil { return nil, err } @@ -2408,7 +2438,7 @@ func (x *journalAppendClient) CloseAndRecv() (*AppendResponse, error) { } func (c *journalClient) Replicate(ctx context.Context, opts ...grpc.CallOption) (Journal_ReplicateClient, error) { - stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[2], "/protocol.Journal/Replicate", opts...) + stream, err := c.cc.NewStream(ctx, &_Journal_serviceDesc.Streams[3], "/protocol.Journal/Replicate", opts...) if err != nil { return nil, err } @@ -2450,7 +2480,7 @@ func (c *journalClient) ListFragments(ctx context.Context, in *FragmentsRequest, // JournalServer is the server API for Journal service. type JournalServer interface { // List Journals, their JournalSpecs and current Routes. - List(context.Context, *ListRequest) (*ListResponse, error) + List(*ListRequest, Journal_ListServer) error // Apply changes to the collection of Journals managed by the brokers. Apply(context.Context, *ApplyRequest) (*ApplyResponse, error) // Read from a specific Journal. @@ -2469,8 +2499,8 @@ type JournalServer interface { type UnimplementedJournalServer struct { } -func (*UnimplementedJournalServer) List(ctx context.Context, req *ListRequest) (*ListResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method List not implemented") +func (*UnimplementedJournalServer) List(req *ListRequest, srv Journal_ListServer) error { + return status.Errorf(codes.Unimplemented, "method List not implemented") } func (*UnimplementedJournalServer) Apply(ctx context.Context, req *ApplyRequest) (*ApplyResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Apply not implemented") @@ -2492,22 +2522,25 @@ func RegisterJournalServer(s *grpc.Server, srv JournalServer) { s.RegisterService(&_Journal_serviceDesc, srv) } -func _Journal_List_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ListRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(JournalServer).List(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/protocol.Journal/List", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(JournalServer).List(ctx, req.(*ListRequest)) +func _Journal_List_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(ListRequest) + if err := stream.RecvMsg(m); err != nil { + return err } - return interceptor(ctx, in, info, handler) + return srv.(JournalServer).List(m, &journalListServer{stream}) +} + +type Journal_ListServer interface { + Send(*ListResponse) error + grpc.ServerStream +} + +type journalListServer struct { + grpc.ServerStream +} + +func (x *journalListServer) Send(m *ListResponse) error { + return x.ServerStream.SendMsg(m) } func _Journal_Apply_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { @@ -2623,10 +2656,6 @@ var _Journal_serviceDesc = grpc.ServiceDesc{ ServiceName: "protocol.Journal", HandlerType: (*JournalServer)(nil), Methods: []grpc.MethodDesc{ - { - MethodName: "List", - Handler: _Journal_List_Handler, - }, { MethodName: "Apply", Handler: _Journal_Apply_Handler, @@ -2637,6 +2666,11 @@ var _Journal_serviceDesc = grpc.ServiceDesc{ }, }, Streams: []grpc.StreamDesc{ + { + StreamName: "List", + Handler: _Journal_List_Handler, + ServerStreams: true, + }, { StreamName: "Read", Handler: _Journal_Read_Handler, @@ -3659,6 +3693,28 @@ func (m *ListRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.WatchResume != nil { + { + size, err := m.WatchResume.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + if m.Watch { + i-- + if m.Watch { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } { size, err := m.Selector.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -3930,12 +3986,12 @@ func (m *FragmentsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x40 } if m.SignatureTTL != nil { - n33, err33 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.SignatureTTL, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.SignatureTTL):]) - if err33 != nil { - return 0, err33 + n34, err34 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.SignatureTTL, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.SignatureTTL):]) + if err34 != nil { + return 0, err34 } - i -= n33 - i = encodeVarintProtocol(dAtA, i, uint64(n33)) + i -= n34 + i = encodeVarintProtocol(dAtA, i, uint64(n34)) i-- dAtA[i] = 0x3a } @@ -4643,6 +4699,13 @@ func (m *ListRequest) ProtoSize() (n int) { _ = l l = m.Selector.ProtoSize() n += 1 + l + sovProtocol(uint64(l)) + if m.Watch { + n += 2 + } + if m.WatchResume != nil { + l = m.WatchResume.ProtoSize() + n += 1 + l + sovProtocol(uint64(l)) + } return n } @@ -7823,6 +7886,62 @@ func (m *ListRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Watch", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Watch = bool(v != 0) + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WatchResume", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.WatchResume == nil { + m.WatchResume = &Header{} + } + if err := m.WatchResume.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProtocol(dAtA[iNdEx:]) diff --git a/broker/protocol/protocol.proto b/broker/protocol/protocol.proto index d60c2cb1..582b4327 100644 --- a/broker/protocol/protocol.proto +++ b/broker/protocol/protocol.proto @@ -578,9 +578,14 @@ message ListRequest { // * prefix=examples/ will match any JournalSpec having prefix "examples/". // The prefix Label value must end in '/'. LabelSelector selector = 1 [ (gogoproto.nullable) = false ]; + // Watch the requested selector and send an updated snapshot upon every + // change to matched journal specifications. + bool watch = 4; + // Header of the last ListResponse processed by a resumed watch of a listing. + Header watch_resume = 5; } -// ListResponse is the unary response message of the broker List RPC. +// ListResponse is the streamed response message of the broker List RPC. message ListResponse { // Tag used for unimplemented, removed next_page_token. reserved 4; @@ -731,7 +736,7 @@ message Header { // Journal is the Gazette broker service API for interacting with Journals. service Journal { // List Journals, their JournalSpecs and current Routes. - rpc List(ListRequest) returns (ListResponse); + rpc List(ListRequest) returns (stream ListResponse); // Apply changes to the collection of Journals managed by the brokers. rpc Apply(ApplyRequest) returns (ApplyResponse); // Read from a specific Journal. diff --git a/broker/protocol/rpc_extensions.go b/broker/protocol/rpc_extensions.go index 78c9f36e..dca30eba 100644 --- a/broker/protocol/rpc_extensions.go +++ b/broker/protocol/rpc_extensions.go @@ -1,8 +1,13 @@ package protocol import ( + "context" "net/url" "strings" + + "github.com/pkg/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) // RoutedJournalClient composes a JournalClient and DispatchRouter. @@ -22,6 +27,21 @@ func NewRoutedJournalClient(jc JournalClient, dr DispatchRouter) RoutedJournalCl } } +// SuppressCancellationErrors filters errors which are common to long-lived +// blocking RPCs which are eventually cancelled by the caller or the server. +// It interprets local or remote cancellation as a graceful closure +// of the RPC and not an error. +func SuppressCancellationError(err error) error { + if ec, sc := errors.Cause(err), status.Code(err); ec == context.Canceled || + ec == context.DeadlineExceeded || + sc == codes.Canceled || + sc == codes.DeadlineExceeded { + + return nil + } + return err +} + // Validate returns an error if the ReadRequest is not well-formed. func (m *ReadRequest) Validate() error { if m.Header != nil { diff --git a/broker/read_api.go b/broker/read_api.go index cddd1b1b..e4edfad4 100644 --- a/broker/read_api.go +++ b/broker/read_api.go @@ -1,20 +1,15 @@ package broker import ( - "context" "io" - "io/ioutil" "net" - "github.com/pkg/errors" log "github.com/sirupsen/logrus" "go.gazette.dev/core/broker/client" "go.gazette.dev/core/broker/fragment" pb "go.gazette.dev/core/broker/protocol" "google.golang.org/grpc" - "google.golang.org/grpc/codes" "google.golang.org/grpc/peer" - "google.golang.org/grpc/status" ) // Read dispatches the JournalServer.Read API. @@ -56,19 +51,7 @@ func (svc *Service) Read(req *pb.ReadRequest, stream pb.Journal_ReadServer) (err return proxyRead(stream, req, svc.jc, svc.stopProxyReadsCh) } - err = serveRead(stream, req, &resolved.Header, resolved.replica.index) - - // Blocking Read RPCs live indefinitely, until cancelled by the caller or - // due to journal reassignment. Interpret local or remote cancellation as - // a graceful closure of the RPC and not an error. - if ec, sc := errors.Cause(err), status.Code(err); ec == context.Canceled || - ec == context.DeadlineExceeded || - sc == codes.Canceled || - sc == codes.DeadlineExceeded { - - err = nil - } - return err + return pb.SuppressCancellationError(serveRead(stream, req, &resolved.Header, resolved.replica.index)) } // proxyRead forwards a ReadRequest to a resolved peer broker. @@ -158,7 +141,7 @@ func serveRead(stream grpc.ServerStream, req *pb.ReadRequest, hdr *pb.Header, in req.Offset = resp.Offset if file != nil { - reader = ioutil.NopCloser(io.NewSectionReader( + reader = io.NopCloser(io.NewSectionReader( file, req.Offset-resp.Fragment.Begin, resp.Fragment.End-req.Offset)) } else { if reader, err = fragment.Open(stream.Context(), *resp.Fragment); err != nil { diff --git a/broker/teststub/broker.go b/broker/teststub/broker.go index 66a0529e..8174e04d 100644 --- a/broker/teststub/broker.go +++ b/broker/teststub/broker.go @@ -179,9 +179,35 @@ func (b *Broker) Append(srv pb.Journal_AppendServer) error { } } -// List implements the JournalServer interface by proxying through ListFunc. -func (b *Broker) List(ctx context.Context, req *pb.ListRequest) (*pb.ListResponse, error) { - return b.ListFunc(ctx, req) +// List implements the JournalServer interface by proxy-ing through ListFunc. +func (b *Broker) List(req *pb.ListRequest, stream pb.Journal_ListServer) error { + var resp, err = b.ListFunc(stream.Context(), req) + if err != nil { + return err + } + // Don't directly mutate ListFunc's returned `resp`. + resp = &pb.ListResponse{ + Status: resp.Status, + Header: resp.Header, + Journals: resp.Journals, + } + + for len(resp.Journals) > 2 { + var tail = resp.Journals[2:] + resp.Journals = resp.Journals[:2] + + _ = stream.Send(resp) + *resp = pb.ListResponse{Journals: tail} + } + _ = stream.Send(resp) + + if !req.Watch { + return nil + } + _ = stream.Send(&pb.ListResponse{}) + + // Simulate server cancellation of watch after single response snapshot. + return context.Canceled } // Apply implements the JournalServer interface by proxying through ApplyFunc. diff --git a/brokertest/broker_test.go b/brokertest/broker_test.go index b12dacb3..82db9404 100644 --- a/brokertest/broker_test.go +++ b/brokertest/broker_test.go @@ -215,8 +215,35 @@ func TestGracefulStopTimeout(t *testing.T) { require.NoError(t, bkA.Tasks.Wait()) } +func TestWatchedListing(t *testing.T) { + var etcd = etcdtest.TestClient() + defer etcdtest.Cleanup() + + var bk = NewBroker(t, etcd, "local", "broker") + + var conn, rjc = newDialedClient(t, bk) + defer conn.Close() + + var ctx, cancel = context.WithCancel(pb.WithDispatchDefault(context.Background())) + defer cancel() + + // Initial listing is empty. + var list = client.NewWatchedList(ctx, rjc, pb.ListRequest{Watch: true}, nil) + require.NoError(t, <-list.UpdateCh()) + require.Empty(t, list.List().Journals) + + // Listing reacts to a created journal. + CreateJournals(t, bk, Journal(pb.JournalSpec{Name: "foo/bar"})) + require.NoError(t, <-list.UpdateCh()) + require.Len(t, list.List().Journals, 1) + + cancel() + bk.Tasks.Cancel() + require.NoError(t, bk.Tasks.Wait()) +} + func updateReplication(t require.TestingT, ctx context.Context, bk pb.JournalClient, journal pb.Journal, r int32) { - var lResp, err = bk.List(ctx, &pb.ListRequest{ + var lResp, err = client.ListAllJournals(ctx, bk, pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("name", journal.String())}, }) require.NoError(t, err) diff --git a/cmd/gazctl/gazctlcmd/journals_append.go b/cmd/gazctl/gazctlcmd/journals_append.go index 05a6cc2c..3b0e881c 100644 --- a/cmd/gazctl/gazctlcmd/journals_append.go +++ b/cmd/gazctl/gazctlcmd/journals_append.go @@ -8,7 +8,6 @@ import ( "io" "os" "sort" - "time" "github.com/pkg/errors" log "github.com/sirupsen/logrus" @@ -103,11 +102,10 @@ func (cmd *cmdJournalAppend) Execute([]string) error { listRequest.Selector, err = pb.ParseLabelSelector(cmd.Selector) mbp.Must(err, "failed to parse label selector", "selector", cmd.Selector) - // Perform an initial load and thereafter periodically poll for journals - // matching the --selector. + // List and watch journals matching the --selector. var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) - list, err := client.NewPolledList(ctx, rjc, time.Minute, listRequest) - mbp.Must(err, "failed to resolve label selector to journals") + var list = client.NewWatchedList(ctx, rjc, listRequest, nil) + mbp.Must(<-list.UpdateCh(), "failed to resolve label selector to journals") var fin = os.Stdin if cmd.Input != "-" { diff --git a/cmd/gazctl/gazctlcmd/journals_list.go b/cmd/gazctl/gazctlcmd/journals_list.go index de252e65..dc4adbab 100644 --- a/cmd/gazctl/gazctlcmd/journals_list.go +++ b/cmd/gazctl/gazctlcmd/journals_list.go @@ -20,6 +20,7 @@ import ( type cmdJournalsList struct { ListConfig Stores bool `long:"stores" description:"Show fragment store column"` + Watch bool `long:"watch" description:"Use a long-lived watch"` } func init() { @@ -53,8 +54,28 @@ func (cmd *cmdJournalsList) Execute([]string) error { var ctx = context.Background() var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) - var resp = listJournals(rjc, cmd.Selector) + if !cmd.Watch { + var resp = listJournals(rjc, cmd.Selector) + cmd.output(resp) + return nil + } + + var req = pb.ListRequest{Watch: true} + var err error + + req.Selector, err = pb.ParseLabelSelector(cmd.Selector) + mbp.Must(err, "failed to parse label selector", "selector", cmd.Selector) + var list = client.NewWatchedList(ctx, rjc, req, nil) + mbp.Must(<-list.UpdateCh(), "failed to list journals") + + for { + <-list.UpdateCh() + cmd.output(list.List()) + } +} + +func (cmd *cmdJournalsList) output(resp *pb.ListResponse) { switch cmd.Format { case "table": cmd.outputTable(resp) @@ -66,7 +87,6 @@ func (cmd *cmdJournalsList) Execute([]string) error { case "proto": mbp.Must(proto.MarshalText(os.Stdout, resp), "failed to write output") } - return nil } func (cmd *cmdJournalsList) outputTable(resp *pb.ListResponse) { diff --git a/cmd/gazctl/gazctlcmd/journals_read.go b/cmd/gazctl/gazctlcmd/journals_read.go index 8a21d659..910dbe55 100644 --- a/cmd/gazctl/gazctlcmd/journals_read.go +++ b/cmd/gazctl/gazctlcmd/journals_read.go @@ -7,7 +7,6 @@ import ( "os" "os/signal" "syscall" - "time" log "github.com/sirupsen/logrus" "go.gazette.dev/core/broker/client" @@ -149,11 +148,10 @@ func (cmd *cmdJournalRead) Execute([]string) error { cancel() }() - // Perform an initial load and thereafter periodically poll for journals - // matching the --selector. + // List and watch journals matching the --selector. var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) - list, err := client.NewPolledList(ctx, rjc, time.Minute, listRequest) - mbp.Must(err, "failed to resolve label selector to journals") + var list = client.NewWatchedList(ctx, rjc, listRequest, nil) + mbp.Must(<-list.UpdateCh(), "failed to resolve label selector to journals") if len(list.List().Journals) == 0 { log.Warn("no journals were matched by the selector") @@ -189,7 +187,7 @@ func (cmd *cmdJournalRead) Execute([]string) error { return nil } -func (cmd *cmdJournalRead) listRefreshed(ctx context.Context, rjc pb.RoutedJournalClient, list *client.PolledList) { +func (cmd *cmdJournalRead) listRefreshed(ctx context.Context, rjc pb.RoutedJournalClient, list *client.WatchedList) { var ( // Construct a new map of CancelFunc, to enable detection of // journals which were but are are no longer in |list|. diff --git a/consumer/protocol/shard_spec_extensions.go b/consumer/protocol/shard_spec_extensions.go index ec614164..691ad474 100644 --- a/consumer/protocol/shard_spec_extensions.go +++ b/consumer/protocol/shard_spec_extensions.go @@ -253,10 +253,15 @@ func SubtractShardSpecs(a, b ShardSpec) ShardSpec { return a } -// ExtractShardSpecMetaLabels returns meta-labels of the ShardSpec, using |out| as a buffer. -func ExtractShardSpecMetaLabels(spec *ShardSpec, out pb.LabelSet) pb.LabelSet { - out.Labels = append(out.Labels[:0], pb.Label{Name: "id", Value: spec.Id.String()}) - return out +// LabelSetExt adds additional metadata labels to the LabelSet of the ShardSpec, +// returning the result. The result is built by truncating `buf` and then appending +// the merged LabelSet. +func (m *ShardSpec) LabelSetExt(buf pb.LabelSet) pb.LabelSet { + return pb.UnionLabelSets(pb.LabelSet{ + Labels: []pb.Label{ + {Name: "id", Value: m.Id.String()}, + }, + }, m.LabelSet, buf) } // Validate returns an error if the ConsumerSpec is not well-formed. diff --git a/consumer/protocol/shard_spec_extensions_test.go b/consumer/protocol/shard_spec_extensions_test.go index 2b6923dc..77a9fd2d 100644 --- a/consumer/protocol/shard_spec_extensions_test.go +++ b/consumer/protocol/shard_spec_extensions_test.go @@ -88,6 +88,7 @@ func (s *SpecSuite) TestShardSpecRoutines(c *gc.C) { var spec = ShardSpec{ Id: "shard-id", HotStandbys: 4, + LabelSet: pb.MustLabelSet("hello", "world"), Disable: false, HintPrefix: "/a/path", HintBackups: 2, @@ -104,8 +105,8 @@ func (s *SpecSuite) TestShardSpecRoutines(c *gc.C) { spec.Disable, spec.HotStandbys = false, 0 c.Check(spec.DesiredReplication(), gc.Equals, 1) - c.Check(ExtractShardSpecMetaLabels(&spec, pb.MustLabelSet("label", "buffer")), - gc.DeepEquals, pb.MustLabelSet("id", "shard-id")) + c.Check(spec.LabelSetExt(pb.MustLabelSet("label", "buffer")), + gc.DeepEquals, pb.MustLabelSet("id", "shard-id", "hello", "world")) c.Check(spec.HintPrimaryKey(), gc.Equals, "/a/path/shard-id.primary") c.Check(spec.HintBackupKeys(), gc.DeepEquals, []string{ diff --git a/consumer/recovery_test.go b/consumer/recovery_test.go index fb18103e..166c20f4 100644 --- a/consumer/recovery_test.go +++ b/consumer/recovery_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/stretchr/testify/require" + "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" pc "go.gazette.dev/core/consumer/protocol" "go.gazette.dev/core/consumer/recoverylog" @@ -126,7 +127,7 @@ func TestRecoveryFailsFromWrongContentType(t *testing.T) { var ctx = pb.WithDispatchDefault(context.Background()) // Fetch current log spec, set an incorrect ContentType, and re-apply. - var lr, err = shard.ajc.List(ctx, &pb.ListRequest{ + lr, err := client.ListAllJournals(ctx, shard.ajc, pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("name", shard.Spec().RecoveryLog().String())}, }) require.NoError(t, err) diff --git a/consumer/shard_api.go b/consumer/shard_api.go index c969bc0e..9f2834ca 100644 --- a/consumer/shard_api.go +++ b/consumer/shard_api.go @@ -49,18 +49,18 @@ func ShardStat(ctx context.Context, srv *Service, req *pc.StatRequest) (*pc.Stat func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.ListResponse, error) { var s = srv.Resolver.state - var resp = &pc.ListResponse{ - Status: pc.Status_OK, - Header: pbx.NewUnroutedHeader(s), - } if err := req.Validate(); err != nil { - return resp, err + return nil, err } defer s.KS.Mu.RUnlock() s.KS.Mu.RLock() - var metaLabels, allLabels pb.LabelSet + var resp = &pc.ListResponse{ + Status: pc.Status_OK, + Header: pbx.NewUnroutedHeader(s), + } + var scratch pb.LabelSet var it = allocator.LeftJoin{ LenL: len(s.Items), @@ -75,10 +75,10 @@ func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.List var shard = pc.ListResponse_Shard{ Spec: *s.Items[cur.Left].Decoded.(allocator.Item).ItemValue.(*pc.ShardSpec)} - metaLabels = pc.ExtractShardSpecMetaLabels(&shard.Spec, metaLabels) - allLabels = pb.UnionLabelSets(metaLabels, shard.Spec.LabelSet, allLabels) + // LabelSetExt() truncates `scratch` while re-using its storage. + scratch = shard.Spec.LabelSetExt(scratch) - if !req.Selector.Matches(allLabels) { + if !req.Selector.Matches(scratch) { continue } shard.ModRevision = s.Items[cur.Left].Raw.ModRevision @@ -100,12 +100,8 @@ func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.List func ShardApply(ctx context.Context, srv *Service, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { var s = srv.Resolver.state - var resp = &pc.ApplyResponse{ - Status: pc.Status_OK, - Header: pbx.NewUnroutedHeader(s), - } if err := req.Validate(); err != nil { - return resp, err + return nil, err } var cmp []clientv3.Cmp @@ -128,6 +124,13 @@ func ShardApply(ctx context.Context, srv *Service, req *pc.ApplyRequest) (*pc.Ap } } + s.KS.Mu.RLock() + var resp = &pc.ApplyResponse{ + Status: pc.Status_OK, + Header: pbx.NewUnroutedHeader(s), + } + s.KS.Mu.RUnlock() + var txnResp, err = srv.Etcd.Do(ctx, clientv3.OpTxn(cmp, ops, nil)) if err != nil { // Pass. @@ -145,23 +148,20 @@ func ShardApply(ctx context.Context, srv *Service, req *pc.ApplyRequest) (*pc.Ap // ShardGetHints is the default implementation of the ShardServer.Hints API. func ShardGetHints(ctx context.Context, srv *Service, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { - var ( - resp = &pc.GetHintsResponse{ - Status: pc.Status_OK, - Header: pbx.NewUnroutedHeader(srv.State), - } - ks = srv.State.KS - spec *pc.ShardSpec - ) - ks.Mu.RLock() - var item, ok = allocator.LookupItem(ks, req.Shard.String()) - ks.Mu.RUnlock() + srv.State.KS.Mu.RLock() + var resp = &pc.GetHintsResponse{ + Status: pc.Status_OK, + Header: pbx.NewUnroutedHeader(srv.State), + } + var item, ok = allocator.LookupItem(srv.State.KS, req.Shard.String()) + srv.State.KS.Mu.RUnlock() + if !ok { resp.Status = pc.Status_SHARD_NOT_FOUND return resp, nil } - spec = item.ItemValue.(*pc.ShardSpec) + var spec = item.ItemValue.(*pc.ShardSpec) var h, err = fetchHints(ctx, spec, srv.Etcd) if err != nil { diff --git a/examples/bike-share/application.go b/examples/bike-share/application.go index 9c5dac20..f1b3d4ce 100644 --- a/examples/bike-share/application.go +++ b/examples/bike-share/application.go @@ -203,16 +203,19 @@ func (app *Application) ConsumeMessage(shard consumer.Shard, store consumer.Stor // them out. func (Application) FinalizeTxn(consumer.Shard, consumer.Store, *message.Publisher) error { return nil } -// buildMapping starts a long-lived polling watch of journals labeled with the +// buildMapping starts a long-lived watch of journals labeled with the // given message type, and returns a mapping to journals using modulo arithmetic // over the provided MappingKeyFunc. func buildMapping(msgType string, fn message.MappingKeyFunc, args runconsumer.InitArgs) message.MappingFunc { - var parts, err = client.NewPolledList(args.Context, args.Service.Journals, time.Second*30, + var parts = client.NewWatchedList(args.Context, args.Service.Journals, pb.ListRequest{ Selector: pb.LabelSelector{ Include: pb.MustLabelSet(labels.MessageType, msgType), - }}) - mbp.Must(err, "failed to fetch partitions") + }, + }, + nil, + ) + mbp.Must(<-parts.UpdateCh(), "failed to fetch partitions") for len(parts.List().Journals) == 0 { log.WithField("msg", msgType).Info("waiting for partitions to be created") diff --git a/examples/stream-sum/stream_sum.go b/examples/stream-sum/stream_sum.go index 958f5198..90dba479 100644 --- a/examples/stream-sum/stream_sum.go +++ b/examples/stream-sum/stream_sum.go @@ -509,12 +509,15 @@ func pumpSums(rr *client.RetryReader, ch chan<- Sum) { // newChunkMapping returns a MappingFunc over journals holding chunks. func newChunkMapping(ctx context.Context, jc pb.JournalClient) (message.MappingFunc, error) { - var parts, err = client.NewPolledList(ctx, jc, time.Second*30, pb.ListRequest{ - Selector: pb.LabelSelector{ - Include: pb.MustLabelSet(labels.MessageType, "stream_sum.Chunk"), - }}) - - if err != nil { + var parts = client.NewWatchedList(ctx, jc, + pb.ListRequest{ + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet(labels.MessageType, "stream_sum.Chunk"), + }, + }, + nil, + ) + if err := <-parts.UpdateCh(); err != nil { return nil, err } diff --git a/examples/word-count/word_count.go b/examples/word-count/word_count.go index 55f1dbcd..d9823d09 100644 --- a/examples/word-count/word_count.go +++ b/examples/word-count/word_count.go @@ -10,7 +10,6 @@ import ( "fmt" "io" "strings" - "time" "unicode" "github.com/pkg/errors" @@ -20,7 +19,7 @@ import ( "go.gazette.dev/core/consumer" pc "go.gazette.dev/core/consumer/protocol" "go.gazette.dev/core/consumer/recoverylog" - "go.gazette.dev/core/consumer/store-rocksdb" + store_rocksdb "go.gazette.dev/core/consumer/store-rocksdb" "go.gazette.dev/core/labels" "go.gazette.dev/core/mainboilerplate/runconsumer" "go.gazette.dev/core/message" @@ -65,11 +64,13 @@ func (counter *Counter) InitApplication(args runconsumer.InitArgs) error { } // Build a "deltas" MappingFunc over "app.gazette.dev/message-type=NGramCount" partitions. - var parts, err = client.NewPolledList(args.Context, args.Service.Journals, time.Second*30, + var parts = client.NewWatchedList(args.Context, args.Service.Journals, pb.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet(labels.MessageType, "word_count.NGramCount")}, - }) - if err != nil { + }, + nil, + ) + if err := <-parts.UpdateCh(); err != nil { return errors.Wrap(err, "building NGramDeltaMapping") } From 98859d9ada5233de244223fd0dcdc718606d46a2 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Mon, 24 Jun 2024 22:57:47 -0500 Subject: [PATCH 4/8] broker: add support for fine-grain Authorizations Authorization is implemented in terms of LabelSelectors and Capabilities. In order for a client to call an API, the client must present an Authorization with a suitable Claim having: - The required Capability, such as READ, LIST, or APPEND. - A LabelSelector which scopes the resources visible to the client. Journals which do not match the authorized Selector are not visible to the client and are indistinguishable from not existing at all. Authorizer and Verifier interfaces are introduced which can implement varying strategies for obtaining and checking Authorization tokens. AuthJournalClient drives an Authorizer to obtain and attach a suitable credential prior to starting an RPC. AuthJournalServer uses a Verifier to verify a caller's claims prior to dispatching a service handler. A new `auth` module introduces NewKeyedAuth() which returns an Authorizer and Verifier implemented in terms of pre-shared symmetric keys. Symmetric keys are a good fit for Gazette given its distributed nature and the requirement that brokers be able to self-sign peer-to-peer requests, such as Replicate and proxied Read or Append RPCs. Client applications may want to use alternative Authorizer implementations, such as an authorization server, but that's out of scope for Gazette proper. --- auth/auth.go | 166 +++++++++++++++ auth/auth_test.go | 54 +++++ broker/append_api.go | 19 +- broker/append_api_test.go | 25 ++- broker/append_fsm.go | 27 ++- broker/append_fsm_test.go | 118 +++++------ broker/client/appender.go | 8 +- broker/client/retry_reader.go | 19 +- broker/e2e_test.go | 17 +- broker/fragments_api.go | 6 +- broker/fragments_api_test.go | 18 ++ broker/http_gateway/http_gateway.go | 9 + broker/list_apply_api.go | 34 +++- broker/list_apply_api_test.go | 56 ++++- broker/protocol/auth.go | 270 +++++++++++++++++++++++++ broker/protocol/label_extensions.go | 10 + broker/protocol/rpc_extensions.go | 14 +- broker/read_api.go | 19 +- broker/read_api_test.go | 27 ++- broker/replica.go | 12 +- broker/replica_test.go | 14 +- broker/replicate_api.go | 17 +- broker/replicate_api_test.go | 36 ++++ broker/resolver.go | 67 +++--- broker/resolver_test.go | 48 ++--- broker/service.go | 29 +-- broker/test_support_test.go | 26 ++- brokertest/broker.go | 47 +++-- brokertest/broker_test.go | 22 +- cmd/gazctl/gazctlcmd/gazctl.go | 6 +- cmd/gazette/main.go | 21 +- examples/stream-sum/stream_sum_test.go | 3 +- mainboilerplate/client.go | 15 +- 33 files changed, 1014 insertions(+), 265 deletions(-) create mode 100644 auth/auth.go create mode 100644 auth/auth_test.go create mode 100644 broker/protocol/auth.go diff --git a/auth/auth.go b/auth/auth.go new file mode 100644 index 00000000..5b4bff93 --- /dev/null +++ b/auth/auth.go @@ -0,0 +1,166 @@ +package auth + +import ( + "context" + "encoding/base64" + "errors" + "fmt" + "strconv" + "strings" + "time" + + "github.com/golang-jwt/jwt/v5" + pb "go.gazette.dev/core/broker/protocol" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" +) + +// NewKeyedAuth returns a KeyedAuth that implements Authorizer and Verifier using +// the given pre-shared secret keys, which are base64 encoded and separate by +// whitespace and/or commas. +// +// The first key is used for signing Authorizations, and any key may verify +// a presented Authorization. +// +// The special value `AA==` (the base64 encoding of a single zero byte) +// will allow requests missing an authorization header to proceed, and should +// only be used temporarily for rollout of authorization in an existing cluster. +func NewKeyedAuth(base64Keys string) (*KeyedAuth, error) { + var keys jwt.VerificationKeySet + var allowMissing bool + + for i, key := range strings.Fields(strings.ReplaceAll(base64Keys, ",", " ")) { + if key == "AA==" { + allowMissing = true + } else if b, err := base64.StdEncoding.DecodeString(key); err != nil { + return nil, fmt.Errorf("failed to decode key at index %d: %w", i, err) + } else { + keys.Keys = append(keys.Keys, b) + } + } + if len(keys.Keys) == 0 { + return nil, fmt.Errorf("at least one key must be provided") + } + return &KeyedAuth{keys, allowMissing}, nil +} + +// KeyedAuth implements the pb.Authorizer and pb.Verifier +// interfaces using symmetric, pre-shared keys. +type KeyedAuth struct { + jwt.VerificationKeySet + allowMissing bool +} + +func (k *KeyedAuth) Authorize(ctx context.Context, claims pb.Claims, exp time.Duration) (context.Context, error) { + var now = time.Now() + claims.IssuedAt = &jwt.NumericDate{Time: now} + claims.ExpiresAt = &jwt.NumericDate{Time: now.Add(exp)} + var token, err = jwt.NewWithClaims(jwt.SigningMethodHS256, claims).SignedString(k.Keys[0]) + + if err != nil { + return nil, err + } + return metadata.AppendToOutgoingContext(ctx, "authorization", fmt.Sprintf("Bearer %s", token)), nil +} + +func (k *KeyedAuth) Verify(ctx context.Context, require pb.Capability) (context.Context, context.CancelFunc, pb.Claims, error) { + if claims, err := verifyWithKeys(ctx, require, k.VerificationKeySet, k.allowMissing); err != nil { + return nil, func() {}, claims, status.Error(codes.Unauthenticated, err.Error()) + } else { + ctx, cancel := context.WithDeadline(ctx, claims.ExpiresAt.Time) + return ctx, cancel, claims, nil + } +} + +// NewNoopAuth returns an Authorizer and Verifier which does nothing. +func NewNoopAuth() interface { + pb.Authorizer + pb.Verifier +} { + return &noop{} +} + +type noop struct{} + +func (k *noop) Authorize(ctx context.Context, claims pb.Claims, exp time.Duration) (context.Context, error) { + return ctx, nil +} +func (v *noop) Verify(ctx context.Context, require pb.Capability) (context.Context, context.CancelFunc, pb.Claims, error) { + return ctx, func() {}, pb.Claims{Capability: require}, nil +} + +func verifyWithKeys(ctx context.Context, require pb.Capability, keys jwt.VerificationKeySet, allowMissing bool) (pb.Claims, error) { + var md, _ = metadata.FromIncomingContext(ctx) + var auth = md.Get("authorization") + + if len(auth) == 0 { + if allowMissing { + return pb.Claims{ + Capability: require, + RegisteredClaims: jwt.RegisteredClaims{ + ExpiresAt: jwt.NewNumericDate(time.Now().Add(time.Hour)), + }, + }, nil + } + return errClaims, ErrMissingAuth + } else if !strings.HasPrefix(auth[0], "Bearer ") { + return errClaims, ErrNotBearer + } + var bearer = strings.TrimPrefix(auth[0], "Bearer ") + var claims pb.Claims + + if token, err := jwt.ParseWithClaims(bearer, &claims, + func(token *jwt.Token) (interface{}, error) { return keys, nil }, + jwt.WithExpirationRequired(), + jwt.WithIssuedAt(), + jwt.WithLeeway(time.Second*5), + jwt.WithValidMethods([]string{"HS256", "HS384"}), + ); err != nil { + return errClaims, fmt.Errorf("verifying Authorization: %w", err) + } else if !token.Valid { + panic("token.Valid must be true") + } else if err = verifyCapability(claims.Capability, require); err != nil { + return errClaims, err + } else { + return claims, nil + } +} + +func verifyCapability(actual, require pb.Capability) error { + if actual&require == require { + return nil + } + + // Nicer messages for common capabilities. + for _, i := range []struct { + cap pb.Capability + name string + }{ + {pb.Capability_LIST, "LIST"}, + {pb.Capability_APPLY, "APPLY"}, + {pb.Capability_READ, "READ"}, + {pb.Capability_APPEND, "APPEND"}, + {pb.Capability_REPLICATE, "REPLICATE"}, + } { + if require&i.cap != 0 && actual&i.cap == 0 { + return fmt.Errorf("authorization is missing required %s capability", i.name) + } + } + + return fmt.Errorf("authorization is missing required capability (have %s, but require %s)", + strconv.FormatUint(uint64(actual), 2), strconv.FormatUint(uint64(require), 2)) +} + +var ( + ErrMissingAuth = errors.New("missing or empty Authorization token") + ErrNotBearer = errors.New("invalid or unsupported Authorization header (expected 'Bearer')") + + // errClaims is a defense-in-depth sentinel LabelSelector that won't match anything, + // just in case a caller fails to properly error-check a verification result. + errClaims = pb.Claims{ + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("this-label-will", "never-match"), + }, + } +) diff --git a/auth/auth_test.go b/auth/auth_test.go new file mode 100644 index 00000000..970d9092 --- /dev/null +++ b/auth/auth_test.go @@ -0,0 +1,54 @@ +package auth_test + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" + "go.gazette.dev/core/auth" + pb "go.gazette.dev/core/broker/protocol" + "google.golang.org/grpc/metadata" +) + +func TestKeyedAuthCases(t *testing.T) { + ka1, err := auth.NewKeyedAuth("c2VjcmV0,b3RoZXI=") + require.NoError(t, err) + ka2, err := auth.NewKeyedAuth("b3RoZXI=,c2VjcmV0") + require.NoError(t, err) + kaM, err := auth.NewKeyedAuth("YXNkZg==,AA==") + require.NoError(t, err) + + // Authorize with one KeyedAuth... + ctx, err := ka1.Authorize(context.Background(), + pb.Claims{ + Capability: pb.Capability_APPEND | pb.Capability_LIST, + Selector: pb.MustLabelSelector("hi=there"), + }, time.Hour) + require.NoError(t, err) + + var md, _ = metadata.FromOutgoingContext(ctx) + ctx = metadata.NewIncomingContext(ctx, md) + + // ...and verify with the other. + _, cancel, claims, err := ka2.Verify(ctx, pb.Capability_APPEND) + require.NoError(t, err) + cancel() + require.Equal(t, pb.MustLabelSelector("hi=there"), claims.Selector) + + // Unless the capability doesn't match. + _, _, _, err = ka2.Verify(ctx, pb.Capability_REPLICATE) + require.EqualError(t, err, + "rpc error: code = Unauthenticated desc = authorization is missing required REPLICATE capability") + + // A KeyedAuth with a diferent key rejects it. + _, _, _, err = kaM.Verify(ctx, pb.Capability_APPEND) + require.EqualError(t, err, + "rpc error: code = Unauthenticated desc = verifying Authorization: token signature is invalid: signature is invalid") + + // A KeyedAuth that allows pass-through will accept a request without a token. + _, cancel, claims, err = kaM.Verify(context.Background(), pb.Capability_READ) + require.NoError(t, err) + cancel() + require.Equal(t, pb.MustLabelSelector(""), claims.Selector) +} diff --git a/broker/append_api.go b/broker/append_api.go index 6f14ff7b..7b31f4f7 100644 --- a/broker/append_api.go +++ b/broker/append_api.go @@ -13,7 +13,7 @@ import ( ) // Append dispatches the JournalServer.Append API. -func (svc *Service) Append(stream pb.Journal_AppendServer) (err error) { +func (svc *Service) Append(claims pb.Claims, stream pb.Journal_AppendServer) (err error) { var ( fsm appendFSM req *pb.AppendRequest @@ -38,9 +38,10 @@ func (svc *Service) Append(stream pb.Journal_AppendServer) (err error) { } fsm = appendFSM{ - svc: svc, - ctx: stream.Context(), - req: *req, + svc: svc, + ctx: stream.Context(), + claims: claims, + req: *req, } fsm.run(stream.Recv) @@ -84,7 +85,15 @@ func (svc *Service) Append(stream pb.Journal_AppendServer) (err error) { // proxyAppend forwards an AppendRequest to a resolved peer broker. // Pass request by value as we'll later mutate it (via RecvMsg). func proxyAppend(stream grpc.ServerStream, req pb.AppendRequest, jc pb.JournalClient) error { - var ctx = pb.WithDispatchRoute(stream.Context(), req.Header.Route, req.Header.ProcessId) + // We verified the client's authorization & claims and are running under its context. + // pb.AuthJournalClient will self-sign claims to proxy this journal on the client's behalf. + var ctx = pb.WithClaims(stream.Context(), pb.Claims{ + Capability: pb.Capability_APPEND, + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("name", req.Journal.String()), + }, + }) + ctx = pb.WithDispatchRoute(ctx, req.Header.Route, req.Header.ProcessId) var client, err = jc.Append(ctx) if err != nil { diff --git a/broker/append_api_test.go b/broker/append_api_test.go index 26167590..3f766060 100644 --- a/broker/append_api_test.go +++ b/broker/append_api_test.go @@ -15,6 +15,7 @@ import ( func TestAppendSingle(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) @@ -52,14 +53,14 @@ func TestAppendSingle(t *testing.T) { func TestAppendRegisterCheckAndUpdateSequence(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) broker.initialFragmentLoad() var selector = func(s string) *pb.LabelSelector { - var sel, err = pb.ParseLabelSelector(s) - require.NoError(t, err) + var sel = pb.MustLabelSelector(s) return &sel } // Run a sequence of appends, where each confirms and modifies registers. @@ -125,6 +126,7 @@ func TestAppendRegisterCheckAndUpdateSequence(t *testing.T) { func TestAppendBadlyBehavedClientCases(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) @@ -192,6 +194,7 @@ func TestAppendBadlyBehavedClientCases(t *testing.T) { func TestAppendRequestErrorCases(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) @@ -268,12 +271,30 @@ func TestAppendRequestErrorCases(t *testing.T) { Header: *broker.header("read/only"), }, resp) + // Case: Insufficient claimed capability. + stream, _ = broker.client().Append(pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_READ})) + _, err = stream.CloseAndRecv() + require.ErrorContains(t, err, "authorization is missing required APPEND capability") + + // Case: Insufficient claimed selector. + stream, _ = broker.client().Append(pb.WithClaims(ctx, pb.Claims{ + Capability: pb.Capability_APPEND, + Selector: pb.MustLabelSelector("name=something/else"), + })) + require.NoError(t, stream.Send(&pb.AppendRequest{Journal: "valid/journal", Offset: 1024})) + + resp, err = stream.CloseAndRecv() + require.NoError(t, err) + require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, resp.Status) // Journal not visible to these claims. + require.Len(t, resp.Header.Route.Endpoints, 0) + broker.cleanup() } func TestAppendProxyCases(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) var peer = newMockBroker(t, etcd, pb.ProcessSpec_ID{Zone: "peer", Suffix: "broker"}) diff --git a/broker/append_fsm.go b/broker/append_fsm.go index 3ba9ee44..91a43e78 100644 --- a/broker/append_fsm.go +++ b/broker/append_fsm.go @@ -19,9 +19,10 @@ import ( // typically awaiting a future KeySpace state, as it converges towards the // distributed consistency required for the execution of appends. type appendFSM struct { - svc *Service - ctx context.Context - req pb.AppendRequest + svc *Service + ctx context.Context + claims pb.Claims + req pb.AppendRequest resolved *resolution // Current journal resolution. pln *pipeline // Current replication pipeline. @@ -128,16 +129,14 @@ func (b *appendFSM) returnPipeline() { func (b *appendFSM) onResolve() { b.mustState(stateResolve) - var args = resolveArgs{ - ctx: b.ctx, - journal: b.req.Journal, + var opts = resolveOpts{ mayProxy: !b.req.DoNotProxy, requirePrimary: true, minEtcdRevision: b.readThroughRev, proxyHeader: b.req.Header, } - if b.resolved, b.err = b.svc.resolver.resolve(args); b.err != nil { + if b.resolved, b.err = b.svc.resolver.resolve(b.ctx, b.claims, b.req.Journal, opts); b.err != nil { b.state = stateError b.err = errors.WithMessage(b.err, "resolve") } else if b.resolved.status != pb.Status_OK { @@ -236,9 +235,17 @@ func (b *appendFSM) onStartPipeline() { } b.registers.Assign(&spool.Registers) - // Build a pipeline around |spool|. Note the pipeline Context is bound - // to the replica (rather than our |b.args.ctx|). - b.pln = newPipeline(b.resolved.replica.ctx, b.resolved.Header, spool, b.resolved.replica.spoolCh, b.svc.jc) + // The pipeline Context is bound to the replica (rather than our `b.args.ctx`). + // It will self-sign Claims to allow replication to the journal. + var plnCtx = pb.WithClaims(b.resolved.replica.ctx, pb.Claims{ + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("name", b.req.Journal.StripMeta().String()), + }, + Capability: pb.Capability_REPLICATE, + }) + + // Build a pipeline around `spool`. + b.pln = newPipeline(plnCtx, b.resolved.Header, spool, b.resolved.replica.spoolCh, b.svc.jc) b.state = stateSendPipelineSync } diff --git a/broker/append_fsm_test.go b/broker/append_fsm_test.go index 3590fa12..1dd56aff 100644 --- a/broker/append_fsm_test.go +++ b/broker/append_fsm_test.go @@ -21,16 +21,16 @@ func TestFSMResolve(t *testing.T) { var peer = newMockBroker(t, etcd, pb.ProcessSpec_ID{Zone: "peer", Suffix: "broker"}) // Case: A resolution status error is returned. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "does/not/exist"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "does/not/exist"}) fsm.onResolve() require.Equal(t, stateError, fsm.state) require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, fsm.resolved.status) // Case: Context canceled and resolution is aborted. - fsm = appendFSM{svc: broker.svc, ctx: newCanceledCtx(), req: pb.AppendRequest{ + fsm = newFSM(broker, newCanceledCtx(), pb.AppendRequest{ Journal: "a/journal", Header: &pb.Header{Etcd: fsm.resolved.Header.Etcd}, - }} + }) fsm.req.Header.Etcd.Revision += 1e10 // Future revision blocks indefinitely. fsm.onResolve() require.Equal(t, stateError, fsm.state) @@ -38,19 +38,19 @@ func TestFSMResolve(t *testing.T) { // Case: Resolution success, but we don't own the pipeline. setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 3}, broker.id) - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() require.Equal(t, stateAcquirePipeline, fsm.state) // Case: Resolution success, and we own the pipeline. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.plnReturnCh = make(chan *pipeline) fsm.onResolve() require.Equal(t, stateStartPipeline, fsm.state) // Case: We're not journal primary, but own the pipeline. setTestJournal(broker, pb.JournalSpec{Name: "other/journal", Replication: 3}, peer.id) - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "other/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "other/journal"}) fsm.plnReturnCh = make(chan *pipeline, 1) fsm.onResolve() require.Equal(t, peer.id, fsm.resolved.ProcessId) @@ -69,7 +69,7 @@ func TestFSMAcquirePipeline(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) // Case: success. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStartPipeline)) require.NotNil(t, fsm.plnReturnCh) fsm.returnPipeline() @@ -79,7 +79,7 @@ func TestFSMAcquirePipeline(t *testing.T) { // immediately select-able. // Case: replica route is invalidated while we wait. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.resolved.invalidateCh = newClosedCh() fsm.onAcquirePipeline() @@ -88,7 +88,7 @@ func TestFSMAcquirePipeline(t *testing.T) { fsm.returnPipeline() // Case: request is cancelled while we wait. - fsm = appendFSM{svc: broker.svc, ctx: newCanceledCtx(), req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, newCanceledCtx(), pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.onAcquirePipeline() require.Equal(t, stateError, fsm.state) @@ -110,7 +110,7 @@ func TestFSMStartAndSync(t *testing.T) { // - PROPOSAL_MISMATCH error // - WRONG_ROUTE error // - An unexpected, terminal error. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateRecvPipelineSync)) // Expect peer reads initial fragment proposal for the journal. @@ -190,10 +190,10 @@ func TestFSMStartAndSync(t *testing.T) { fsm.returnPipeline() // Case: New pipeline from scratch, and sync is successful. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateRecvPipelineSync)) - _ = <-peer.ReplReqCh + <-peer.ReplReqCh peer.ReplRespCh <- pb.ReplicateResponse{Status: pb.Status_OK} fsm.onRecvPipelineSync() @@ -202,14 +202,14 @@ func TestFSMStartAndSync(t *testing.T) { // Case: As the pipeline is intact and of the correct route, // expect it's not synchronized again. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStartPipeline)) fsm.onStartPipeline() require.Equal(t, stateUpdateAssignments, fsm.state) fsm.returnPipeline() // Case: A synced pipeline of non-equivalent route is torn down and restarted. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.onAcquirePipeline() fsm.pln.Route = wrongRouteHeader.Route @@ -231,7 +231,7 @@ func TestFSMStartAndSync(t *testing.T) { }(fsm.resolved.replica.pipelineCh, fsm.pln) // Case: context error while awaiting the spool. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStartPipeline)) fsm.ctx = newCanceledCtx() fsm.onStartPipeline() @@ -240,7 +240,7 @@ func TestFSMStartAndSync(t *testing.T) { fsm.returnPipeline() // Case: route invalidation while awaiting the spool. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStartPipeline)) fsm.resolved.invalidateCh = newClosedCh() fsm.onStartPipeline() @@ -265,9 +265,9 @@ func TestFSMUpdateAssignments(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 2}, broker.id, peer.id) // Case: error while updating assignments. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateRecvPipelineSync)) - _ = <-peer.ReplReqCh + <-peer.ReplReqCh peer.ReplRespCh <- pb.ReplicateResponse{Status: pb.Status_OK} fsm.onRecvPipelineSync() fsm.ctx = newCanceledCtx() // Cause updateAssignments to fail. @@ -277,7 +277,7 @@ func TestFSMUpdateAssignments(t *testing.T) { fsm.returnPipeline() // Case: assignments must be updated in Etcd. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateUpdateAssignments)) fsm.onUpdateAssignments() require.Equal(t, stateResolve, fsm.state) @@ -310,25 +310,25 @@ func TestFSMDesiredReplicas(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "too/few", Replication: 3}, peer.id, broker.id) // Case: local primary with correct number of replicas. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.returnPipeline() // Case: remote primary with correct number of replicas. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "remote/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "remote/journal"}) fsm.onResolve() fsm.onAwaitDesiredReplicas() require.Equal(t, stateProxy, fsm.state) // Case: journal with too many replicas. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "too/many"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "too/many"}) fsm.onResolve() fsm.onAwaitDesiredReplicas() require.Equal(t, stateResolve, fsm.state) require.Equal(t, fsm.resolved.Etcd.Revision+1, fsm.readThroughRev) // Case: journal with too few replicas. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "too/few"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "too/few"}) fsm.onResolve() fsm.onAwaitDesiredReplicas() require.Equal(t, stateError, fsm.state) @@ -346,7 +346,7 @@ func TestFSMValidatePreconditions(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) // Case: We're canceled while awaiting the first remote fragment refresh. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.ctx = newCanceledCtx() fsm.onValidatePreconditions() @@ -355,7 +355,7 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: Route is invalidated while awaiting first remote fragment refresh. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.resolved.invalidateCh = newClosedCh() fsm.onValidatePreconditions() @@ -363,7 +363,7 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: Spool & fragment index agree on non-zero end offset. Success. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.onAcquirePipeline() fsm.resolved.replica.index.ReplaceRemote(fragment.CoverSet{ @@ -374,10 +374,10 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: Register selector is not matched, but journal has no registers. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{ + fsm = newFSM(broker, ctx, pb.AppendRequest{ Journal: "a/journal", CheckRegisters: &pb.LabelSelector{Include: pb.MustLabelSet("not", "matched")}, - }} + }) require.True(t, fsm.runTo(stateStreamContent)) // Set fixture for next run. @@ -385,10 +385,10 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: Register selector doesn't match non-empty journal registers. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{ + fsm = newFSM(broker, ctx, pb.AppendRequest{ Journal: "a/journal", CheckRegisters: &pb.LabelSelector{Include: pb.MustLabelSet("not", "matched")}, - }} + }) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.onValidatePreconditions() require.Equal(t, stateError, fsm.state) @@ -396,15 +396,15 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: Register selector _is_ matched. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{ + fsm = newFSM(broker, ctx, pb.AppendRequest{ Journal: "a/journal", CheckRegisters: &pb.LabelSelector{Exclude: pb.MustLabelSet("is", "matched")}, - }} + }) require.True(t, fsm.runTo(stateStreamContent)) fsm.returnPipeline() // Case: remote index contains a greater offset than the pipeline, and request omits offset. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.resolved.replica.index.ReplaceRemote(fragment.CoverSet{ {Fragment: pb.Fragment{End: 456}}, @@ -416,14 +416,14 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: remote index contains a greater offset, request omits offset, but journal is not writable. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() fsm.resolved.journalSpec.Flags = pb.JournalSpec_O_RDONLY require.True(t, fsm.runTo(stateStreamContent)) fsm.returnPipeline() // Case: request offset doesn't match the max journal offset. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal", Offset: 455}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal", Offset: 455}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.onValidatePreconditions() require.Equal(t, stateError, fsm.state) @@ -431,7 +431,7 @@ func TestFSMValidatePreconditions(t *testing.T) { fsm.returnPipeline() // Case: request offset does match the max journal offset. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal", Offset: 456}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal", Offset: 456}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.onValidatePreconditions() require.Equal(t, stateSendPipelineSync, fsm.state) @@ -466,12 +466,12 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { // Case: successful append is streamed from the client, and verifies & updates // registers which are initially known only to peers (and not this appendFSM). - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{ + var fsm = newFSM(broker, ctx, pb.AppendRequest{ Journal: "a/journal", CheckRegisters: &pb.LabelSelector{Include: pb.MustLabelSet("before", "")}, UnionRegisters: boxLabels("after", ""), SubtractRegisters: boxLabels("before", ""), - }} + }) fsm.onResolve() // Asynchronously run the expected peer message flow. @@ -570,10 +570,10 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.Nil(t, fsm.plnReturnCh) // Case: A request with register modifications which writes no bytes is an error. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{ + fsm = newFSM(broker, ctx, pb.AppendRequest{ Journal: "a/journal", UnionRegisters: boxLabels("foo", ""), - }} + }) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{}, nil) // Intent to commit. @@ -605,7 +605,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { // Case: Expect a non-validating AppendRequest is treated as a client error, // and triggers rollback. Note that an updating proposal is not required and // is not sent this time. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("baz")}, nil) // Valid 1st chunk. @@ -620,7 +620,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.EqualError(t, fsm.err, `append stream: Journal: cannot begin with '/' (/invalid)`) // Case: Valid but unexpected non-content chunk. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("baz")}, nil) // Valid 1st chunk. @@ -638,7 +638,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { // and triggers a rollback. Also change the compression spec but expect an // updated proposal is still not sent, as the spool is non-empty and not // over the Fragment Length. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.resolved.journalSpec.Fragment.CompressionCodec = pb.CompressionCodec_GZIP @@ -656,7 +656,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.EqualError(t, fsm.err, `append stream: unexpected EOF`) // Case: Expect any other client read error triggers a rollback. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("baz")}, nil) // 1st chunk. @@ -671,7 +671,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.EqualError(t, fsm.err, `append stream: some error`) // Case: Expect *not* reading an EOF after a commit chunk triggers an error and rollback. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("baz")}, nil) // 1st chunk. @@ -688,7 +688,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.EqualError(t, fsm.err, `append stream: expected EOF after empty Content chunk`) // Case: journal writes are disallowed. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.resolved.journalSpec.Flags = pb.JournalSpec_O_RDONLY @@ -704,7 +704,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.Equal(t, pb.Status_NOT_ALLOWED, fsm.resolved.status) // Case: journal writes are still disallowed, but append is zero-length. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.onStreamContent(&pb.AppendRequest{}, nil) // Intent to commit. @@ -719,7 +719,7 @@ func TestFSMStreamAndReadAcknowledgements(t *testing.T) { require.Equal(t, pb.Status_OK, fsm.resolved.status) // Case: Writes are allowed again, but pipeline is broken. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.runTo(stateStreamContent) fsm.resolved.journalSpec.Flags = pb.JournalSpec_O_RDWR // Reset. @@ -785,7 +785,7 @@ func TestFSMRunBasicCases(t *testing.T) { } // Case: successful append is streamed from the client. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.run(makeRecv([]appendChunk{ {req: &pb.AppendRequest{Content: []byte("bar")}}, {req: &pb.AppendRequest{Content: []byte("bing")}}, @@ -807,7 +807,7 @@ func TestFSMRunBasicCases(t *testing.T) { // Case: client timeout triggers a context.DeadlineExceeded. var uninstall = installAppendTimeoutFixture() - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.run(makeRecv([]appendChunk{ {req: &pb.AppendRequest{Content: []byte("bar")}}, // Wait indefinitely for next chunk. @@ -818,7 +818,7 @@ func TestFSMRunBasicCases(t *testing.T) { uninstall() // Case: client read error. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.run(makeRecv([]appendChunk{ {req: &pb.AppendRequest{Content: []byte("bar")}}, {err: errors.New("foobar")}, @@ -828,7 +828,7 @@ func TestFSMRunBasicCases(t *testing.T) { require.EqualError(t, fsm.err, "append stream: foobar") // Case: terminal error prior to content streaming. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "does/not/exist"}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "does/not/exist"}) fsm.run(nil) // |recv| not called. require.Equal(t, stateError, fsm.state) @@ -847,8 +847,8 @@ func TestFSMPipelineRace(t *testing.T) { broker.initialFragmentLoad() // Start two raced requests. - var fsm1 = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} - var fsm2 = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm1 = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) + var fsm2 = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm1.onResolve() fsm2.onResolve() fsm1.onAcquirePipeline() // Wins race to acquire the pipeline. @@ -952,7 +952,7 @@ func TestFSMOffsetResetFlow(t *testing.T) { broker.initialFragmentLoad() // Add a remote Fragment fixture which ends at offset 1024. - var res, _ = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + var res, _ = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) res.replica.index.ReplaceRemote(fragment.CoverSet{fragment.Fragment{ Fragment: pb.Fragment{Journal: "a/journal", Begin: 0, End: 1024}}}) @@ -972,13 +972,13 @@ func TestFSMOffsetResetFlow(t *testing.T) { }() // Part 1: Offset is not provided, and remote index has a fragment with larger offset. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.run(nil) require.Equal(t, stateError, fsm.state) require.Equal(t, pb.Status_INDEX_HAS_GREATER_OFFSET, fsm.resolved.status) // Part 2: We now submit a request offset which matches the remote fragment offset. - fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal", Offset: 1024}} + fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal", Offset: 1024}) require.True(t, fsm.runTo(stateValidatePreconditions)) fsm.onValidatePreconditions() fsm.onSendPipelineSync() @@ -1037,3 +1037,7 @@ func newCanceledCtx() context.Context { cancel() return ctx } + +func newFSM(b *testBroker, ctx context.Context, req pb.AppendRequest) appendFSM { + return appendFSM{svc: b.svc, ctx: ctx, claims: allClaims, req: req} +} diff --git a/broker/client/appender.go b/broker/client/appender.go index 1423e81b..264da2ac 100644 --- a/broker/client/appender.go +++ b/broker/client/appender.go @@ -136,8 +136,14 @@ func (a *Appender) lazyInit() (err error) { return pb.ExtendContext(err, "Request") } + var ctx = pb.WithClaims(a.ctx, pb.Claims{ + Capability: pb.Capability_APPEND, + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("name", a.Request.Journal.StripMeta().String()), + }, + }) a.stream, err = a.client.Append( - pb.WithDispatchItemRoute(a.ctx, a.client, a.Request.Journal.String(), true)) + pb.WithDispatchItemRoute(ctx, a.client, a.Request.Journal.String(), true)) if err == nil { // Send request preamble metadata prior to append content chunks. diff --git a/broker/client/retry_reader.go b/broker/client/retry_reader.go index a0e5f4a4..72e731b5 100644 --- a/broker/client/retry_reader.go +++ b/broker/client/retry_reader.go @@ -53,12 +53,13 @@ func (rr *RetryReader) Offset() int64 { // Read returns the next bytes of journal content. It will return a non-nil // error in the following cases: -// * Cancel is called, or the RetryReader context is cancelled. -// * The broker returns OFFSET_NOT_YET_AVAILABLE (ErrOffsetNotYetAvailable) -// for a non-blocking ReadRequest. -// * An offset jump occurred (ErrOffsetJump), in which case the client -// should inspect the new Offset and may continue reading if desired. -// * The broker returns io.EOF upon reaching the requested EndOffset. +// - Cancel is called, or the RetryReader context is cancelled. +// - The broker returns OFFSET_NOT_YET_AVAILABLE (ErrOffsetNotYetAvailable) +// for a non-blocking ReadRequest. +// - An offset jump occurred (ErrOffsetJump), in which case the client +// should inspect the new Offset and may continue reading if desired. +// - The broker returns io.EOF upon reaching the requested EndOffset. +// // All other errors are retried. func (rr *RetryReader) Read(p []byte) (n int, err error) { for attempt := 0; true; attempt++ { @@ -99,10 +100,14 @@ func (rr *RetryReader) Read(p []byte) (n int, err error) { } else { return // Surface to caller. } - case io.EOF, ErrInsufficientJournalBrokers, ErrNotJournalBroker, ErrJournalNotFound: + case ErrInsufficientJournalBrokers, ErrNotJournalBroker, ErrJournalNotFound: // Suppress logging for expected errors on first read attempt. // We may be racing a concurrent Etcd watch and assignment of the broker cluster. squelch = attempt == 0 + case io.EOF: + // Repeated EOF is common if topology changes or authorizations + // expire on a journal with no active appends. + squelch = true default: } diff --git a/broker/e2e_test.go b/broker/e2e_test.go index 5e37d849..a1bd5f82 100644 --- a/broker/e2e_test.go +++ b/broker/e2e_test.go @@ -33,14 +33,14 @@ func TestE2EAppendAndReplicatedRead(t *testing.T) { var rTwo, _ = broker.client().Read(ctx, &pb.ReadRequest{Journal: "journal/two", Block: true}) // First Append is served by |broker|, with its Read served by |peer|. - var stream, _ = broker.client().Append(ctx) + var stream, _ = broker.client().Append(pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND})) require.NoError(t, stream.Send(&pb.AppendRequest{Journal: "journal/one"})) require.NoError(t, stream.Send(&pb.AppendRequest{Content: []byte("hello")})) require.NoError(t, stream.Send(&pb.AppendRequest{})) _, _ = stream.CloseAndRecv() // Second Append is served by |peer| (through |broker|), with its Read served by |broker|. - stream, _ = broker.client().Append(ctx) + stream, _ = broker.client().Append(pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND})) require.NoError(t, stream.Send(&pb.AppendRequest{Journal: "journal/two"})) require.NoError(t, stream.Send(&pb.AppendRequest{Content: []byte("world!")})) require.NoError(t, stream.Send(&pb.AppendRequest{})) @@ -76,7 +76,7 @@ func TestE2EShutdownWithOngoingAppend(t *testing.T) { // the required knobs to ensure the Append wins required races vs replica // shutdown (eg, after updating assignments it must complete re-resolution // and begin streaming content _before_ this test aborts the journal). - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStreamContent)) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("woot!")}, nil) fsm.onStreamContent(&pb.AppendRequest{}, nil) // Commit intent. @@ -112,7 +112,7 @@ func TestE2EShutdownWithOngoingAppendWhichLosesRace(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) broker.initialFragmentLoad() - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) fsm.onResolve() require.Equal(t, pb.Status_OK, fsm.resolved.status) @@ -139,7 +139,7 @@ func TestE2EReassignmentWithOngoingAppend(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) broker.initialFragmentLoad() - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStreamContent)) fsm.onStreamContent(&pb.AppendRequest{Content: []byte("woot!")}, nil) fsm.onStreamContent(&pb.AppendRequest{}, nil) // Commit intent. @@ -178,14 +178,14 @@ func TestE2EShutdownWithProxyAppend(t *testing.T) { broker.initialFragmentLoad() // Start a long-lived appendFSM, which builds and holds the pipeline. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateStreamContent)) var conn, client = newDialedClient(t, broker) defer conn.Close() // Start an Append RPC which will queue behind appendFSM. - var app, err = client.Append(ctx) + var app, err = client.Append(pb.WithClaims(ctx, allClaims)) require.NoError(t, err) require.NoError(t, app.Send(&pb.AppendRequest{Journal: "a/journal"})) require.NoError(t, app.Send(&pb.AppendRequest{Content: []byte("world!")})) @@ -341,7 +341,8 @@ func newDialedClient(t *testing.T, bk *testBroker) (*grpc.ClientConn, pb.Journal grpc.WithTransportCredentials(pb.NewDispatchedCredentials(tlsConfig, bk.srv.Endpoint())), ) require.NoError(t, err) - return conn, pb.NewJournalClient(conn) + + return conn, pb.NewAuthJournalClient(pb.NewJournalClient(conn), bk.auth) } // ensureStreamsAreStarted completes a no-op RPC, the purpose of which is to diff --git a/broker/fragments_api.go b/broker/fragments_api.go index c6a69f6d..b8a507bf 100644 --- a/broker/fragments_api.go +++ b/broker/fragments_api.go @@ -14,7 +14,7 @@ import ( var defaultPageLimit = int32(1000) // ListFragments dispatches the JournalServer.ListFragments API. -func (svc *Service) ListFragments(ctx context.Context, req *pb.FragmentsRequest) (resp *pb.FragmentsResponse, err error) { +func (svc *Service) ListFragments(ctx context.Context, claims pb.Claims, req *pb.FragmentsRequest) (resp *pb.FragmentsResponse, err error) { var res *resolution defer instrumentJournalServerRPC("ListFragments", &err, &res)() @@ -36,9 +36,7 @@ func (svc *Service) ListFragments(ctx context.Context, req *pb.FragmentsRequest) req.PageLimit = defaultPageLimit } - res, err = svc.resolver.resolve(resolveArgs{ - ctx: ctx, - journal: req.Journal, + res, err = svc.resolver.resolve(ctx, claims, req.Journal, resolveOpts{ mayProxy: !req.DoNotProxy, requirePrimary: false, proxyHeader: req.Header, diff --git a/broker/fragments_api_test.go b/broker/fragments_api_test.go index ae4f4cb3..c0971693 100644 --- a/broker/fragments_api_test.go +++ b/broker/fragments_api_test.go @@ -71,6 +71,24 @@ func TestFragmentsResolutionCases(t *testing.T) { NextPageToken: fragments[5].Spec.End, }, resp) + // Case: Insufficient claimed capability. + _, err = broker.client().ListFragments( + pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}), + &pb.FragmentsRequest{Journal: "proxy/journal"}) + require.ErrorContains(t, err, "authorization is missing required READ capability") + + // Case: Insufficient claimed selector. + resp, err = broker.client().ListFragments( + pb.WithClaims(ctx, + pb.Claims{ + Capability: pb.Capability_READ, + Selector: pb.MustLabelSelector("name=something/else"), + }), + &pb.FragmentsRequest{Journal: "proxy/journal"}) + require.NoError(t, err) + require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, resp.Status) // Journal not visible to these claims. + require.Len(t, resp.Header.Route.Endpoints, 0) + broker.cleanup() } diff --git a/broker/http_gateway/http_gateway.go b/broker/http_gateway/http_gateway.go index abbd41c5..95270287 100644 --- a/broker/http_gateway/http_gateway.go +++ b/broker/http_gateway/http_gateway.go @@ -18,6 +18,7 @@ import ( log "github.com/sirupsen/logrus" "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" + "google.golang.org/grpc/metadata" ) // Gateway presents an HTTP gateway to Gazette brokers, by mapping GET, HEAD, @@ -39,6 +40,14 @@ func NewGateway(client pb.RoutedJournalClient) *Gateway { } func (h *Gateway) ServeHTTP(w http.ResponseWriter, r *http.Request) { + const AUTHORIZATION = "authorization" + + // Expose an HTTP Authorization header for the gRPC handler. + if token := r.Header.Get(AUTHORIZATION); token != "" { + r = r.WithContext(metadata.NewIncomingContext( + r.Context(), metadata.Pairs(AUTHORIZATION, token))) + } + switch r.Method { case "GET", "HEAD": h.serveRead(w, r) diff --git a/broker/list_apply_api.go b/broker/list_apply_api.go index 6d2e2fa9..7f8f7dc9 100644 --- a/broker/list_apply_api.go +++ b/broker/list_apply_api.go @@ -2,6 +2,7 @@ package broker import ( "context" + "fmt" "net" "strings" @@ -11,11 +12,13 @@ import ( pb "go.gazette.dev/core/broker/protocol" pbx "go.gazette.dev/core/broker/protocol/ext" "go.gazette.dev/core/keyspace" + "google.golang.org/grpc/codes" "google.golang.org/grpc/peer" + "google.golang.org/grpc/status" ) // List dispatches the JournalServer.List API. -func (svc *Service) List(req *pb.ListRequest, stream pb.Journal_ListServer) (err error) { +func (svc *Service) List(claims pb.Claims, req *pb.ListRequest, stream pb.Journal_ListServer) (err error) { defer instrumentJournalServerRPC("List", &err, nil)() defer func() { @@ -36,7 +39,12 @@ func (svc *Service) List(req *pb.ListRequest, stream pb.Journal_ListServer) (err // Gazette has historically offered a special "prefix" label which matches // slash-terminated prefixes of a journal name. Today, it's implemented in // terms of a LabelSelector prefix match. - for _, set := range []*pb.LabelSet{&req.Selector.Include, &req.Selector.Exclude} { + for _, set := range []*pb.LabelSet{ + &req.Selector.Include, + &req.Selector.Exclude, + &claims.Selector.Include, + &claims.Selector.Exclude, + } { if prefix := set.ValuesOf("prefix"); len(prefix) != 0 { for _, val := range prefix { set.AddValue("name:prefix", val) @@ -61,6 +69,7 @@ func (svc *Service) List(req *pb.ListRequest, stream pb.Journal_ListServer) (err for { var resp, err = listRound( stream.Context(), + &claims, &req.Selector, svc.resolver.state, &minEtcdRevision, @@ -101,6 +110,7 @@ func (svc *Service) List(req *pb.ListRequest, stream pb.Journal_ListServer) (err func listRound( ctx context.Context, + claims *pb.Claims, selector *pb.LabelSelector, state *allocator.State, minEtcdRevision *int64, @@ -145,7 +155,7 @@ func listRound( // LabelSetExt() truncates `scratch` while re-using its storage. scratch = spec.LabelSetExt(scratch) - if selector.Matches(scratch) { + if selector.Matches(scratch) && claims.Selector.Matches(scratch) { if item.Raw.ModRevision > maxModRevision { maxModRevision = item.Raw.ModRevision } @@ -191,7 +201,7 @@ func listRound( } // Apply dispatches the JournalServer.Apply API. -func (svc *Service) Apply(ctx context.Context, req *pb.ApplyRequest) (resp *pb.ApplyResponse, err error) { +func (svc *Service) Apply(ctx context.Context, claims pb.Claims, req *pb.ApplyRequest) (resp *pb.ApplyResponse, err error) { defer instrumentJournalServerRPC("Apply", &err, nil)() defer func() { @@ -206,7 +216,15 @@ func (svc *Service) Apply(ctx context.Context, req *pb.ApplyRequest) (resp *pb.A }() if err = req.Validate(); err != nil { - return new(pb.ApplyResponse), err + return nil, err + } + + // The Apply API is authorized exclusively through the "name" label. + var authorizeJournal = func(claims *pb.Claims, journal pb.Journal) error { + if !claims.Selector.Matches(pb.MustLabelSet("name", journal.String())) { + return status.Error(codes.Unauthenticated, fmt.Sprintf("not authorized to %s", journal)) + } + return nil } var cmp []clientv3.Cmp @@ -217,9 +235,15 @@ func (svc *Service) Apply(ctx context.Context, req *pb.ApplyRequest) (resp *pb.A var key string if change.Upsert != nil { + if err = authorizeJournal(&claims, change.Upsert.Name); err != nil { + return nil, err + } key = allocator.ItemKey(s.KS, change.Upsert.Name.String()) ops = append(ops, clientv3.OpPut(key, change.Upsert.MarshalString())) } else { + if err = authorizeJournal(&claims, change.Delete); err != nil { + return nil, err + } key = allocator.ItemKey(s.KS, change.Delete.String()) ops = append(ops, clientv3.OpDelete(key)) } diff --git a/broker/list_apply_api_test.go b/broker/list_apply_api_test.go index d4e93384..1c2973f5 100644 --- a/broker/list_apply_api_test.go +++ b/broker/list_apply_api_test.go @@ -155,6 +155,25 @@ func TestListCases(t *testing.T) { _, err = stream.Recv() require.Regexp(t, `.* Selector.Include.Labels\["prefix"\]: expected trailing '/' (.*)`, err) + // Case: Insufficient claimed capability. + stream, err = broker.client().List( + pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}), + &pb.ListRequest{Selector: pb.LabelSelector{}}) + require.NoError(t, err) + _, err = stream.Recv() + require.ErrorContains(t, err, "authorization is missing required LIST capability") + + // Case: Insufficient claimed selector. + stream, err = broker.client().List( + pb.WithClaims(ctx, + pb.Claims{ + Capability: pb.Capability_LIST, + Selector: pb.MustLabelSelector("name=something/else"), + }), + &pb.ListRequest{Selector: pb.LabelSelector{}}) + require.NoError(t, err) + verify(stream) // All journals filtered by the claims selector. + // Case: streaming watch of a prefix. var cancelCtx, cancel = context.WithCancel(ctx) stream, err = broker.client().List(cancelCtx, &pb.ListRequest{ @@ -306,10 +325,45 @@ func TestApplyCases(t *testing.T) { })).Status) // Case: Invalid requests fail with an error. - var _, err = broker.client().Apply(ctx, &pb.ApplyRequest{ + var _, err = broker.client().Apply(pb.WithClaims(ctx, allClaims), &pb.ApplyRequest{ Changes: []pb.ApplyRequest_Change{{Delete: "invalid journal name"}}, }) require.Regexp(t, `.* Changes\[0\].Delete: not a valid token \(invalid journal name\)`, err) + // Case: Insufficient claimed capability. + _, err = broker.client().Apply( + pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_READ}), + &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{{Delete: "journal/A"}}, + }) + require.ErrorContains(t, err, "authorization is missing required APPLY capability") + + var ctxNarrowClaims = pb.WithClaims(ctx, + pb.Claims{ + Capability: pb.Capability_APPLY, + Selector: pb.MustLabelSelector("name=something/else"), + }) + + // Case: Insufficient claimed selector on delete. + _, err = broker.client().Apply( + ctxNarrowClaims, + &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{{Delete: "journal/A", ExpectModRevision: -1}}, + }) + require.ErrorContains(t, err, "not authorized to journal/A") + + // Case: Insufficient claimed selector on upsert. + _, err = broker.client().Apply( + ctxNarrowClaims, + &pb.ApplyRequest{ + Changes: []pb.ApplyRequest_Change{{Upsert: &specB}}, + }) + require.ErrorContains(t, err, "not authorized to journal/B") + broker.cleanup() } + +var allClaims = pb.Claims{ + Capability: pb.Capability_ALL, // All APIs. + Selector: pb.LabelSelector{}, // Match anything. +} diff --git a/broker/protocol/auth.go b/broker/protocol/auth.go new file mode 100644 index 00000000..ef030561 --- /dev/null +++ b/broker/protocol/auth.go @@ -0,0 +1,270 @@ +package protocol + +import ( + "context" + "math" + "time" + + "github.com/golang-jwt/jwt/v5" + grpc "google.golang.org/grpc" +) + +// Capability is a bit-mask of authorized capabilities to a resource. +type Capability uint32 + +const ( + Capability_LIST Capability = 1 << 1 + Capability_APPLY Capability = 1 << 2 + Capability_READ Capability = 1 << 3 + Capability_APPEND Capability = 1 << 4 + Capability_REPLICATE Capability = 1 << 5 + + // Applications may use Capability bits starting at 1 << 16. + + Capability_ALL Capability = math.MaxUint32 +) + +// Claims reflect the scope of an authorization. They grant the client the +// indicated Capability against resources matched by the corresponding +// Selector. +type Claims struct { + Capability Capability `json:"cap"` + Selector LabelSelector `json:"sel"` + jwt.RegisteredClaims +} + +// WithClaims attaches desired Claims to a Context. +// The attached claims will be used by AuthJournalClient to obtain Authorization tokens. +func WithClaims(ctx context.Context, claims Claims) context.Context { + return context.WithValue(ctx, claimsCtxKey{}, claims) +} + +// GetClaims retrieves attached Claims from a Context. +func GetClaims(ctx context.Context) (Claims, bool) { + var claims, ok = ctx.Value(claimsCtxKey{}).(Claims) + return claims, ok +} + +// Authorizer returns a Context which incorporates a gRPC Authorization for the given Claims. +// It could do so by directly signing the claims with a pre-shared key, +// or by requesting a signature from a separate authorization service. +type Authorizer interface { + Authorize(ctx context.Context, claims Claims, exp time.Duration) (context.Context, error) +} + +// Verifier verifies an Authorization token, returning its validated Claims. +type Verifier interface { + Verify(ctx context.Context, require Capability) (context.Context, context.CancelFunc, Claims, error) +} + +// NewAuthJournalClient returns an *AuthJournalClient which uses the Authorizer +// to obtain and attach an Authorization bearer token to every issued request. +func NewAuthJournalClient(jc JournalClient, auth Authorizer) *AuthJournalClient { + return &AuthJournalClient{Authorizer: auth, Inner: jc} +} + +type AuthJournalClient struct { + Authorizer Authorizer + Inner JournalClient +} + +func (a *AuthJournalClient) List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (Journal_ListClient, error) { + var claims, ok = GetClaims(ctx) + if !ok { + claims = Claims{ + Capability: Capability_LIST, + Selector: in.Selector, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(in.Watch)); err != nil { + return nil, err + } else { + return a.Inner.List(ctx, in, opts...) + } +} + +func (a *AuthJournalClient) Apply(ctx context.Context, in *ApplyRequest, opts ...grpc.CallOption) (*ApplyResponse, error) { + var claims, ok = GetClaims(ctx) + if !ok { + claims = Claims{Capability: Capability_APPLY} + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.Apply(ctx, in, opts...) + } +} + +func (a *AuthJournalClient) Read(ctx context.Context, in *ReadRequest, opts ...grpc.CallOption) (Journal_ReadClient, error) { + var claims, ok = GetClaims(ctx) + if !ok { + claims = Claims{ + Capability: Capability_READ, + Selector: LabelSelector{ + Include: MustLabelSet("name", in.Journal.StripMeta().String()), + }, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(true)); err != nil { + return nil, err + } else { + return a.Inner.Read(ctx, in, opts...) + } +} + +func (a *AuthJournalClient) Append(ctx context.Context, opts ...grpc.CallOption) (Journal_AppendClient, error) { + var claims, ok = GetClaims(ctx) + if !ok { + panic("Append requires a context having WithClaims") + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.Append(ctx, opts...) + } +} + +func (a *AuthJournalClient) Replicate(ctx context.Context, opts ...grpc.CallOption) (Journal_ReplicateClient, error) { + var claims, ok = GetClaims(ctx) + if !ok { + panic("Replicate requires a context having WithClaims") + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(true)); err != nil { + return nil, err + } else { + return a.Inner.Replicate(ctx, opts...) + } +} + +func (a *AuthJournalClient) ListFragments(ctx context.Context, in *FragmentsRequest, opts ...grpc.CallOption) (*FragmentsResponse, error) { + var claims, ok = GetClaims(ctx) + if !ok { + claims = Claims{ + Capability: Capability_READ, // Allows direct fragment access. + Selector: LabelSelector{ + Include: MustLabelSet("name", in.Journal.StripMeta().String()), + }, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.ListFragments(ctx, in, opts...) + } +} + +func withExp(blocking bool) time.Duration { + if blocking { + return time.Hour + } else { + return time.Minute + } +} + +// AuthJournalServer is similar to JournalServer except: +// - Requests have already been verified with accompanying Claims. +// - The Context or Stream.Context() argument may be subject to a deadline +// bound to the expiration of the user's Claims. +type AuthJournalServer interface { + List(Claims, *ListRequest, Journal_ListServer) error + Apply(context.Context, Claims, *ApplyRequest) (*ApplyResponse, error) + Read(Claims, *ReadRequest, Journal_ReadServer) error + Append(Claims, Journal_AppendServer) error + Replicate(Claims, Journal_ReplicateServer) error + ListFragments(context.Context, Claims, *FragmentsRequest) (*FragmentsResponse, error) +} + +// NewVerifiedJournalServer adapts an AuthJournalServer into a JournalServer by +// using the provided Verifier to verify incoming request Authorizations. +func NewVerifiedJournalServer(ajs AuthJournalServer, verifier Verifier) *VerifiedJournalServer { + return &VerifiedJournalServer{ + Verifier: verifier, + Inner: ajs, + } +} + +type VerifiedJournalServer struct { + Verifier Verifier + Inner AuthJournalServer +} + +func (s *VerifiedJournalServer) List(req *ListRequest, stream Journal_ListServer) error { + if ctx, cancel, claims, err := s.Verifier.Verify(stream.Context(), Capability_LIST); err != nil { + return err + } else { + defer cancel() + return s.Inner.List(claims, req, verifiedListServer{ctx, stream}) + } +} + +func (s *VerifiedJournalServer) Apply(ctx context.Context, req *ApplyRequest) (*ApplyResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, Capability_APPLY); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.Apply(ctx, claims, req) + } +} + +func (s *VerifiedJournalServer) Read(req *ReadRequest, stream Journal_ReadServer) error { + if ctx, cancel, claims, err := s.Verifier.Verify(stream.Context(), Capability_READ); err != nil { + return err + } else { + defer cancel() + return s.Inner.Read(claims, req, verifiedReadServer{ctx, stream}) + } +} + +func (s *VerifiedJournalServer) Append(stream Journal_AppendServer) error { + if ctx, cancel, claims, err := s.Verifier.Verify(stream.Context(), Capability_APPEND); err != nil { + return err + } else { + defer cancel() + return s.Inner.Append(claims, verifiedAppendServer{ctx, stream}) + } +} + +func (s *VerifiedJournalServer) Replicate(stream Journal_ReplicateServer) error { + if ctx, cancel, claims, err := s.Verifier.Verify(stream.Context(), Capability_REPLICATE); err != nil { + return err + } else { + defer cancel() + return s.Inner.Replicate(claims, verifiedReplicateServer{ctx, stream}) + } +} + +func (s *VerifiedJournalServer) ListFragments(ctx context.Context, req *FragmentsRequest) (*FragmentsResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, Capability_READ); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.ListFragments(ctx, claims, req) + } +} + +var _ JournalServer = &VerifiedJournalServer{} +var _ JournalClient = &AuthJournalClient{} + +type claimsCtxKey struct{} + +type verifiedListServer struct { + ctx context.Context + Journal_ListServer +} +type verifiedReadServer struct { + ctx context.Context + Journal_ReadServer +} +type verifiedAppendServer struct { + ctx context.Context + Journal_AppendServer +} +type verifiedReplicateServer struct { + ctx context.Context + Journal_ReplicateServer +} + +func (s verifiedListServer) Context() context.Context { return s.ctx } +func (s verifiedReadServer) Context() context.Context { return s.ctx } +func (s verifiedAppendServer) Context() context.Context { return s.ctx } +func (s verifiedReplicateServer) Context() context.Context { return s.ctx } diff --git a/broker/protocol/label_extensions.go b/broker/protocol/label_extensions.go index 0fe416f6..2d0cb35c 100644 --- a/broker/protocol/label_extensions.go +++ b/broker/protocol/label_extensions.go @@ -462,6 +462,16 @@ func ParseLabelSelector(s string) (LabelSelector, error) { return out, out.Validate() } +// MustLabelSelector is a convenience for constructing a parsed LabelSelector. +// The LabelSelector must parse and Validate or MustLabelSelector panics. +func MustLabelSelector(selector string) LabelSelector { + if out, err := ParseLabelSelector(selector); err != nil { + panic(err.Error()) + } else { + return out + } +} + func parseSetParts(set *LabelSet, name, s string) error { for len(s) != 0 { var m []string diff --git a/broker/protocol/rpc_extensions.go b/broker/protocol/rpc_extensions.go index dca30eba..255d3e45 100644 --- a/broker/protocol/rpc_extensions.go +++ b/broker/protocol/rpc_extensions.go @@ -16,12 +16,16 @@ type RoutedJournalClient interface { DispatchRouter } +// ComposedRoutedJournalClient implements the RoutedJournalClient interface +// by composing separate implementations of its constituent interfaces. +type ComposedRoutedJournalClient struct { + JournalClient + DispatchRouter +} + // NewRoutedJournalClient composes a JournalClient and DispatchRouter. -func NewRoutedJournalClient(jc JournalClient, dr DispatchRouter) RoutedJournalClient { - return struct { - JournalClient - DispatchRouter - }{ +func NewRoutedJournalClient(jc JournalClient, dr DispatchRouter) *ComposedRoutedJournalClient { + return &ComposedRoutedJournalClient{ JournalClient: jc, DispatchRouter: dr, } diff --git a/broker/read_api.go b/broker/read_api.go index e4edfad4..39ff8cc2 100644 --- a/broker/read_api.go +++ b/broker/read_api.go @@ -13,7 +13,7 @@ import ( ) // Read dispatches the JournalServer.Read API. -func (svc *Service) Read(req *pb.ReadRequest, stream pb.Journal_ReadServer) (err error) { +func (svc *Service) Read(claims pb.Claims, req *pb.ReadRequest, stream pb.Journal_ReadServer) (err error) { var resolved *resolution defer instrumentJournalServerRPC("Read", &err, &resolved)() @@ -32,9 +32,7 @@ func (svc *Service) Read(req *pb.ReadRequest, stream pb.Journal_ReadServer) (err return err } - resolved, err = svc.resolver.resolve(resolveArgs{ - ctx: stream.Context(), - journal: req.Journal, + resolved, err = svc.resolver.resolve(stream.Context(), claims, req.Journal, resolveOpts{ mayProxy: !req.DoNotProxy, requirePrimary: false, proxyHeader: req.Header, @@ -56,6 +54,8 @@ func (svc *Service) Read(req *pb.ReadRequest, stream pb.Journal_ReadServer) (err // proxyRead forwards a ReadRequest to a resolved peer broker. func proxyRead(stream grpc.ServerStream, req *pb.ReadRequest, jc pb.JournalClient, stopCh <-chan struct{}) error { + // We verified the client's authorization and are running under its context. + // pb.AuthJournalClient will self-sign claims to proxy this journal on the client's behalf. var ctx = pb.WithDispatchRoute(stream.Context(), req.Header.Route, req.Header.ProcessId) // We use the |stream| context for this RPC, which means a cancellation from @@ -109,11 +109,14 @@ type proxyChunk struct { // serveRead evaluates a client's Read RPC against the local replica index. func serveRead(stream grpc.ServerStream, req *pb.ReadRequest, hdr *pb.Header, index *fragment.Index) error { - var buffer = make([]byte, chunkSize) - var reader io.ReadCloser + var ( + buffer = make([]byte, chunkSize) + ctx = stream.Context() + reader io.ReadCloser + ) for i := 0; true; i++ { - var resp, file, err = index.Query(stream.Context(), req) + var resp, file, err = index.Query(ctx, req) if err != nil { return err } @@ -144,7 +147,7 @@ func serveRead(stream grpc.ServerStream, req *pb.ReadRequest, hdr *pb.Header, in reader = io.NopCloser(io.NewSectionReader( file, req.Offset-resp.Fragment.Begin, resp.Fragment.End-req.Offset)) } else { - if reader, err = fragment.Open(stream.Context(), *resp.Fragment); err != nil { + if reader, err = fragment.Open(ctx, *resp.Fragment); err != nil { return err } else if reader, err = client.NewFragmentReader(reader, *resp.Fragment, req.Offset); err != nil { return err diff --git a/broker/read_api_test.go b/broker/read_api_test.go index 29832a23..b587922e 100644 --- a/broker/read_api_test.go +++ b/broker/read_api_test.go @@ -4,7 +4,6 @@ import ( "context" "errors" "io" - "io/ioutil" "os" "path/filepath" "testing" @@ -465,6 +464,30 @@ func TestReadRequestErrorCases(t *testing.T) { Header: broker.header("write/only"), }, resp) + setTestJournal(broker, pb.JournalSpec{Name: "valid/journal", Replication: 1}, broker.id) + + // Case: Insufficient claimed capability. + stream, err = broker.client().Read( + pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}), + &pb.ReadRequest{Journal: "valid/journal"}) + require.NoError(t, err) + _, err = stream.Recv() + require.ErrorContains(t, err, "authorization is missing required READ capability") + + // Case: Insufficient claimed selector. + stream, err = broker.client().Read( + pb.WithClaims(ctx, + pb.Claims{ + Capability: pb.Capability_READ, + Selector: pb.MustLabelSelector("name=something/else"), + }), + &pb.ReadRequest{Journal: "valid/journal"}) + require.NoError(t, err) + resp, err = stream.Recv() + require.NoError(t, err) + require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, resp.Status) // Journal not visible to these claims. + require.Len(t, resp.Header.Route.Endpoints, 0) + broker.cleanup() } @@ -472,7 +495,7 @@ func buildRemoteFragmentFixture(t require.TestingT) (frag pb.Fragment, dir strin const data = "XXXXXremote fragment data" var err error - dir, err = ioutil.TempDir("", "BrokerSuite") + dir, err = os.MkdirTemp("", "BrokerSuite") require.NoError(t, err) frag = pb.Fragment{ diff --git a/broker/replica.go b/broker/replica.go index 153fdcf1..cc61da30 100644 --- a/broker/replica.go +++ b/broker/replica.go @@ -109,11 +109,11 @@ func pulseDaemon(svc *Service, r *replica) { var invalidateCh <-chan struct{} for { select { - case _ = <-r.ctx.Done(): + case <-r.ctx.Done(): return - case _ = <-timer.C: + case <-timer.C: timer.Reset(healthCheckInterval) - case _ = <-invalidateCh: + case <-invalidateCh: invalidateCh = nil } @@ -121,6 +121,12 @@ func pulseDaemon(svc *Service, r *replica) { var fsm = appendFSM{ svc: svc, ctx: ctx, + claims: pb.Claims{ + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("name", r.journal.String()), + }, + Capability: pb.Capability_APPEND, + }, req: pb.AppendRequest{ Journal: r.journal, DoNotProxy: true, diff --git a/broker/replica_test.go b/broker/replica_test.go index ed5e9810..8854e077 100644 --- a/broker/replica_test.go +++ b/broker/replica_test.go @@ -25,7 +25,7 @@ func TestReplicaShutdown(t *testing.T) { broker.initialFragmentLoad() // Start the journal's replication pipeline. - var fsm = appendFSM{svc: broker.svc, ctx: ctx, req: pb.AppendRequest{Journal: "a/journal"}} + var fsm = newFSM(broker, ctx, pb.AppendRequest{Journal: "a/journal"}) require.True(t, fsm.runTo(stateAwaitDesiredReplicas)) fsm.returnPipeline() @@ -57,7 +57,7 @@ func TestReplicaAssignmentUpdateCases(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 2}, broker.id, pb.ProcessSpec_ID{Zone: "peer", Suffix: "broker"}) - var res, err = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + var res, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) require.NoError(t, err) // Case: assignments have been modified since resolution. @@ -74,8 +74,7 @@ func TestReplicaAssignmentUpdateCases(t *testing.T) { } // Case: assignments haven't been modified since resolution. - res, err = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "a/journal", minEtcdRevision: rev}) + res, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{minEtcdRevision: rev}) require.NoError(t, err) rev, err = updateAssignments(ctx, res.assignments, etcd) @@ -83,8 +82,7 @@ func TestReplicaAssignmentUpdateCases(t *testing.T) { // Expect that, after resolving at the returned Etcd revision, // Etcd assignment routes match the expectation. - res, err = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "a/journal", minEtcdRevision: rev}) + res, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{minEtcdRevision: rev}) require.NoError(t, err) for _, a := range res.assignments { @@ -94,9 +92,9 @@ func TestReplicaAssignmentUpdateCases(t *testing.T) { } // Case: Perform a second assignment update. Expect assignments are not modified. - rev, err = updateAssignments(ctx, res.assignments, etcd) + _, err = updateAssignments(ctx, res.assignments, etcd) require.NoError(t, err) - res2, err := broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + res2, err := broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) require.NoError(t, err) // Assignments were not modified, as they were already equivalent. diff --git a/broker/replicate_api.go b/broker/replicate_api.go index d04722a9..9260120b 100644 --- a/broker/replicate_api.go +++ b/broker/replicate_api.go @@ -12,8 +12,9 @@ import ( ) // Replicate dispatches the JournalServer.Replicate API. -func (svc *Service) Replicate(stream pb.Journal_ReplicateServer) (err error) { +func (svc *Service) Replicate(claims pb.Claims, stream pb.Journal_ReplicateServer) (err error) { var ( + ctx = stream.Context() req *pb.ReplicateRequest resolved *resolution ) @@ -22,7 +23,7 @@ func (svc *Service) Replicate(stream pb.Journal_ReplicateServer) (err error) { defer func() { if err != nil { var addr net.Addr - if p, ok := peer.FromContext(stream.Context()); ok { + if p, ok := peer.FromContext(ctx); ok { addr = p.Addr } log.WithFields(log.Fields{"err": err, "req": req, "client": addr}). @@ -40,9 +41,7 @@ func (svc *Service) Replicate(stream pb.Journal_ReplicateServer) (err error) { var spool fragment.Spool for done := false; !done; { - resolved, err = svc.resolver.resolve(resolveArgs{ - ctx: stream.Context(), - journal: req.Proposal.Journal, + resolved, err = svc.resolver.resolve(ctx, claims, req.Proposal.Journal, resolveOpts{ mayProxy: false, requirePrimary: false, proxyHeader: req.Header, @@ -59,12 +58,12 @@ func (svc *Service) Replicate(stream pb.Journal_ReplicateServer) (err error) { // Attempt to obtain exclusive ownership of the replica's Spool. select { case spool = <-resolved.replica.spoolCh: - addTrace(stream.Context(), "<-replica.spoolCh => %s", spool) + addTrace(ctx, "<-replica.spoolCh => %s", spool) done = true - case <-stream.Context().Done(): // Request was cancelled. - return stream.Context().Err() + case <-ctx.Done(): // Request was cancelled. + return ctx.Err() case <-resolved.invalidateCh: // Replica assignments changed. - addTrace(stream.Context(), " ... resolution was invalidated") + addTrace(ctx, " ... resolution was invalidated") // Loop to retry. } } diff --git a/broker/replicate_api_test.go b/broker/replicate_api_test.go index f3ef10f1..4cd779ae 100644 --- a/broker/replicate_api_test.go +++ b/broker/replicate_api_test.go @@ -14,6 +14,7 @@ import ( func TestReplicateStreamAndCommit(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_REPLICATE}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 2}, @@ -71,6 +72,7 @@ func TestReplicateStreamAndCommit(t *testing.T) { func TestReplicateRequestErrorCases(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_REPLICATE}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) @@ -177,12 +179,46 @@ func TestReplicateRequestErrorCases(t *testing.T) { _, err = stream.Recv() require.Regexp(t, `.* no ack requested but status != OK: status:PROPOSAL_MISMATCH .*`, err) + // Case: Insufficient claimed capability. + stream, _ = broker.client().Replicate( + pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_APPEND}), + ) + _, err = stream.Recv() + require.ErrorContains(t, err, "authorization is missing required REPLICATE capability") + + // Case: Insufficient claimed selector. + stream, _ = broker.client().Replicate( + pb.WithClaims(ctx, + pb.Claims{ + Capability: pb.Capability_REPLICATE, + Selector: pb.MustLabelSelector("name=something/else"), + }), + ) + require.NoError(t, stream.Send(&pb.ReplicateRequest{ + Header: broker.header("a/journal"), + Proposal: &pb.Fragment{ + Journal: "a/journal", + CompressionCodec: pb.CompressionCodec_NONE, + }, + Registers: boxLabels(), + Acknowledge: true, + })) + resp, err := stream.Recv() + require.NoError(t, err) + require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, resp.Status) // Journal not visible to these claims. + require.Len(t, resp.Header.Route.Endpoints, 0) + + // Expect broker closes. + _, err = stream.Recv() + require.Equal(t, io.EOF, err) + broker.cleanup() } func TestReplicateBlockingRestart(t *testing.T) { var ctx, etcd = pb.WithDispatchDefault(context.Background()), etcdtest.TestClient() defer etcdtest.Cleanup() + ctx = pb.WithClaims(ctx, pb.Claims{Capability: pb.Capability_REPLICATE}) var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 3}, diff --git a/broker/resolver.go b/broker/resolver.go index 04de287d..7a70d836 100644 --- a/broker/resolver.go +++ b/broker/resolver.go @@ -7,7 +7,7 @@ import ( "github.com/pkg/errors" log "github.com/sirupsen/logrus" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/allocator" pb "go.gazette.dev/core/broker/protocol" pbx "go.gazette.dev/core/broker/protocol/ext" @@ -46,17 +46,14 @@ func newResolver(state *allocator.State, newReplica func(pb.Journal) *replica) * return r } -type resolveArgs struct { - ctx context.Context - // Journal to be dispatched. - journal pb.Journal +type resolveOpts struct { // Whether we may proxy to another broker. mayProxy bool // Whether we require the primary broker of the journal. requirePrimary bool // Minimum Etcd Revision to have read through, before generating a resolution. minEtcdRevision int64 - // Optional Header attached to the request from a proxying peer. + // Optional Header attached to the request from a proxy-ing peer. proxyHeader *pb.Header } @@ -81,12 +78,12 @@ type resolution struct { invalidateCh <-chan struct{} } -func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { +func (r *resolver) resolve(ctx context.Context, claims pb.Claims, journal pb.Journal, opts resolveOpts) (res *resolution, err error) { var ks = r.state.KS res = new(resolution) // Discard metadata path segment, which doesn't alter resolution outcomes. - args.journal = args.journal.StripMeta() + journal = journal.StripMeta() ks.Mu.RLock() defer ks.Mu.RUnlock() @@ -102,7 +99,7 @@ func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { res.localID = pb.ProcessSpec_ID{Zone: "local-BrokerSpec", Suffix: "missing-from-Etcd"} } - if hdr := args.proxyHeader; hdr != nil { + if hdr := opts.proxyHeader; hdr != nil { // Sanity check the proxy broker is using our same Etcd cluster. if hdr.Etcd.ClusterId != ks.Header.ClusterId { err = fmt.Errorf("proxied request Etcd ClusterId doesn't match our own (%d vs %d)", @@ -116,39 +113,49 @@ func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { return } // We want to wait for the greater of a |proxyHeader| or |minEtcdRevision|. - if args.proxyHeader.Etcd.Revision > args.minEtcdRevision { - args.minEtcdRevision = args.proxyHeader.Etcd.Revision + if opts.proxyHeader.Etcd.Revision > opts.minEtcdRevision { + opts.minEtcdRevision = opts.proxyHeader.Etcd.Revision } } - if args.minEtcdRevision > ks.Header.Revision { - addTrace(args.ctx, " ... at revision %d, but want at least %d", - ks.Header.Revision, args.minEtcdRevision) + if opts.minEtcdRevision > ks.Header.Revision { + addTrace(ctx, " ... at revision %d, but want at least %d", + ks.Header.Revision, opts.minEtcdRevision) - if err = ks.WaitForRevision(args.ctx, args.minEtcdRevision); err != nil { + if err = ks.WaitForRevision(ctx, opts.minEtcdRevision); err != nil { return } - addTrace(args.ctx, "WaitForRevision(%d) => %d", - args.minEtcdRevision, ks.Header.Revision) + addTrace(ctx, "WaitForRevision(%d) => %d", + opts.minEtcdRevision, ks.Header.Revision) } res.Etcd = pbx.FromEtcdResponseHeader(ks.Header) + // Extract Assignments. + res.assignments = ks.KeyValues.Prefixed( + allocator.ItemAssignmentsPrefix(ks, journal.String())).Copy() + // Extract JournalSpec. - if item, ok := allocator.LookupItem(ks, args.journal.String()); ok { - res.journalSpec = item.ItemValue.(*pb.JournalSpec) + if item, ok := allocator.LookupItem(ks, journal.String()); ok { + var spec = item.ItemValue.(*pb.JournalSpec) + + // Is the caller authorized to the journal? + if claims.Selector.Matches(spec.LabelSetExt(pb.LabelSet{})) { + res.journalSpec = spec + } else { + // Clear to act as if the journal doesn't exist. + res.assignments = keyspace.KeyValues{} + } } - // Extract Assignments and build Route. - res.assignments = ks.KeyValues.Prefixed( - allocator.ItemAssignmentsPrefix(ks, args.journal.String())).Copy() + // Build Route from extracted assignments. pbx.Init(&res.Route, res.assignments) pbx.AttachEndpoints(&res.Route, ks) // Select a definite ProcessID if we require the primary and there is one, // or if we're a member of the Route (and authoritative). - if args.requirePrimary && res.Route.Primary != -1 { + if opts.requirePrimary && res.Route.Primary != -1 { res.ProcessId = res.Route.Members[res.Route.Primary] - } else if !args.requirePrimary { + } else if !opts.requirePrimary { for i := range res.Route.Members { if res.Route.Members[i] == res.localID { res.ProcessId = res.localID @@ -167,7 +174,7 @@ func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { // don't have updated route assignments. err = errResolverStopped return - } else if replica := r.replicas[args.journal]; replica != nil { + } else if replica := r.replicas[journal]; replica != nil { res.replica = replica.replica res.invalidateCh = replica.signalCh } @@ -175,12 +182,12 @@ func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { // Select a response Status code. if res.journalSpec == nil { res.status = pb.Status_JOURNAL_NOT_FOUND - } else if args.requirePrimary && res.Route.Primary == -1 { + } else if opts.requirePrimary && res.Route.Primary == -1 { res.status = pb.Status_NO_JOURNAL_PRIMARY_BROKER } else if len(res.Route.Members) == 0 { res.status = pb.Status_INSUFFICIENT_JOURNAL_BROKERS - } else if !args.mayProxy && res.ProcessId != res.localID { - if args.requirePrimary { + } else if !opts.mayProxy && res.ProcessId != res.localID { + if opts.requirePrimary { res.status = pb.Status_NOT_JOURNAL_PRIMARY_BROKER } else { res.status = pb.Status_NOT_JOURNAL_BROKER @@ -195,8 +202,8 @@ func (r *resolver) resolve(args resolveArgs) (res *resolution, err error) { res.ProcessId = res.localID } - addTrace(args.ctx, "resolve(%s) => %s, local: %t, header: %s", - args.journal, res.status, res.replica != nil, &res.Header) + addTrace(ctx, "resolve(%s) => %s, local: %t, header: %s", + journal, res.status, res.replica != nil, &res.Header) return } diff --git a/broker/resolver_test.go b/broker/resolver_test.go index f989f204..846e26d9 100644 --- a/broker/resolver_test.go +++ b/broker/resolver_test.go @@ -52,7 +52,7 @@ func TestResolveCases(t *testing.T) { require.Len(t, resolver.replicas, 3) // Case: simple resolution of local replica. - var r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "replica/journal;ignored/meta"}) + var r, _ = resolver.resolve(ctx, allClaims, "replica/journal;ignored/meta", resolveOpts{}) require.Equal(t, pb.Status_OK, r.status) // Expect the local replica is attached. require.Equal(t, resolver.replicas["replica/journal"].replica, r.replica) @@ -68,13 +68,13 @@ func TestResolveCases(t *testing.T) { require.Equal(t, broker.id, r.localID) // Case: primary is required, and we are primary. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "primary/journal", requirePrimary: true}) + r, _ = resolver.resolve(ctx, allClaims, "primary/journal", resolveOpts{requirePrimary: true}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, broker.id, r.Header.ProcessId) require.Equal(t, mkRoute(0, broker.id, peer.id), r.Header.Route) // Case: primary is required, we are not primary, and may not proxy. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "replica/journal", requirePrimary: true}) + r, _ = resolver.resolve(ctx, allClaims, "replica/journal", resolveOpts{requirePrimary: true}) require.Equal(t, pb.Status_NOT_JOURNAL_PRIMARY_BROKER, r.status) // As status != OK and we authored the resolution, ProcessId is still |broker|. require.Equal(t, broker.id, r.Header.ProcessId) @@ -84,8 +84,7 @@ func TestResolveCases(t *testing.T) { require.NotNil(t, r.replica) // Case: primary is required, and we may proxy. - r, _ = resolver.resolve( - resolveArgs{ctx: ctx, journal: "replica/journal", requirePrimary: true, mayProxy: true}) + r, _ = resolver.resolve(ctx, allClaims, "replica/journal", resolveOpts{requirePrimary: true, mayProxy: true}) require.Equal(t, pb.Status_OK, r.status) // The resolution is specifically to |peer|. require.Equal(t, peer.id, r.Header.ProcessId) @@ -94,22 +93,21 @@ func TestResolveCases(t *testing.T) { require.NotNil(t, r.replica) // Case: primary is required, we may proxy, but there is no primary. - r, _ = resolver.resolve( - resolveArgs{ctx: ctx, journal: "no/primary/journal", requirePrimary: true, mayProxy: true}) + r, _ = resolver.resolve(ctx, allClaims, "no/primary/journal", resolveOpts{requirePrimary: true, mayProxy: true}) require.Equal(t, pb.Status_NO_JOURNAL_PRIMARY_BROKER, r.status) require.Equal(t, broker.id, r.Header.ProcessId) // We authored the error. require.Equal(t, mkRoute(-1, broker.id, peer.id), r.Header.Route) require.NotNil(t, r.replica) // Case: we may not proxy, and are not a replica. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "peer/only/journal"}) + r, _ = resolver.resolve(ctx, allClaims, "peer/only/journal", resolveOpts{}) require.Equal(t, pb.Status_NOT_JOURNAL_BROKER, r.status) require.Equal(t, broker.id, r.Header.ProcessId) // We authored the error. require.Equal(t, mkRoute(0, peer.id), r.Header.Route) require.Nil(t, r.replica) // Case: we may proxy, and are not a replica. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "peer/only/journal", mayProxy: true}) + r, _ = resolver.resolve(ctx, allClaims, "peer/only/journal", resolveOpts{mayProxy: true}) require.Equal(t, pb.Status_OK, r.status) // ProcessId is left empty as we could proxy to any of multiple peers. require.Equal(t, pb.ProcessSpec_ID{}, r.Header.ProcessId) @@ -117,13 +115,13 @@ func TestResolveCases(t *testing.T) { require.Nil(t, r.replica) // Case: the journal has no brokers. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "no/brokers/journal", mayProxy: true}) + r, _ = resolver.resolve(ctx, allClaims, "no/brokers/journal", resolveOpts{mayProxy: true}) require.Equal(t, pb.Status_INSUFFICIENT_JOURNAL_BROKERS, r.status) require.Equal(t, broker.id, r.Header.ProcessId) require.Equal(t, mkRoute(-1), r.Header.Route) // Case: the journal doesn't exist. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "does/not/exist"}) + r, _ = resolver.resolve(ctx, allClaims, "does/not/exist", resolveOpts{}) require.Equal(t, pb.Status_JOURNAL_NOT_FOUND, r.status) require.Equal(t, broker.id, r.Header.ProcessId) require.Equal(t, mkRoute(-1), r.Header.Route) @@ -137,14 +135,14 @@ func TestResolveCases(t *testing.T) { broker.ks.Mu.RUnlock() // Subcase 1: We can still resolve for peer journals. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "peer/only/journal", mayProxy: true}) + r, _ = resolver.resolve(ctx, allClaims, "peer/only/journal", resolveOpts{mayProxy: true}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, pb.ProcessSpec_ID{}, r.Header.ProcessId) require.Equal(t, mkRoute(0, peer.id), r.Header.Route) require.Nil(t, r.replica) // Subcase 2: We use a placeholder ProcessId. - r, _ = resolver.resolve(resolveArgs{ctx: ctx, journal: "peer/only/journal"}) + r, _ = resolver.resolve(ctx, allClaims, "peer/only/journal", resolveOpts{}) require.Equal(t, pb.Status_NOT_JOURNAL_BROKER, r.status) require.Equal(t, pb.ProcessSpec_ID{Zone: "local-BrokerSpec", Suffix: "missing-from-Etcd"}, r.Header.ProcessId) require.Equal(t, mkRoute(0, peer.id), r.Header.Route) @@ -163,7 +161,7 @@ func TestResolverLocalReplicaStopping(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "peer/journal", Replication: 1}, peer.id) // Precondition: journal & replica resolve as per expectation. - var r, _ = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + var r, _ = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, broker.id, r.Header.ProcessId) require.NotNil(t, r.replica) @@ -177,11 +175,10 @@ func TestResolverLocalReplicaStopping(t *testing.T) { <-r.replica.ctx.Done() // Attempts to resolve a local journal fail. - var _, err = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + var _, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) require.Equal(t, errResolverStopped, err) // However we'll still return proxy resolutions to peers. - r, _ = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "peer/journal", requirePrimary: true, mayProxy: true}) + r, _ = broker.svc.resolver.resolve(ctx, allClaims, "peer/journal", resolveOpts{requirePrimary: true, mayProxy: true}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, peer.id, r.Header.ProcessId) @@ -190,11 +187,10 @@ func TestResolverLocalReplicaStopping(t *testing.T) { setTestJournal(broker, pb.JournalSpec{Name: "new/peer/journal", Replication: 1}, peer.id) // An attempt for this new local journal still fails. - _, err = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal"}) + _, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{}) require.Equal(t, errResolverStopped, err) // But we successfully resolve to a peer. - r, _ = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "peer/journal", requirePrimary: true, mayProxy: true}) + r, _ = broker.svc.resolver.resolve(ctx, allClaims, "peer/journal", resolveOpts{requirePrimary: true, mayProxy: true}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, peer.id, r.Header.ProcessId) @@ -228,7 +224,7 @@ func TestResolveFutureRevisionCasesWithProxyHeader(t *testing.T) { }) // Expect the resolution succeeds, despite the journal not yet existing. - var r, _ = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "journal/one", proxyHeader: &hdr}) + var r, _ = broker.svc.resolver.resolve(ctx, allClaims, "journal/one", resolveOpts{proxyHeader: &hdr}) require.Equal(t, pb.Status_OK, r.status) require.Equal(t, hdr, r.Header) @@ -238,16 +234,14 @@ func TestResolveFutureRevisionCasesWithProxyHeader(t *testing.T) { time.AfterFunc(time.Millisecond, func() { setTestJournal(broker, pb.JournalSpec{Name: "journal/two", Replication: 1}, broker.id) }) - r, _ = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "journal/two", minEtcdRevision: futureRevision}) + r, _ = broker.svc.resolver.resolve(ctx, allClaims, "journal/two", resolveOpts{minEtcdRevision: futureRevision}) require.Equal(t, pb.Status_OK, r.status) // Case: finally, specify a future revision which doesn't come about and cancel the context. ctx, cancel := context.WithCancel(ctx) time.AfterFunc(time.Millisecond, cancel) - var _, err = broker.svc.resolver.resolve( - resolveArgs{ctx: ctx, journal: "journal/three", minEtcdRevision: futureRevision + 1e10}) + var _, err = broker.svc.resolver.resolve(ctx, allClaims, "journal/three", resolveOpts{minEtcdRevision: futureRevision + 1e10}) require.Equal(t, context.Canceled, err) broker.cleanup() @@ -266,13 +260,13 @@ func TestResolveProxyHeaderErrorCases(t *testing.T) { } // Case: proxy header references a broker other than this one. - var _, err = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal", proxyHeader: &proxy}) + var _, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{proxyHeader: &proxy}) require.Regexp(t, `proxied request ProcessId doesn't match our own \(zone.*`, err) proxy.ProcessId = broker.id // Case: proxy header references a ClusterId other than our own. proxy.Etcd.ClusterId = 8675309 - _, err = broker.svc.resolver.resolve(resolveArgs{ctx: ctx, journal: "a/journal", proxyHeader: &proxy}) + _, err = broker.svc.resolver.resolve(ctx, allClaims, "a/journal", resolveOpts{proxyHeader: &proxy}) require.Regexp(t, `proxied request Etcd ClusterId doesn't match our own \(\d+.*`, err) broker.cleanup() diff --git a/broker/service.go b/broker/service.go index 6ae5da35..665b1f51 100644 --- a/broker/service.go +++ b/broker/service.go @@ -3,7 +3,7 @@ package broker import ( "context" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/allocator" pb "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/server" @@ -93,33 +93,6 @@ func (svc *Service) QueueTasks(tasks *task.Group, server *server.Server, finishF }) } -// Route an item using the Service resolver. Route implements the -// protocol.DispatchRouter interface, and enables usages of -// protocol.WithDispatchItemRoute (eg, `client` & `http_gateway` packages) to -// resolve items via the Service resolver. -func (svc *Service) Route(ctx context.Context, item string) pb.Route { - var res, err = svc.resolver.resolve(resolveArgs{ - ctx: ctx, - journal: pb.Journal(item), - mayProxy: true, - }) - if err == errResolverStopped { - return pb.Route{Primary: -1} // We're shutting down. - } else if err != nil { - // Otherwise cannot err because we use neither minEtcdRevision nor proxyHeader. - panic(err) - } - // If Status != OK, Route will be zero-valued, which directs dispatcher - // to use the default service address (localhost), which will then re-run - // resolution and generate a proper error message for the client. - return res.Route -} - -// UpdateRoute is a no-op implementation of protocol.DispatchRouter. -func (svc *Service) UpdateRoute(string, *pb.Route) {} // No-op. -// IsNoopRouter returns false. -func (svc *Service) IsNoopRouter() bool { return false } - func addTrace(ctx context.Context, format string, args ...interface{}) { if tr, ok := trace.FromContext(ctx); ok { tr.LazyPrintf(format, args...) diff --git a/broker/test_support_test.go b/broker/test_support_test.go index 69085f70..a67dfb5a 100644 --- a/broker/test_support_test.go +++ b/broker/test_support_test.go @@ -5,8 +5,9 @@ import ( "time" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/allocator" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker/fragment" pb "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/broker/teststub" @@ -21,11 +22,11 @@ import ( // * Presenting the broker Service over a gRPC loopback server. // // A few bits are deliberately left out: -// * It doesn't run an allocator or obtain a lease. It simply reacts to manual -// KeySpace changes as they are made. -// * pulseDeamon() and fragmentRefreshDaemon() loops are not started with -// each assigned replica. Unit tests should perform (or test) these functions -// as needed. +// - It doesn't run an allocator or obtain a lease. It simply reacts to manual +// KeySpace changes as they are made. +// - pulseDeamon() and fragmentRefreshDaemon() loops are not started with +// each assigned replica. Unit tests should perform (or test) these functions +// as needed. type testBroker struct { t require.TestingT id pb.ProcessSpec_ID @@ -33,6 +34,7 @@ type testBroker struct { ks *keyspace.KeySpace svc *Service srv *server.Server + auth pb.Authorizer } // mockBroker pairs a teststub.Broker with that broker's announcement to Etcd. @@ -44,11 +46,15 @@ type mockBroker struct { // newTestBroker returns a local testBroker of |id|. |newReplicaFn| should be // either |newReadyReplica| or |newReplica|. func newTestBroker(t require.TestingT, etcd *clientv3.Client, id pb.ProcessSpec_ID) *testBroker { + var auth, err = auth.NewKeyedAuth("c3VwZXIgc2VjcmV0") + require.NoError(t, err) + var bk = &testBroker{ t: t, id: id, tasks: task.NewGroup(context.Background()), ks: NewKeySpace("/broker.test"), + auth: auth, } var state = allocator.NewObservedState(bk.ks, allocator.MemberKey(bk.ks, bk.id.Zone, bk.id.Suffix), JournalIsConsistent) @@ -56,7 +62,7 @@ func newTestBroker(t require.TestingT, etcd *clientv3.Client, id pb.ProcessSpec_ // Initialize server. bk.srv = server.MustLoopback() bk.svc = &Service{ - jc: pb.NewJournalClient(bk.srv.GRPCLoopback), + jc: pb.NewAuthJournalClient(pb.NewJournalClient(bk.srv.GRPCLoopback), auth), etcd: etcd, resolver: newResolver(state, newReplica), stopProxyReadsCh: make(chan struct{}), @@ -73,7 +79,7 @@ func newTestBroker(t require.TestingT, etcd *clientv3.Client, id pb.ProcessSpec_ // Set, but don't start a Persister for the test. SetSharedPersister(fragment.NewPersister(bk.ks)) - pb.RegisterJournalServer(bk.srv.GRPCServer, bk.svc) + pb.RegisterJournalServer(bk.srv.GRPCServer, pb.NewVerifiedJournalServer(bk.svc, auth)) bk.srv.QueueTasks(bk.tasks) bk.svc.QueueTasks(bk.tasks, bk.srv, nil) @@ -102,9 +108,7 @@ func (bk *testBroker) cleanup() { // resolve returns the resolution of |journal| against the testBroker. func (bk *testBroker) resolve(journal pb.Journal) *resolution { - var res, err = bk.svc.resolver.resolve(resolveArgs{ - ctx: context.Background(), - journal: journal, + var res, err = bk.svc.resolver.resolve(context.Background(), allClaims, journal, resolveOpts{ mayProxy: true, }) require.NoError(bk.t, err) diff --git a/brokertest/broker.go b/brokertest/broker.go index 874fbbe7..2868a540 100644 --- a/brokertest/broker.go +++ b/brokertest/broker.go @@ -3,14 +3,17 @@ package brokertest import ( "context" + "crypto/rand" + "encoding/base64" "net/http" "os" "syscall" "time" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/allocator" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker" "go.gazette.dev/core/broker/client" "go.gazette.dev/core/broker/fragment" @@ -25,9 +28,10 @@ import ( // Broker is a lightweight, embedded Gazette broker suitable for testing client // functionality which depends on the availability of the Gazette service. type Broker struct { - ID pb.ProcessSpec_ID - Tasks *task.Group - Server *server.Server + ID pb.ProcessSpec_ID + Authorizer pb.Authorizer + Tasks *task.Group + Server *server.Server etcd *clientv3.Client sigCh chan os.Signal @@ -36,6 +40,17 @@ type Broker struct { // NewBroker builds and returns an in-process Broker identified by |zone| and |suffix|. func NewBroker(t require.TestingT, etcd *clientv3.Client, zone, suffix string) *Broker { + var key = make([]uint8, 12) + var _, err = rand.Read(key) + require.NoError(t, err) + return NewBrokerWithKeys(t, etcd, zone, suffix, base64.StdEncoding.EncodeToString(key)) +} + +// NewBroker builds and returns an in-process Broker identified by |zone| and |suffix|. +func NewBrokerWithKeys(t require.TestingT, etcd *clientv3.Client, zone, suffix, encodedKeys string) *Broker { + var auth, err = auth.NewKeyedAuth(encodedKeys) + require.NoError(t, err) + var ( id = pb.ProcessSpec_ID{Zone: zone, Suffix: suffix} ks = broker.NewKeySpace("/broker.test") @@ -43,9 +58,8 @@ func NewBroker(t require.TestingT, etcd *clientv3.Client, zone, suffix string) * allocator.MemberKey(ks, id.Zone, id.Suffix), broker.JournalIsConsistent) srv = server.MustLoopback() - lo = pb.NewJournalClient(srv.GRPCLoopback) + lo = pb.NewAuthJournalClient(pb.NewJournalClient(srv.GRPCLoopback), auth) service = broker.NewService(state, lo, etcd) - rjc = pb.NewRoutedJournalClient(lo, service) tasks = task.NewGroup(context.Background()) sigCh = make(chan os.Signal, 1) allocArgs = allocator.SessionArgs{ @@ -62,10 +76,10 @@ func NewBroker(t require.TestingT, etcd *clientv3.Client, zone, suffix string) * ) require.NoError(t, allocator.StartSession(allocArgs)) - pb.RegisterJournalServer(srv.GRPCServer, service) + pb.RegisterJournalServer(srv.GRPCServer, pb.NewVerifiedJournalServer(service, auth)) srv.HTTPMux = http.NewServeMux() - srv.HTTPMux.Handle("/", http_gateway.NewGateway(rjc)) + srv.HTTPMux.Handle("/", http_gateway.NewGateway(pb.NewRoutedJournalClient(lo, pb.NoopDispatchRouter{}))) // Set, but don't start a Persister for the test. broker.SetSharedPersister(fragment.NewPersister(ks)) @@ -76,18 +90,21 @@ func NewBroker(t require.TestingT, etcd *clientv3.Client, zone, suffix string) * tasks.GoRun() return &Broker{ - ID: id, - Tasks: tasks, - Server: srv, - etcd: etcd, - sigCh: sigCh, - state: state, + ID: id, + Authorizer: auth, + Tasks: tasks, + Server: srv, + etcd: etcd, + sigCh: sigCh, + state: state, } } // Client returns a RoutedJournalClient wrapping the GRPCLoopback. func (b *Broker) Client() pb.RoutedJournalClient { - return pb.NewRoutedJournalClient(pb.NewJournalClient(b.Server.GRPCLoopback), pb.NoopDispatchRouter{}) + var jc = pb.NewJournalClient(b.Server.GRPCLoopback) + jc = pb.NewAuthJournalClient(jc, b.Authorizer) + return pb.NewRoutedJournalClient(jc, pb.NoopDispatchRouter{}) } // Endpoint of the test Broker. diff --git a/brokertest/broker_test.go b/brokertest/broker_test.go index 82db9404..4d444205 100644 --- a/brokertest/broker_test.go +++ b/brokertest/broker_test.go @@ -62,8 +62,8 @@ func TestReplicatedReadAndWrite(t *testing.T) { defer etcdtest.Cleanup() var ctx = pb.WithDispatchDefault(context.Background()) - var bkA = NewBroker(t, etcd, "A", "broker-one") - var bkB = NewBroker(t, etcd, "B", "broker-two") + var bkA = NewBrokerWithKeys(t, etcd, "A", "broker-one", "b25l, dHdv") + var bkB = NewBrokerWithKeys(t, etcd, "B", "broker-two", "dHdv, b25l") CreateJournals(t, bkA, Journal(pb.JournalSpec{Name: "foo/bar", Replication: 2})) @@ -110,8 +110,8 @@ func TestReassignment(t *testing.T) { defer etcdtest.Cleanup() var ctx = pb.WithDispatchDefault(context.Background()) - var bkA = NewBroker(t, etcd, "zone", "broker-A") - var bkB = NewBroker(t, etcd, "zone", "broker-B") + var bkA = NewBrokerWithKeys(t, etcd, "zone", "broker-A", "c2VjcmV0") + var bkB = NewBrokerWithKeys(t, etcd, "zone", "broker-B", "c2VjcmV0") CreateJournals(t, bkA, Journal(pb.JournalSpec{Name: "foo/bar", Replication: 2})) @@ -127,7 +127,7 @@ func TestReassignment(t *testing.T) { }) // Broker C starts, and A signals for exit. - var bkC = NewBroker(t, etcd, "zone", "broker-C") + var bkC = NewBrokerWithKeys(t, etcd, "zone", "broker-C", "c2VjcmV0") bkA.Signal() require.NoError(t, bkA.Tasks.Wait()) // Exits gracefully. @@ -156,8 +156,8 @@ func TestGracefulStopTimeout(t *testing.T) { defer etcdtest.Cleanup() var ctx = pb.WithDispatchDefault(context.Background()) - var bkA = NewBroker(t, etcd, "zone", "broker-A") - var bkB = NewBroker(t, etcd, "zone", "broker-B") + var bkA = NewBrokerWithKeys(t, etcd, "zone", "broker-A", "c2VjcmV0") + var bkB = NewBrokerWithKeys(t, etcd, "zone", "broker-B", "c2VjcmV0") // Journal is owned by either |bkA| or |bkB| (they race). CreateJournals(t, bkA, Journal(pb.JournalSpec{Name: "foo/bar", Replication: 1})) @@ -180,8 +180,7 @@ func TestGracefulStopTimeout(t *testing.T) { // Begin a blocking read over |bkB|'s gRPC service. // This request is proxied to |bkA|. Then, don't make progress. - var r = client.NewReader(ctx, rjcB, - pb.ReadRequest{Journal: "foo/bar", Block: true}) + var r = client.NewReader(ctx, rjcB, pb.ReadRequest{Journal: "foo/bar", Block: true}) var _, err = r.Read(nil) require.NoError(t, err) @@ -282,7 +281,10 @@ func newDialedClient(t *testing.T, bk *Broker) (*grpc.ClientConn, pb.RoutedJourn grpc.WithTransportCredentials(pb.NewDispatchedCredentials(tlsConfig, bk.Endpoint())), ) require.NoError(t, err) - return conn, pb.NewRoutedJournalClient(pb.NewJournalClient(conn), pb.NoopDispatchRouter{}) + + var jc = pb.NewJournalClient(conn) + jc = pb.NewAuthJournalClient(jc, bk.Authorizer) + return conn, pb.NewRoutedJournalClient(jc, pb.NoopDispatchRouter{}) } func TestMain(m *testing.M) { etcdtest.TestMainWithEtcd(m) } diff --git a/cmd/gazctl/gazctlcmd/gazctl.go b/cmd/gazctl/gazctlcmd/gazctl.go index 26a64404..0b5fd204 100644 --- a/cmd/gazctl/gazctlcmd/gazctl.go +++ b/cmd/gazctl/gazctlcmd/gazctl.go @@ -2,7 +2,7 @@ package gazctlcmd import ( "errors" - "io/ioutil" + "io" "os" "go.gazette.dev/core/broker/protocol" @@ -76,9 +76,9 @@ func (cfg ApplyConfig) decode(into interface{}) error { var err error if cfg.SpecsPath == "-" { - buffer, err = ioutil.ReadAll(os.Stdin) + buffer, err = io.ReadAll(os.Stdin) } else { - buffer, err = ioutil.ReadFile(cfg.SpecsPath) + buffer, err = os.ReadFile(cfg.SpecsPath) } mbp.Must(err, "failed to read YAML input") diff --git a/cmd/gazette/main.go b/cmd/gazette/main.go index adf49abd..3e28000f 100644 --- a/cmd/gazette/main.go +++ b/cmd/gazette/main.go @@ -11,6 +11,7 @@ import ( "github.com/jessevdk/go-flags" log "github.com/sirupsen/logrus" "go.gazette.dev/core/allocator" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker" "go.gazette.dev/core/broker/fragment" "go.gazette.dev/core/broker/http_gateway" @@ -33,6 +34,7 @@ var Config = new(struct { MinAppendRate uint32 `long:"min-append-rate" env:"MIN_APPEND_RATE" default:"65536" description:"Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to sustain this rate are aborted"` DisableStores bool `long:"disable-stores" env:"DISABLE_STORES" description:"Disable use of any configured journal fragment stores. The broker will neither list or persist remote fragments, and all data is discarded on broker exit."` WatchDelay time.Duration `long:"watch-delay" env:"WATCH_DELAY" default:"30ms" description:"Delay applied to the application of watched Etcd events. Larger values amortize the processing of fast-changing Etcd keys."` + AuthKeys string `long:"auth-keys" env:"AUTH_KEYS" description:"Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all keys) Authorization tokens." json:"-"` } `group:"Broker" namespace:"broker" env-namespace:"BROKER"` Etcd struct { @@ -50,6 +52,18 @@ func (cmdServe) Execute(args []string) error { defer mbp.InitDiagnosticsAndRecover(Config.Diagnostics)() mbp.InitLog(Config.Log) + var authorizer pb.Authorizer + var verifier pb.Verifier + + if Config.Broker.AuthKeys != "" { + var a, err = auth.NewKeyedAuth(Config.Broker.AuthKeys) + mbp.Must(err, "parsing authorization keys") + authorizer, verifier = a, a + } else { + var a = auth.NewNoopAuth() + authorizer, verifier = a, a + } + log.WithFields(log.Fields{ "config": Config, "version": mbp.Version, @@ -87,7 +101,7 @@ func (cmdServe) Execute(args []string) error { fragment.DisableStores = Config.Broker.DisableStores var ( - lo = pb.NewJournalClient(srv.GRPCLoopback) + lo = pb.NewAuthJournalClient(pb.NewJournalClient(srv.GRPCLoopback), authorizer) etcd = Config.Etcd.MustDial() spec = &pb.BrokerSpec{ JournalLimit: Config.Broker.Limit, @@ -98,12 +112,11 @@ func (cmdServe) Execute(args []string) error { allocator.MemberKey(ks, spec.Id.Zone, spec.Id.Suffix), broker.JournalIsConsistent) service = broker.NewService(allocState, lo, etcd) - rjc = pb.NewRoutedJournalClient(lo, service) tasks = task.NewGroup(context.Background()) signalCh = make(chan os.Signal, 1) ) - pb.RegisterJournalServer(srv.GRPCServer, service) - srv.HTTPMux.Handle("/", http_gateway.NewGateway(rjc)) + pb.RegisterJournalServer(srv.GRPCServer, pb.NewVerifiedJournalServer(service, verifier)) + srv.HTTPMux.Handle("/", http_gateway.NewGateway(pb.NewRoutedJournalClient(lo, pb.NoopDispatchRouter{}))) ks.WatchApplyDelay = Config.Broker.WatchDelay log.WithFields(log.Fields{ diff --git a/examples/stream-sum/stream_sum_test.go b/examples/stream-sum/stream_sum_test.go index bdc2c77d..9882a4c8 100644 --- a/examples/stream-sum/stream_sum_test.go +++ b/examples/stream-sum/stream_sum_test.go @@ -141,7 +141,7 @@ func (s *StreamSumSuite) TestEndToEnd(c *gc.C) { var testJournals, testShards = buildSpecFixtures(4) // Start a broker & create journal fixtures. - var broker = brokertest.NewBroker(c, etcd, "local", "broker") + var broker = brokertest.NewBrokerWithKeys(c, etcd, "local", "broker", "c2VjcmV0") var rjc = pb.NewRoutedJournalClient(broker.Client(), pb.NoopDispatchRouter{}) brokertest.CreateJournals(c, broker, testJournals...) @@ -158,6 +158,7 @@ func (s *StreamSumSuite) TestEndToEnd(c *gc.C) { var cfg ChunkerConfig cfg.Broker.Address = broker.Endpoint() + cfg.Broker.AuthKeys = "c2VjcmV0" cfg.Chunker.Streams = 10 cfg.Chunker.Chunks = 10 cfg.Chunker.Workers = 4 diff --git a/mainboilerplate/client.go b/mainboilerplate/client.go index 7c83d803..1d087633 100644 --- a/mainboilerplate/client.go +++ b/mainboilerplate/client.go @@ -7,6 +7,7 @@ import ( "time" grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" pc "go.gazette.dev/core/consumer/protocol" @@ -21,6 +22,7 @@ type AddressConfig struct { CertFile string `long:"cert-file" env:"CERT_FILE" default:"" description:"Path to the client TLS certificate"` CertKeyFile string `long:"cert-key-file" env:"CERT_KEY_FILE" default:"" description:"Path to the client TLS private key"` TrustedCAFile string `long:"trusted-ca-file" env:"TRUSTED_CA_FILE" default:"" description:"Path to the trusted CA for client verification of server certificates"` + AuthKeys string `long:"auth-keys" env:"AUTH_KEYS" description:"Whitespace or comma separated, base64-encoded keys. The first key is used to sign Authorization tokens." json:"-"` } // MustDial dials the server address using a protocol.Dispatcher balancer, and panics on error. @@ -56,7 +58,18 @@ func (c *AddressConfig) MustDial(ctx context.Context) *grpc.ClientConn { // MustJournalClient dials and returns a new JournalClient. func (c *AddressConfig) MustJournalClient(ctx context.Context) pb.JournalClient { - return pb.NewJournalClient(c.MustDial(ctx)) + var authorizer pb.Authorizer + var err error + + if c.AuthKeys != "" { + authorizer, err = auth.NewKeyedAuth(c.AuthKeys) + Must(err, "parsing authorization keys") + } else { + authorizer = auth.NewNoopAuth() + } + + var jc = pb.NewJournalClient(c.MustDial(ctx)) + return pb.NewAuthJournalClient(jc, authorizer) } // MustShardClient dials and returns a new ShardClient. From aaa6cd8eebd04ff94d6443955c70c56e63a44b01 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Tue, 25 Jun 2024 12:42:48 -0500 Subject: [PATCH 5/8] consumer: add support for fine-grain Authorizations This change closely parallels the corersponding change to the Gazette broker and uses much of the same infrastructure. Built-in Shard APIs now expect and verify Authorizations which carry a set of capabilities and a label selector which scopes the resources (shards) which are authorized to the caller. Shards which don't match the claim's selector are not visible to the client and are indistinguishable from not existing at all. AuthShardClient drives an Authorizer to obtain and attach a suitable credential prior to starting an RPC. AuthShardServer uses a Verifier to verify a caller's claims prior to dispatching a service handler. Custom and application-specific APIs will likely want to use the Service.Authorizer and Service.Verifier fields to power their own authorizing client and server middleware. --- consumer/protocol/auth.go | 178 ++++++++++++++++++++ consumer/recovery.go | 6 +- consumer/resolver.go | 49 +++--- consumer/service.go | 46 +++-- consumer/shard_api.go | 51 ++++-- consumer/shard_api_test.go | 121 +++++++++---- consumer/test_support_test.go | 7 +- consumertest/consumer.go | 46 ++--- consumertest/consumer_test.go | 7 +- examples/stream-sum/stream_sum.go | 5 +- examples/word-count/word_count_test.go | 2 +- mainboilerplate/client.go | 27 ++- mainboilerplate/runconsumer/run_consumer.go | 18 +- 13 files changed, 446 insertions(+), 117 deletions(-) create mode 100644 consumer/protocol/auth.go diff --git a/consumer/protocol/auth.go b/consumer/protocol/auth.go new file mode 100644 index 00000000..d13fd56b --- /dev/null +++ b/consumer/protocol/auth.go @@ -0,0 +1,178 @@ +package protocol + +import ( + context "context" + time "time" + + pb "go.gazette.dev/core/broker/protocol" + grpc "google.golang.org/grpc" +) + +// NewAuthShardClient returns an *AuthShardClient which uses the Authorizer +// to obtain and attach an Authorization bearer token to every issued request. +func NewAuthShardClient(sc ShardClient, auth pb.Authorizer) *AuthShardClient { + return &AuthShardClient{Authorizer: auth, Inner: sc} +} + +type AuthShardClient struct { + Authorizer pb.Authorizer + Inner ShardClient +} + +func (a *AuthShardClient) Stat(ctx context.Context, in *StatRequest, opts ...grpc.CallOption) (*StatResponse, error) { + var claims, ok = pb.GetClaims(ctx) + if !ok { + claims = pb.Claims{ + Capability: pb.Capability_READ, + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("id", in.Shard.String()), + }, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.Stat(ctx, in, opts...) + } +} + +func (a *AuthShardClient) List(ctx context.Context, in *ListRequest, opts ...grpc.CallOption) (*ListResponse, error) { + var claims, ok = pb.GetClaims(ctx) + if !ok { + claims = pb.Claims{ + Capability: pb.Capability_LIST, + Selector: in.Selector, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.List(ctx, in, opts...) + } +} + +func (a *AuthShardClient) Apply(ctx context.Context, in *ApplyRequest, opts ...grpc.CallOption) (*ApplyResponse, error) { + var claims, ok = pb.GetClaims(ctx) + if !ok { + claims = pb.Claims{Capability: pb.Capability_APPLY} + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.Apply(ctx, in, opts...) + } +} + +func (a *AuthShardClient) GetHints(ctx context.Context, in *GetHintsRequest, opts ...grpc.CallOption) (*GetHintsResponse, error) { + var claims, ok = pb.GetClaims(ctx) + if !ok { + claims = pb.Claims{ + Capability: pb.Capability_READ, + Selector: pb.LabelSelector{ + Include: pb.MustLabelSet("id", in.Shard.String()), + }, + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.GetHints(ctx, in, opts...) + } +} + +func (a *AuthShardClient) Unassign(ctx context.Context, in *UnassignRequest, opts ...grpc.CallOption) (*UnassignResponse, error) { + var claims, ok = pb.GetClaims(ctx) + if !ok { + claims = pb.Claims{Capability: pb.Capability_APPLY} + for _, id := range in.Shards { + claims.Selector.Include.AddValue("id", id.String()) + } + } + if ctx, err := a.Authorizer.Authorize(ctx, claims, withExp(false)); err != nil { + return nil, err + } else { + return a.Inner.Unassign(ctx, in, opts...) + } +} + +func withExp(blocking bool) time.Duration { + if blocking { + return time.Hour + } else { + return time.Minute + } +} + +// AuthShardServer is similar to ShardServer except: +// - Requests have already been verified with accompanying Claims. +// - The Context or Stream.Context() argument may be subject to a deadline +// bound to the expiration of the user's Claims. +type AuthShardServer interface { + Stat(context.Context, pb.Claims, *StatRequest) (*StatResponse, error) + List(context.Context, pb.Claims, *ListRequest) (*ListResponse, error) + Apply(context.Context, pb.Claims, *ApplyRequest) (*ApplyResponse, error) + GetHints(context.Context, pb.Claims, *GetHintsRequest) (*GetHintsResponse, error) + Unassign(context.Context, pb.Claims, *UnassignRequest) (*UnassignResponse, error) +} + +// NewVerifiedShardServer adapts an AuthShardServer into a ShardServer by +// using the provided Verifier to verify incoming request Authorizations. +func NewVerifiedShardServer(ajs AuthShardServer, verifier pb.Verifier) *VerifiedAuthServer { + return &VerifiedAuthServer{ + Verifier: verifier, + Inner: ajs, + } +} + +type VerifiedAuthServer struct { + Verifier pb.Verifier + Inner AuthShardServer +} + +func (s *VerifiedAuthServer) Stat(ctx context.Context, req *StatRequest) (*StatResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, pb.Capability_READ); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.Stat(ctx, claims, req) + } +} + +func (s *VerifiedAuthServer) List(ctx context.Context, req *ListRequest) (*ListResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, pb.Capability_LIST); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.List(ctx, claims, req) + } +} + +func (s *VerifiedAuthServer) Apply(ctx context.Context, req *ApplyRequest) (*ApplyResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, pb.Capability_APPLY); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.Apply(ctx, claims, req) + } +} + +func (s *VerifiedAuthServer) GetHints(ctx context.Context, req *GetHintsRequest) (*GetHintsResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, pb.Capability_READ); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.GetHints(ctx, claims, req) + } +} + +func (s *VerifiedAuthServer) Unassign(ctx context.Context, req *UnassignRequest) (*UnassignResponse, error) { + if ctx, cancel, claims, err := s.Verifier.Verify(ctx, pb.Capability_APPLY); err != nil { + return nil, err + } else { + defer cancel() + return s.Inner.Unassign(ctx, claims, req) + } +} + +var _ ShardServer = &VerifiedAuthServer{} +var _ ShardClient = &AuthShardClient{} diff --git a/consumer/recovery.go b/consumer/recovery.go index 607b905b..a4aa3196 100644 --- a/consumer/recovery.go +++ b/consumer/recovery.go @@ -198,9 +198,9 @@ func beginRecovery(s *shard) error { var err error var spec = s.Spec() - s.recovery.hints, err = s.svc.GetHints(s.ctx, &pc.GetHintsRequest{ - Shard: spec.Id, - }) + s.recovery.hints, err = s.svc.GetHints(s.ctx, + pb.Claims{Capability: pb.Capability_READ}, + &pc.GetHintsRequest{Shard: spec.Id}) if err == nil && s.recovery.hints.Status != pc.Status_OK { err = fmt.Errorf(s.recovery.hints.Status.String()) diff --git a/consumer/resolver.go b/consumer/resolver.go index 58337802..089aa0cc 100644 --- a/consumer/resolver.go +++ b/consumer/resolver.go @@ -51,6 +51,8 @@ func NewResolver(state *allocator.State, newShard func(keyspace.KeyValue) *shard // responsible consumer process. type ResolveArgs struct { Context context.Context + // Authorized Claims under which we're resolving. + Claims pb.Claims // ShardID to be resolved. ShardID pc.ShardID // Whether we may resolve to another consumer peer. If false and this @@ -144,14 +146,24 @@ func (r *Resolver) Resolve(args ResolveArgs) (res Resolution, err error) { } res.Header.Etcd = pbx.FromEtcdResponseHeader(ks.Header) + // Extract Assignments. + var assignments = ks.KeyValues.Prefixed( + allocator.ItemAssignmentsPrefix(ks, args.ShardID.String())) + // Extract ShardSpec. if item, ok := allocator.LookupItem(ks, args.ShardID.String()); ok { - res.Spec = item.ItemValue.(*pc.ShardSpec) + var spec = item.ItemValue.(*pc.ShardSpec) + + // Is the caller authorized to the shard? + if args.Claims.Selector.Matches(spec.LabelSetExt(pb.LabelSet{})) { + res.Spec = spec + } else { + // Clear to act as if the shard doesn't exist. + assignments = keyspace.KeyValues{} + } } - // Extract Route. - var assignments = ks.KeyValues.Prefixed( - allocator.ItemAssignmentsPrefix(ks, args.ShardID.String())) + // Build Route from extracted assignments. pbx.Init(&res.Header.Route, assignments) pbx.AttachEndpoints(&res.Header.Route, ks) @@ -337,23 +349,22 @@ func (r *Resolver) updateLocalShards() { // application must make an appropriate selection from among the returned // ShardSpecs for its use case. // -// var mapping message.MappingFunc = ... -// var mappedID pc.ShardID -// -// if journal, _, err := mapping(key); err != nil { -// // Handle error. -// } else if specs := resolver.ShardsWithSource(journal); len(specs) == 0 { -// err = fmt.Errorf("no ShardSpec is consuming mapped journal %s", journal) -// // Handle error. -// } else { -// mappedID = specs[0].Id -// } +// var mapping message.MappingFunc = ... +// var mappedID pc.ShardID // -// var resolution, err = svc.Resolver.Resolve(consumer.ResolveArgs{ -// ShardID: specs[0].Id, -// ... -// }) +// if journal, _, err := mapping(key); err != nil { +// // Handle error. +// } else if specs := resolver.ShardsWithSource(journal); len(specs) == 0 { +// err = fmt.Errorf("no ShardSpec is consuming mapped journal %s", journal) +// // Handle error. +// } else { +// mappedID = specs[0].Id +// } // +// var resolution, err = svc.Resolver.Resolve(consumer.ResolveArgs{ +// ShardID: specs[0].Id, +// ... +// }) func (r *Resolver) ShardsWithSource(journal pb.Journal) []*pc.ShardSpec { r.state.KS.Mu.RLock() var specs = r.journals[journal] diff --git a/consumer/service.go b/consumer/service.go index 9d1d69e2..c671ff9a 100644 --- a/consumer/service.go +++ b/consumer/service.go @@ -23,6 +23,14 @@ import ( type Service struct { // Application served by the Service. App Application + // Authorizer of peer-to-peer requests. + // Consumer applications may want to use pc.NewAuthShardClient() to build + // clients with self-signed Authorizations using the Loopback connection. + Authorizer pb.Authorizer + // Verifier of requests. + // Custom consumer application APIs should secure themselves by verifying + // authorizations using this Verifier. See pc.NewVerifiedShardServer() as a model. + Verifier pb.Verifier // Resolver of Service shards. Resolver *Resolver // Distributed allocator state of the service. @@ -47,11 +55,11 @@ type Service struct { // ShardAPI holds function delegates which power the ShardServer API. // They're exposed to allow consumer applications to wrap or alter their behavior. ShardAPI struct { - Stat func(context.Context, *Service, *pc.StatRequest) (*pc.StatResponse, error) - List func(context.Context, *Service, *pc.ListRequest) (*pc.ListResponse, error) - Apply func(context.Context, *Service, *pc.ApplyRequest) (*pc.ApplyResponse, error) - GetHints func(context.Context, *Service, *pc.GetHintsRequest) (*pc.GetHintsResponse, error) - Unassign func(context.Context, *Service, *pc.UnassignRequest) (*pc.UnassignResponse, error) + Stat func(context.Context, pb.Claims, *Service, *pc.StatRequest) (*pc.StatResponse, error) + List func(context.Context, pb.Claims, *Service, *pc.ListRequest) (*pc.ListResponse, error) + Apply func(context.Context, pb.Claims, *Service, *pc.ApplyRequest) (*pc.ApplyResponse, error) + GetHints func(context.Context, pb.Claims, *Service, *pc.GetHintsRequest) (*pc.GetHintsResponse, error) + Unassign func(context.Context, pb.Claims, *Service, *pc.UnassignRequest) (*pc.UnassignResponse, error) } // stoppingCh is closed when the Service is in the process of shutting down. @@ -59,9 +67,11 @@ type Service struct { } // NewService constructs a new Service of the Application, driven by allocator.State. -func NewService(app Application, state *allocator.State, rjc pb.RoutedJournalClient, lo *grpc.ClientConn, etcd *clientv3.Client) *Service { +func NewService(app Application, authorizer pb.Authorizer, verifier pb.Verifier, state *allocator.State, rjc pb.RoutedJournalClient, lo *grpc.ClientConn, etcd *clientv3.Client) *Service { var svc = &Service{ App: app, + Authorizer: authorizer, + Verifier: verifier, State: state, Loopback: lo, Journals: rjc, @@ -137,29 +147,29 @@ func addTrace(ctx context.Context, format string, args ...interface{}) { } // Stat calls its ShardAPI delegate. -func (svc *Service) Stat(ctx context.Context, req *pc.StatRequest) (*pc.StatResponse, error) { - return svc.ShardAPI.Stat(ctx, svc, req) +func (svc *Service) Stat(ctx context.Context, claims pb.Claims, req *pc.StatRequest) (*pc.StatResponse, error) { + return svc.ShardAPI.Stat(ctx, claims, svc, req) } // List calls its ShardAPI delegate. -func (svc *Service) List(ctx context.Context, req *pc.ListRequest) (*pc.ListResponse, error) { - return svc.ShardAPI.List(ctx, svc, req) +func (svc *Service) List(ctx context.Context, claims pb.Claims, req *pc.ListRequest) (*pc.ListResponse, error) { + return svc.ShardAPI.List(ctx, claims, svc, req) } // Apply calls its ShardAPI delegate. -func (svc *Service) Apply(ctx context.Context, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { - return svc.ShardAPI.Apply(ctx, svc, req) +func (svc *Service) Apply(ctx context.Context, claims pb.Claims, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { + return svc.ShardAPI.Apply(ctx, claims, svc, req) } // GetHints calls its ShardAPI delegate. -func (svc *Service) GetHints(ctx context.Context, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { - return svc.ShardAPI.GetHints(ctx, svc, req) +func (svc *Service) GetHints(ctx context.Context, claims pb.Claims, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { + return svc.ShardAPI.GetHints(ctx, claims, svc, req) } // Unassign calls its ShardAPI delegate. -func (svc *Service) Unassign(ctx context.Context, req *pc.UnassignRequest) (*pc.UnassignResponse, error) { - return svc.ShardAPI.Unassign(ctx, svc, req) +func (svc *Service) Unassign(ctx context.Context, claims pb.Claims, req *pc.UnassignRequest) (*pc.UnassignResponse, error) { + return svc.ShardAPI.Unassign(ctx, claims, svc, req) } -// Service implements the ShardServer interface. -var _ pc.ShardServer = (*Service)(nil) +// Service implements the AuthShardServer interface. +var _ pc.AuthShardServer = (*Service)(nil) diff --git a/consumer/shard_api.go b/consumer/shard_api.go index 9f2834ca..57ae9b7e 100644 --- a/consumer/shard_api.go +++ b/consumer/shard_api.go @@ -15,14 +15,17 @@ import ( "go.gazette.dev/core/labels" "go.gazette.dev/core/message" "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) // ShardStat is the default implementation of the ShardServer.Stat API. -func ShardStat(ctx context.Context, srv *Service, req *pc.StatRequest) (*pc.StatResponse, error) { +func ShardStat(ctx context.Context, claims pb.Claims, srv *Service, req *pc.StatRequest) (*pc.StatResponse, error) { var ( resp = new(pc.StatResponse) res, err = srv.Resolver.Resolve(ResolveArgs{ Context: ctx, + Claims: claims, ShardID: req.Shard, MayProxy: req.Header == nil, // MayProxy if request hasn't already been proxied. ProxyHeader: req.Header, @@ -46,7 +49,7 @@ func ShardStat(ctx context.Context, srv *Service, req *pc.StatRequest) (*pc.Stat } // ShardList is the default implementation of the ShardServer.List API. -func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.ListResponse, error) { +func ShardList(ctx context.Context, claims pb.Claims, srv *Service, req *pc.ListRequest) (*pc.ListResponse, error) { var s = srv.Resolver.state if err := req.Validate(); err != nil { @@ -78,7 +81,7 @@ func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.List // LabelSetExt() truncates `scratch` while re-using its storage. scratch = shard.Spec.LabelSetExt(scratch) - if !req.Selector.Matches(scratch) { + if !req.Selector.Matches(scratch) || !claims.Selector.Matches(scratch) { continue } shard.ModRevision = s.Items[cur.Left].Raw.ModRevision @@ -97,30 +100,43 @@ func ShardList(ctx context.Context, srv *Service, req *pc.ListRequest) (*pc.List } // ShardApply is the default implementation of the ShardServer.Apply API. -func ShardApply(ctx context.Context, srv *Service, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { +func ShardApply(ctx context.Context, claims pb.Claims, srv *Service, req *pc.ApplyRequest) (*pc.ApplyResponse, error) { var s = srv.Resolver.state if err := req.Validate(); err != nil { return nil, err } + // The Apply API is authorized exclusively through the "id" label. + var authorizeShard = func(claims *pb.Claims, shard pc.ShardID) error { + if !claims.Selector.Matches(pb.MustLabelSet("id", shard.String())) { + return status.Error(codes.Unauthenticated, fmt.Sprintf("not authorized to %s", shard)) + } + return nil + } var cmp []clientv3.Cmp var ops []clientv3.Op - for _, changes := range req.Changes { + for _, change := range req.Changes { var key string - if changes.Upsert != nil { - key = allocator.ItemKey(s.KS, changes.Upsert.Id.String()) - ops = append(ops, clientv3.OpPut(key, changes.Upsert.MarshalString())) + if change.Upsert != nil { + if err := authorizeShard(&claims, change.Upsert.Id); err != nil { + return nil, err + } + key = allocator.ItemKey(s.KS, change.Upsert.Id.String()) + ops = append(ops, clientv3.OpPut(key, change.Upsert.MarshalString())) } else { - key = allocator.ItemKey(s.KS, changes.Delete.String()) + if err := authorizeShard(&claims, change.Delete); err != nil { + return nil, err + } + key = allocator.ItemKey(s.KS, change.Delete.String()) ops = append(ops, clientv3.OpDelete(key)) } // Allow caller to explicitly ignore revision comparison // by passing a value of -1 for revision. - if changes.ExpectModRevision != -1 { - cmp = append(cmp, clientv3.Compare(clientv3.ModRevision(key), "=", changes.ExpectModRevision)) + if change.ExpectModRevision != -1 { + cmp = append(cmp, clientv3.Compare(clientv3.ModRevision(key), "=", change.ExpectModRevision)) } } @@ -147,7 +163,7 @@ func ShardApply(ctx context.Context, srv *Service, req *pc.ApplyRequest) (*pc.Ap } // ShardGetHints is the default implementation of the ShardServer.Hints API. -func ShardGetHints(ctx context.Context, srv *Service, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { +func ShardGetHints(ctx context.Context, claims pb.Claims, srv *Service, req *pc.GetHintsRequest) (*pc.GetHintsResponse, error) { srv.State.KS.Mu.RLock() var resp = &pc.GetHintsResponse{ @@ -163,6 +179,11 @@ func ShardGetHints(ctx context.Context, srv *Service, req *pc.GetHintsRequest) ( } var spec = item.ItemValue.(*pc.ShardSpec) + if !claims.Selector.Matches(spec.LabelSetExt(pb.LabelSet{})) { + resp.Status = pc.Status_SHARD_NOT_FOUND + return resp, nil + } + var h, err = fetchHints(ctx, spec, srv.Etcd) if err != nil { return nil, err @@ -184,7 +205,7 @@ func ShardGetHints(ctx context.Context, srv *Service, req *pc.GetHintsRequest) ( return resp, nil } -func ShardUnassign(ctx context.Context, srv *Service, req *pc.UnassignRequest) (*pc.UnassignResponse, error) { +func ShardUnassign(ctx context.Context, claims pb.Claims, srv *Service, req *pc.UnassignRequest) (*pc.UnassignResponse, error) { var resp = &pc.UnassignResponse{ Status: pc.Status_OK, Shards: make([]pc.ShardID, 0), @@ -202,6 +223,10 @@ func ShardUnassign(ctx context.Context, srv *Service, req *pc.UnassignRequest) ( var ops []clientv3.Op for _, shard := range req.Shards { + // Like Apply, the Unassign API is authorized exclusively through the "id" label. + if !claims.Selector.Matches(pb.MustLabelSet("id", shard.String())) { + return nil, status.Error(codes.Unauthenticated, fmt.Sprintf("not authorized to %s", shard)) + } var assignments = state.Assignments.Prefixed(allocator.ItemAssignmentsPrefix(state.KS, shard.String())) for _, kv := range assignments { diff --git a/consumer/shard_api_test.go b/consumer/shard_api_test.go index ce29c9bf..e0300025 100644 --- a/consumer/shard_api_test.go +++ b/consumer/shard_api_test.go @@ -23,10 +23,11 @@ func TestAPIStatCases(t *testing.T) { <-aa.Done() // Case: Stat local shard. Expect it to have read-through our publish. - resp, err := tf.service.Stat(context.Background(), &pc.StatRequest{ - Shard: shardA, - ReadThrough: pb.Offsets{sourceB.Name: aa.Response().Commit.End}, - }) + resp, err := tf.service.Stat(context.Background(), allClaims, + &pc.StatRequest{ + Shard: shardA, + ReadThrough: pb.Offsets{sourceB.Name: aa.Response().Commit.End}, + }) require.NoError(t, err) require.Equal(t, pc.Status_OK, resp.Status) require.Equal(t, map[pb.Journal]int64{ @@ -36,10 +37,18 @@ func TestAPIStatCases(t *testing.T) { require.Equal(t, localID, resp.Header.ProcessId) // Case: Stat of non-existent Shard. - resp, err = tf.service.Stat(context.Background(), &pc.StatRequest{Shard: "missing-shard"}) + resp, err = tf.service.Stat(context.Background(), allClaims, + &pc.StatRequest{Shard: "missing-shard"}) require.NoError(t, err) require.Equal(t, pc.Status_SHARD_NOT_FOUND, resp.Status) + // Case: Insufficient claimed selector. + resp, err = tf.service.Stat(context.Background(), noClaims, + &pc.StatRequest{Shard: shardA}) + require.NoError(t, err) + require.Equal(t, pc.Status_SHARD_NOT_FOUND, resp.Status) // Shard not visible to these claims. + require.Len(t, resp.Header.Route.Endpoints, 0) + // TODO(johnny): Proxy case ought to be unit-tested here. // Adding it is currently low-value because it's covered by other E2E tests // and newTestFixture isn't set up for verifying proxy interactions. @@ -84,7 +93,7 @@ func TestAPIListCases(t *testing.T) { } // Case: Empty selector returns all shards. - var resp, err = tf.service.List(context.Background(), &pc.ListRequest{ + var resp, err = tf.service.List(context.Background(), allClaims, &pc.ListRequest{ Selector: pb.LabelSelector{}, }) require.NoError(t, err) @@ -94,21 +103,36 @@ func TestAPIListCases(t *testing.T) { require.Equal(t, pc.ReplicaStatus_PRIMARY, resp.Shards[2].Status[0].Code) // Case: Exclude on label. - resp, err = tf.service.List(context.Background(), &pc.ListRequest{ + resp, err = tf.service.List(context.Background(), allClaims, &pc.ListRequest{ Selector: pb.LabelSelector{Exclude: pb.MustLabelSet("foo", "")}, }) require.NoError(t, err) verify(resp, specB, specC) // Case: Meta-label "id" selects specific shards. - resp, err = tf.service.List(context.Background(), &pc.ListRequest{ + resp, err = tf.service.List(context.Background(), allClaims, &pc.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("id", shardC)}, }) require.NoError(t, err) verify(resp, specC) + // Case: Claims scope the visibile shards. + resp, err = tf.service.List(context.Background(), + pb.Claims{ + Capability: pb.Capability_LIST, + Selector: pb.MustLabelSelector("foo"), + }, + &pc.ListRequest{Selector: pb.LabelSelector{}}) + require.NoError(t, err) + verify(resp, specA) + + resp, err = tf.service.List(context.Background(), noClaims, + &pc.ListRequest{Selector: pb.LabelSelector{}}) + require.NoError(t, err) + verify(resp) + // Case: Errors on request validation error. - _, err = tf.service.List(context.Background(), &pc.ListRequest{ + _, err = tf.service.List(context.Background(), allClaims, &pc.ListRequest{ Selector: pb.LabelSelector{Include: pb.LabelSet{Labels: []pb.Label{{Name: "invalid label"}}}}, }) require.EqualError(t, err, `Selector.Include.Labels[0].Name: not a valid token (invalid label)`) @@ -125,7 +149,7 @@ func TestAPIApplyCases(t *testing.T) { var specB = makeShard(shardB) var verifyAndFetchRev = func(id pc.ShardID, expect pc.ShardSpec) int64 { - var resp, err = tf.service.List(context.Background(), &pc.ListRequest{ + var resp, err = tf.service.List(context.Background(), allClaims, &pc.ListRequest{ Selector: pb.LabelSelector{Include: pb.MustLabelSet("id", id.String())}, }) require.NoError(t, err) @@ -134,7 +158,7 @@ func TestAPIApplyCases(t *testing.T) { return resp.Shards[0].ModRevision } var apply = func(req *pc.ApplyRequest) *pc.ApplyResponse { - var resp, err = tf.service.Apply(context.Background(), req) + var resp, err = tf.service.Apply(context.Background(), allClaims, req) require.NoError(t, err) return resp } @@ -192,8 +216,20 @@ func TestAPIApplyCases(t *testing.T) { }, }).Status) + // Case: Insufficient claimed selector on delete. + var _, err = tf.service.Apply(context.Background(), noClaims, &pc.ApplyRequest{ + Changes: []pc.ApplyRequest_Change{{Delete: "shard-A", ExpectModRevision: -1}}, + }) + require.EqualError(t, err, `rpc error: code = Unauthenticated desc = not authorized to shard-A`) + + // Case: Insufficient claimed selector on upsert. + _, err = tf.service.Apply(context.Background(), noClaims, &pc.ApplyRequest{ + Changes: []pc.ApplyRequest_Change{{Upsert: specB}}, + }) + require.EqualError(t, err, `rpc error: code = Unauthenticated desc = not authorized to shard-B`) + // Case: Invalid requests fail with an error. - var _, err = tf.service.Apply(context.Background(), &pc.ApplyRequest{ + _, err = tf.service.Apply(context.Background(), allClaims, &pc.ApplyRequest{ Changes: []pc.ApplyRequest_Change{{Delete: "invalid shard id"}}, }) require.EqualError(t, err, `Changes[0].Delete: not a valid token (invalid shard id)`) @@ -265,7 +301,7 @@ func TestAPIHintsCases(t *testing.T) { tf.allocateShard(spec) // Case: No hints exist. - var resp, err = tf.service.GetHints(context.Background(), &pc.GetHintsRequest{Shard: shardA}) + var resp, err = tf.service.GetHints(context.Background(), allClaims, &pc.GetHintsRequest{Shard: shardA}) require.NoError(t, err) require.Equal(t, &pc.GetHintsResponse{ Status: pc.Status_OK, @@ -305,7 +341,7 @@ func TestAPIHintsCases(t *testing.T) { require.NoError(t, storeRecoveredHints(shard, mkHints(333))) // Case: Correctly fetch hints - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.NoError(t, err) require.Equal(t, &pc.GetHintsResponse{ Status: pc.Status_OK, @@ -317,7 +353,7 @@ func TestAPIHintsCases(t *testing.T) { // Case: No primary hints _, _ = tf.etcd.Delete(shard.ctx, spec.HintPrimaryKey()) - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.NoError(t, err) require.Equal(t, &pc.GetHintsResponse{ Status: pc.Status_OK, @@ -334,7 +370,7 @@ func TestAPIHintsCases(t *testing.T) { tf.resolver.shards[shardA].resolved.spec.HintBackups = tf.resolver.shards[shardA].resolved.spec.HintBackups + 1 tf.ks.Mu.RUnlock() } - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.NoError(t, err) require.Equal(t, &pc.GetHintsResponse{ Status: pc.Status_OK, @@ -349,7 +385,7 @@ func TestAPIHintsCases(t *testing.T) { tf.resolver.shards[shardA].resolved.spec.HintBackups = 0 tf.ks.Mu.RUnlock() } - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.NoError(t, err) require.Equal(t, &pc.GetHintsResponse{ Status: pc.Status_OK, @@ -358,7 +394,7 @@ func TestAPIHintsCases(t *testing.T) { }, resp) // Case: Fetch hints for a non-existent shard - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: "missing-shard"}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: "missing-shard"}) require.NoError(t, err) require.Equal(t, pc.Status_SHARD_NOT_FOUND, resp.Status) @@ -366,7 +402,7 @@ func TestAPIHintsCases(t *testing.T) { var hints = mkHints(444) hints.Log = "incorrect/log" require.NoError(t, storeRecordedHints(shard, hints)) - resp, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + resp, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.Nil(t, resp) require.EqualError(t, err, "hints.Log incorrect/log != ShardSpec.RecoveryLog recovery/logs/shard-A") @@ -374,9 +410,14 @@ func TestAPIHintsCases(t *testing.T) { hints = mkHints(555) hints.Log = "" require.NoError(t, storeRecordedHints(shard, hints)) - _, err = tf.service.GetHints(shard.ctx, &pc.GetHintsRequest{Shard: shardA}) + _, err = tf.service.GetHints(shard.ctx, allClaims, &pc.GetHintsRequest{Shard: shardA}) require.EqualError(t, err, "validating FSMHints: hinted log not provided") + // Case: Insufficient claimed selector. + resp, err = tf.service.GetHints(shard.ctx, noClaims, &pc.GetHintsRequest{Shard: shardA}) + require.NoError(t, err) + require.Equal(t, pc.Status_SHARD_NOT_FOUND, resp.Status) + tf.allocateShard(spec) // Cleanup. } @@ -451,9 +492,10 @@ func TestAPIUnassignCases(t *testing.T) { require.Equal(t, pc.Status_OK, resp.Status) require.Equal(t, affectedShards, resp.Shards) // Immediately query for remaining shard replicas. - listResp, err := tf.service.List(context.Background(), &pc.ListRequest{ - Selector: pb.LabelSelector{Include: pb.MustLabelSet("id", expectedSpec.Id.String())}, - }) + listResp, err := tf.service.List(context.Background(), allClaims, + &pc.ListRequest{ + Selector: pb.LabelSelector{Include: pb.MustLabelSet("id", expectedSpec.Id.String())}, + }) require.NoError(t, err) require.Equal(t, 1, len(listResp.Shards)) if len(expectedStatuses) > 0 { @@ -463,29 +505,34 @@ func TestAPIUnassignCases(t *testing.T) { } // Case: A shard with no assignments - resp, err := tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}}) + resp, err := tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}}) require.NoError(t, err) check(resp, specA, []pc.ShardID{}, []pc.ReplicaStatus{}) // Case: A shard with a single PRIMARY assignment - resp, err = tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specB.Id}}) + resp, err = tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specB.Id}}) require.NoError(t, err) check(resp, specB, []pc.ShardID{specB.Id}, []pc.ReplicaStatus{}) // Case: A shard with multiple assignments. PRIMARY is removed, FAILED is removed, STANDBY remains. - resp, err = tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specC.Id}}) + resp, err = tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specC.Id}}) require.NoError(t, err) check(resp, specC, []pc.ShardID{specC.Id}, []pc.ReplicaStatus{{Code: pc.ReplicaStatus_STANDBY}}) // Case: Only unassign failed shards tf.allocateShard(specA, localID) tf.setReplicaStatus(specA, localID, 0, pc.ReplicaStatus_PRIMARY) - resp, err = tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}, OnlyFailed: true}) + resp, err = tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}, OnlyFailed: true}) require.NoError(t, err) check(resp, specA, []pc.ShardID{}, []pc.ReplicaStatus{{Code: pc.ReplicaStatus_PRIMARY}}) tf.setReplicaStatus(specA, localID, 0, pc.ReplicaStatus_FAILED) - resp, err = tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}, OnlyFailed: true}) + resp, err = tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id}, OnlyFailed: true}) require.NoError(t, err) check(resp, specA, []pc.ShardID{specA.Id}, []pc.ReplicaStatus{}) @@ -496,14 +543,30 @@ func TestAPIUnassignCases(t *testing.T) { tf.setReplicaStatus(specB, localID, 0, pc.ReplicaStatus_FAILED) tf.allocateShard(specC, localID) tf.setReplicaStatus(specC, localID, 0, pc.ReplicaStatus_PRIMARY) - resp, err = tf.service.Unassign(context.Background(), &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id, specB.Id, specC.Id}, OnlyFailed: true}) + resp, err = tf.service.Unassign(context.Background(), allClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specA.Id, specB.Id, specC.Id}, OnlyFailed: true}) require.NoError(t, err) check(resp, specA, []pc.ShardID{specA.Id, specB.Id}, []pc.ReplicaStatus{}) check(resp, specB, []pc.ShardID{specA.Id, specB.Id}, []pc.ReplicaStatus{}) check(resp, specC, []pc.ShardID{specA.Id, specB.Id}, []pc.ReplicaStatus{{Code: pc.ReplicaStatus_PRIMARY}}) + // Case: Insufficient claimed capability + _, err = tf.service.Unassign(context.Background(), noClaims, + &pc.UnassignRequest{Shards: []pc.ShardID{specB.Id}}) + require.EqualError(t, err, `rpc error: code = Unauthenticated desc = not authorized to shard-B`) + // Cleanup. tf.allocateShard(specA) tf.allocateShard(specB) tf.allocateShard(specC) } + +var ( + allClaims = pb.Claims{ + Capability: pb.Capability_ALL, + } + noClaims = pb.Claims{ + Capability: pb.Capability_APPLY, + Selector: pb.MustLabelSelector("this-does=not-match"), + } +) diff --git a/consumer/test_support_test.go b/consumer/test_support_test.go index f81c04ad..d88c7e3f 100644 --- a/consumer/test_support_test.go +++ b/consumer/test_support_test.go @@ -4,7 +4,6 @@ import ( "context" "database/sql" "errors" - "io/ioutil" "os" "testing" "time" @@ -211,11 +210,11 @@ func newTestFixture(t require.TestingT) (*testFixture, func()) { var state = allocator.NewObservedState(ks, allocator.MemberKey(ks, localID.Zone, localID.Suffix), ShardIsConsistent) - var tmpSqlite, err = ioutil.TempFile("", "consumer-test") + var tmpSqlite, err = os.CreateTemp("", "consumer-test") require.NoError(t, err) var app = newTestApplication(t, tmpSqlite.Name()) - var svc = NewService(app, state, bk.Client(), nil, etcd) + var svc = NewService(app, nil, nil, state, bk.Client(), nil, etcd) var tasks = task.NewGroup(context.Background()) require.NoError(t, ks.Load(tasks.Context(), etcd, 0)) @@ -526,7 +525,7 @@ func runTransaction(tf *testFixture, s Shard, in map[string]string) { } // Block until ACKs have been read through, or an error occurred. - var _, err = ShardStat(tf.tasks.Context(), tf.service, &pc.StatRequest{ + var _, err = ShardStat(tf.tasks.Context(), pb.Claims{}, tf.service, &pc.StatRequest{ Shard: s.Spec().Id, ReadThrough: offsets, }) diff --git a/consumertest/consumer.go b/consumertest/consumer.go index f472b664..908d6555 100644 --- a/consumertest/consumer.go +++ b/consumertest/consumer.go @@ -8,8 +8,9 @@ import ( "time" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/client/v3" + clientv3 "go.etcd.io/etcd/client/v3" "go.gazette.dev/core/allocator" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" pbx "go.gazette.dev/core/broker/protocol/ext" @@ -17,7 +18,6 @@ import ( pc "go.gazette.dev/core/consumer/protocol" "go.gazette.dev/core/server" "go.gazette.dev/core/task" - "google.golang.org/grpc" ) // Consumer is a lightweight, embedded Gazette consumer runtime suitable for @@ -43,6 +43,7 @@ type Args struct { Root string // Consumer root in Etcd. Defaults to "/consumertest". Zone string // Zone of the consumer. Defaults to "local". Suffix string // ID Suffix of the consumer. Defaults to "consumer". + AuthKeys string // Authentication keys. Defaults to base64("secret") ("c2VjcmV0"). } // NewConsumer builds and returns a Consumer. @@ -56,13 +57,18 @@ func NewConsumer(args Args) *Consumer { if args.Suffix == "" { args.Suffix = "consumer" } + if args.AuthKeys == "" { + args.AuthKeys = "c2VjcmV0" + } + var auth, err = auth.NewKeyedAuth(args.AuthKeys) + require.NoError(args.C, err) var ( id = pb.ProcessSpec_ID{Zone: args.Zone, Suffix: args.Suffix} ks = consumer.NewKeySpace(args.Root) state = allocator.NewObservedState(ks, allocator.MemberKey(ks, id.Zone, id.Suffix), consumer.ShardIsConsistent) srv = server.MustLoopback() - svc = consumer.NewService(args.App, state, args.Journals, srv.GRPCLoopback, args.Etcd) + svc = consumer.NewService(args.App, auth, auth, state, args.Journals, srv.GRPCLoopback, args.Etcd) tasks = task.NewGroup(context.Background()) sigCh = make(chan os.Signal, 1) allocArgs = allocator.SessionArgs{ @@ -79,7 +85,7 @@ func NewConsumer(args Args) *Consumer { ) require.NoError(args.C, allocator.StartSession(allocArgs)) - pc.RegisterShardServer(srv.GRPCServer, svc) + pc.RegisterShardServer(srv.GRPCServer, pc.NewVerifiedShardServer(svc, auth)) ks.WatchApplyDelay = 0 // Speedup test execution. srv.QueueTasks(tasks) @@ -142,8 +148,11 @@ func CreateShards(t require.TestingT, cmr *Consumer, specs ...*pc.ShardSpec) { for _, spec := range specs { req.Changes = append(req.Changes, pc.ApplyRequest_Change{Upsert: spec}) } - var resp, err = consumer.ApplyShards(context.Background(), - pc.NewShardClient(cmr.Service.Loopback), req) + + var sc = pc.NewShardClient(cmr.Service.Loopback) + sc = pc.NewAuthShardClient(sc, cmr.Service.Authorizer) + + var resp, err = consumer.ApplyShards(context.Background(), sc, req) require.NoError(t, err) require.Equal(t, pc.Status_OK, resp.Status) @@ -156,14 +165,15 @@ func CreateShards(t require.TestingT, cmr *Consumer, specs ...*pc.ShardSpec) { // the current write-heads of journals being consumed by matched shards, and // polls shards until each has caught up to the determined write-heads of its // consumed journals. -func WaitForShards(ctx context.Context, rjc pb.RoutedJournalClient, conn *grpc.ClientConn, sel pb.LabelSelector) error { - var sc = pc.NewShardClient(conn) - ctx = pb.WithDispatchDefault(ctx) +func WaitForShards(t require.TestingT, cmr *Consumer, sel pb.LabelSelector) { + var sc = pc.NewShardClient(cmr.Service.Loopback) + sc = pc.NewAuthShardClient(sc, cmr.Service.Authorizer) + + var ctx = pb.WithDispatchDefault(context.Background()) var shards, err = consumer.ListShards(ctx, sc, &pc.ListRequest{Selector: sel}) - if err != nil { - return err - } + require.NoError(t, err) + // Collect the set of journals being read by shards. var expect = make(map[pb.Journal]int64) for _, shard := range shards.Shards { @@ -173,25 +183,23 @@ func WaitForShards(ctx context.Context, rjc pb.RoutedJournalClient, conn *grpc.C } // Determine the write-head of each journal. for journal := range expect { - var r = client.NewReader(ctx, rjc, pb.ReadRequest{ + var r = client.NewReader(ctx, cmr.Service.Journals, pb.ReadRequest{ Journal: journal, Offset: -1, Block: false, }) if _, err = r.Read(nil); err != client.ErrOffsetNotYetAvailable { - return err + require.NoError(t, err) } expect[journal] = r.Response.WriteHead } // Stat each shard, blocking until it reads through journal write-heads. for len(shards.Shards) != 0 { - if _, err = sc.Stat(ctx, &pc.StatRequest{ + _, err = sc.Stat(ctx, &pc.StatRequest{ Shard: shards.Shards[0].Spec.Id, ReadThrough: expect, - }); err != nil { - return err - } + }) + require.NoError(t, err) shards.Shards = shards.Shards[1:] } - return nil } diff --git a/consumertest/consumer_test.go b/consumertest/consumer_test.go index e221dd07..7fbc7029 100644 --- a/consumertest/consumer_test.go +++ b/consumertest/consumer_test.go @@ -70,8 +70,7 @@ func (s *ConsumerSuite) TestConsumeWithHandoff(c *gc.C) { c.Check(wc.Close(), gc.IsNil) // Wait for the consumer shard to catch up to written journal content. - c.Assert(WaitForShards(ctx, rjc, cmr1.Service.Loopback, - pb.LabelSelector{Include: pb.MustLabelSet("id", "a-shard")}), gc.IsNil) + WaitForShards(c, cmr1, pb.LabelSelector{Include: pb.MustLabelSet("id", "a-shard")}) // Expect the shard store reflects consumed messages. var res, err = cmr1.Service.Resolver.Resolve(consumer.ResolveArgs{Context: ctx, ShardID: "a-shard"}) @@ -102,7 +101,7 @@ func (s *ConsumerSuite) TestConsumeWithHandoff(c *gc.C) { c.Check(enc.Encode(testMsg{Key: "the", Value: "replaced value"}), gc.IsNil) c.Check(enc.Encode(testMsg{Key: "added", Value: "key"}), gc.IsNil) c.Check(wc.Close(), gc.IsNil) - c.Check(WaitForShards(ctx, rjc, cmr2.Service.Loopback, pb.LabelSelector{}), gc.IsNil) + WaitForShards(c, cmr2, pb.LabelSelector{}) // Expect the shard store reflects all consumed messages. res, err = cmr2.Service.Resolver.Resolve(consumer.ResolveArgs{Context: ctx, ShardID: "a-shard"}) @@ -179,7 +178,7 @@ func (s *ConsumerSuite) TestConsumeWithHotStandby(c *gc.C) { c.Check(enc.Encode(testMsg{Key: "brown", Value: "fox"}), gc.IsNil) c.Check(wc.Close(), gc.IsNil) - c.Check(WaitForShards(ctx, rjc, cmr1.Service.Loopback, pb.LabelSelector{}), gc.IsNil) + WaitForShards(c, cmr1, pb.LabelSelector{}) // Crash |cmr1|. cmr1.Tasks.Cancel() diff --git a/examples/stream-sum/stream_sum.go b/examples/stream-sum/stream_sum.go index 90dba479..41a86fba 100644 --- a/examples/stream-sum/stream_sum.go +++ b/examples/stream-sum/stream_sum.go @@ -110,8 +110,7 @@ func (s *Sum) Update(chunk Chunk) (done bool, err error) { // GenerateAndVerifyStreams is the main routine of the `chunker` job. It // generates and verifies streams based on the ChunkerConfig. func GenerateAndVerifyStreams(ctx context.Context, cfg *ChunkerConfig) error { - var conn = cfg.Broker.MustDial(ctx) - var rjc = pb.NewRoutedJournalClient(pb.NewJournalClient(conn), cfg.Broker.BuildRouter()) + var rjc = cfg.Broker.MustRoutedJournalClient(ctx) var as = client.NewAppendService(ctx, rjc) var chunksMapping, err = newChunkMapping(ctx, rjc) @@ -170,7 +169,7 @@ func GenerateAndVerifyStreams(ctx context.Context, cfg *ChunkerConfig) error { w++ // Worker finished. } } - return conn.Close() + return nil } // Summer consumes stream chunks, aggregates chunk data, and emits final sums. diff --git a/examples/word-count/word_count_test.go b/examples/word-count/word_count_test.go index fadfb847..a3c1b54f 100644 --- a/examples/word-count/word_count_test.go +++ b/examples/word-count/word_count_test.go @@ -62,7 +62,7 @@ func (s *WordCountSuite) TestPublishAndQuery(c *gc.C) { c.Assert(err, gc.IsNil) // Wait for consumer shards to catch up to written topic content. - c.Assert(consumertest.WaitForShards(ctx, rjc, cmr.Service.Loopback, pb.LabelSelector{}), gc.IsNil) + consumertest.WaitForShards(c, cmr, pb.LabelSelector{}) var queryCases = []struct { prefix NGram diff --git a/mainboilerplate/client.go b/mainboilerplate/client.go index 1d087633..40b85ec2 100644 --- a/mainboilerplate/client.go +++ b/mainboilerplate/client.go @@ -68,13 +68,36 @@ func (c *AddressConfig) MustJournalClient(ctx context.Context) pb.JournalClient authorizer = auth.NewNoopAuth() } - var jc = pb.NewJournalClient(c.MustDial(ctx)) + var conn = c.MustDial(ctx) + go func() { + <-ctx.Done() + _ = conn.Close() + }() + + var jc = pb.NewJournalClient(conn) return pb.NewAuthJournalClient(jc, authorizer) } // MustShardClient dials and returns a new ShardClient. func (c *AddressConfig) MustShardClient(ctx context.Context) pc.ShardClient { - return pc.NewShardClient(c.MustDial(ctx)) + var authorizer pb.Authorizer + var err error + + if c.AuthKeys != "" { + authorizer, err = auth.NewKeyedAuth(c.AuthKeys) + Must(err, "parsing authorization keys") + } else { + authorizer = auth.NewNoopAuth() + } + + var conn = c.MustDial(ctx) + go func() { + <-ctx.Done() + _ = conn.Close() + }() + + var sc = pc.NewShardClient(conn) + return pc.NewAuthShardClient(sc, authorizer) } // ClientConfig configures the client of a remote Gazette service. diff --git a/mainboilerplate/runconsumer/run_consumer.go b/mainboilerplate/runconsumer/run_consumer.go index e7d9e670..8c9fb4eb 100644 --- a/mainboilerplate/runconsumer/run_consumer.go +++ b/mainboilerplate/runconsumer/run_consumer.go @@ -17,6 +17,7 @@ import ( "github.com/prometheus/client_golang/prometheus" log "github.com/sirupsen/logrus" "go.gazette.dev/core/allocator" + "go.gazette.dev/core/auth" "go.gazette.dev/core/broker/client" pb "go.gazette.dev/core/broker/protocol" "go.gazette.dev/core/consumer" @@ -80,6 +81,7 @@ type BaseConfig struct { Limit uint32 `long:"limit" env:"LIMIT" default:"32" description:"Maximum number of Shards this consumer process will allocate"` MaxHotStandbys uint32 `long:"max-hot-standbys" env:"MAX_HOT_STANDBYS" default:"3" description:"Maximum effective hot standbys of any one shard, which upper-bounds its stated hot-standbys."` WatchDelay time.Duration `long:"watch-delay" env:"WATCH_DELAY" default:"30ms" description:"Delay applied to the application of watched Etcd events. Larger values amortize the processing of fast-changing Etcd keys."` + AuthKeys string `long:"auth-keys" env:"AUTH_KEYS" description:"Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all keys) Authorization tokens." json:"-"` } `group:"Consumer" namespace:"consumer" env-namespace:"CONSUMER"` Broker struct { @@ -114,6 +116,18 @@ func (sc Cmd) Execute(args []string) error { defer mbp.InitDiagnosticsAndRecover(bc.Diagnostics)() mbp.InitLog(bc.Log) + var authorizer pb.Authorizer + var verifier pb.Verifier + + if bc.Consumer.AuthKeys != "" { + var a, err = auth.NewKeyedAuth(bc.Consumer.AuthKeys) + mbp.Must(err, "parsing authorization keys") + authorizer, verifier = a, a + } else { + var a = auth.NewNoopAuth() + authorizer, verifier = a, a + } + log.WithFields(log.Fields{ "config": sc.Cfg, "version": mbp.Version, @@ -163,11 +177,11 @@ func (sc Cmd) Execute(args []string) error { ks = consumer.NewKeySpace(bc.Etcd.Prefix) state = allocator.NewObservedState(ks, allocator.MemberKey(ks, spec.Id.Zone, spec.Id.Suffix), consumer.ShardIsConsistent) rjc = bc.Broker.MustRoutedJournalClient(context.Background()) - service = consumer.NewService(sc.App, state, rjc, srv.GRPCLoopback, etcd) + service = consumer.NewService(sc.App, authorizer, verifier, state, rjc, srv.GRPCLoopback, etcd) tasks = task.NewGroup(context.Background()) signalCh = make(chan os.Signal, 1) ) - pc.RegisterShardServer(srv.GRPCServer, service) + pc.RegisterShardServer(srv.GRPCServer, pc.NewVerifiedShardServer(service, service.Verifier)) ks.WatchApplyDelay = bc.Consumer.WatchDelay // Register Resolver as a prometheus.Collector for tracking shard status From f6ad71339ced208eef4c90bfd86ab38928f96d1a Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 11 Jul 2024 13:04:11 -0500 Subject: [PATCH 6/8] message: remove the experimental and unused PendingPublish API --- message/publisher.go | 101 ++------------------------------- message/publisher_test.go | 115 -------------------------------------- 2 files changed, 5 insertions(+), 211 deletions(-) diff --git a/message/publisher.go b/message/publisher.go index e98f9f9f..9cc2bc02 100644 --- a/message/publisher.go +++ b/message/publisher.go @@ -71,11 +71,11 @@ func (p *Publisher) ProducerID() ProducerID { return p.producer } // its marshaled content, with a UUID sequenced for immediate consumption. // An error is returned if: // -// * The Message implements Validator, and it returns an error. -// * The MappingFunc returns an error while mapping the Message to a journal. -// * The journal's Framing returns an error while marshaling the Message, -// or an os.PathError occurs while spooling the frame to a temporary file -// (eg, because local disk is full). +// - The Message implements Validator, and it returns an error. +// - The MappingFunc returns an error while mapping the Message to a journal. +// - The journal's Framing returns an error while marshaling the Message, +// or an os.PathError occurs while spooling the frame to a temporary file +// (eg, because local disk is full). // // A particular MappingFunc error to be aware of is ErrEmptyListResponse, // returned by mapping routines of this package when there are no journals @@ -128,97 +128,6 @@ func (p *Publisher) PublishUncommitted(mapping MappingFunc, msg Message) (*clien return aa, nil } -// PendingPublish is returned from DeferPublishUncommitted, and allows appending a single message -// that had previously been sequenced. -// -// **This is a new and unstable API, that is subject to breaking changes.** -type PendingPublish struct { - publisher *Publisher - journal pb.Journal - contentType string - uuid UUID -} - -// Resolve completes a PendingPublish by appending the finalized content of a message that had -// previously been sequenced. See DeferPublishUncommitted docs for more. -// -// **This is a new and unstable API, that is subject to breaking changes.** -func (pf *PendingPublish) Resolve(msg Message) error { - if pf.publisher == nil { - // Sanity check for if Resolve has already been called, or if PendingPublish is zero-valued - // due to SequenceFutureMessage having returned an error. - panic("Pending publish has already been resolved") - } - if v, ok := msg.(Validator); ok { - if err := v.Validate(); err != nil { - return err - } - } - msg.SetUUID(pf.uuid) - - var framing, err = FramingByContentType(pf.contentType) - if err != nil { - return err - } - - var aa = pf.publisher.ajc.StartAppend(pb.AppendRequest{Journal: pf.journal}, nil) - aa.Require(framing.Marshal(msg, aa.Writer())) - err = aa.Release() - pf.publisher = nil // so that we can sanity check that Resolve isn't called twice - return err -} - -// DeferPublishUncommitted is used to sequence a message that will be published at some future -// point, but before the end of the transaction. It returns a PendingPublish, which can be resolved -// by passing it the actual message to be published. This is used in situations where you need to -// transactionally publish a message when you don't have the content of that message until after the -// ack intents are built. This is an advanced, low level api, and care must be taken to use it -// correctly to avoid corruption of journal content. -// -// The journal and contentType must be known up front, and the acknowledgement Message must also be -// provided by the caller. It's up to the caller to ensure that these things are correct and -// consistent. -// -// The returned PendingPublish does not need to ever be resolved, and can be dropped with no harm -// done. If Resolve is called, then it must be called _before_ the acknowledgements are written. -// Otherwise the resolved message will be ignored by ReadCommitted consumers. Also note that the -// PendingPublish is not safe to Resolve concurrently with other uses of a Publisher. -// -// No other messages should be published to the journal using PublishUncommitted or PublishCommitted -// before the PendingPublish is resolved. It it permissible to publish more than one message using -// DeferPublishUncommitted, as long as all PendingPublish instances are resolved in exactly the -// order in which they were created. -// -// **This is a new and unstable API, that is subject to breaking changes.** -func (p *Publisher) DeferPublishUncommitted(journal pb.Journal, contentType string, ack Message) (fut PendingPublish, err error) { - if p.autoUpdate { - p.clock.Update(time.Now()) - } - - var framing Framing - if framing, err = FramingByContentType(contentType); err != nil { - return - } - - var uuid = BuildUUID(p.producer, p.clock.Tick(), Flag_CONTINUE_TXN) - // Is this the first publish to this journal since our last commit? - if _, ok := p.intentIdx[journal]; !ok { - p.intentIdx[journal] = len(p.intents) - p.intents = append(p.intents, AckIntent{ - Journal: journal, - // Call NewAcknowledgement to create the ack, to ensure that each ack message is unique. - msg: ack.NewAcknowledgement(journal), - framing: framing, - }) - } - return PendingPublish{ - publisher: p, - journal: journal, - contentType: contentType, - uuid: uuid, - }, nil -} - // BuildAckIntents returns the []AckIntents which acknowledge all pending // Messages published since its last invocation. It's the caller's job to // actually append the intents to their respective journals, and only *after* diff --git a/message/publisher_test.go b/message/publisher_test.go index 5f518893..b708fb3a 100644 --- a/message/publisher_test.go +++ b/message/publisher_test.go @@ -215,121 +215,6 @@ func TestIntegrationOfPublisherWithSequencerAndReader(t *testing.T) { require.NoError(t, bk.Tasks.Wait()) } -func TestDeferPublishUncommitted(t *testing.T) { - var etcd = etcdtest.TestClient() - defer etcdtest.Cleanup() - - var ( - clock Clock - ctx = context.Background() - spec = newTestMsgSpec("a/journal") - bk = brokertest.NewBroker(t, etcd, "local", "broker") - ajc = client.NewAppendService(ctx, bk.Client()) - ) - brokertest.CreateJournals(t, bk, spec) - - // Start a long-lived RetryReader of |spec|. - var rr = client.NewRetryReader(ctx, bk.Client(), pb.ReadRequest{ - Journal: spec.Name, - Block: true, - }) - var r = NewReadUncommittedIter(rr, newTestMsg) - - var seq = NewSequencer(nil, nil, 5) - - var seqPump = func() (out []testMsg) { - var env, err = r.Next() - require.NoError(t, err) - - if seq.QueueUncommitted(env) == QueueAckCommitReplay { - // The sequencer buffer is large enough that we should never need to replay for this - // test. - panic("unexpected need to replay") - } - for { - if err := seq.Step(); err == io.EOF { - return - } - require.NoError(t, err) - out = append(out, *seq.Dequeued.Message.(*testMsg)) - } - } - - var mapping = func(Mappable) (pb.Journal, string, error) { - return spec.Name, labels.ContentType_JSONLines, nil - } - var pub = NewPublisher(ajc, &clock) - - // Happy path: An uncommitted message can be written before a deferred one, and should get - // sequenced normally with respect to the deferred message, since the deferred publish is - // started after. - var _, err = pub.PublishUncommitted(mapping, &testMsg{Str: "one"}) - require.NoError(t, err) - require.Equal(t, []testMsg(nil), seqPump()) - - fut, err := pub.DeferPublishUncommitted(spec.Name, labels.ContentType_JSONLines, new(testMsg)) - require.NoError(t, err) - - intents, err := pub.BuildAckIntents() - require.NoError(t, err) - - require.NoError(t, fut.Resolve(&testMsg{Str: "two"})) - require.Equal(t, []testMsg(nil), seqPump()) - - writeIntents(t, ajc, intents) - - var actual = seqPump() - require.Equal(t, 3, len(actual)) - require.Equal(t, "one", actual[0].Str) - require.Equal(t, "two", actual[1].Str) - require.Equal(t, "", actual[2].Str) - - // Sad path cases: - // The deferred publish message will not be seen because it sequences before "three" - fut, err = pub.DeferPublishUncommitted(spec.Name, labels.ContentType_JSONLines, new(testMsg)) - require.NoError(t, err) - - _, err = pub.PublishUncommitted(mapping, &testMsg{Str: "three"}) - require.NoError(t, err) - require.Equal(t, []testMsg(nil), seqPump()) - intents, err = pub.BuildAckIntents() - require.NoError(t, err) - require.NoError(t, fut.Resolve(&testMsg{Str: "wont see four"})) - require.Equal(t, []testMsg(nil), seqPump()) - - writeIntents(t, ajc, intents) - actual = seqPump() - require.Equal(t, 2, len(actual)) - require.Equal(t, "three", actual[0].Str) - require.Equal(t, "", actual[1].Str) - - // The deferred publish isn't resolved until after the acks were written, so will not be seen. - _, err = pub.PublishUncommitted(mapping, &testMsg{Str: "five"}) - require.NoError(t, err) - require.Equal(t, []testMsg(nil), seqPump()) - - fut, err = pub.DeferPublishUncommitted(spec.Name, labels.ContentType_JSONLines, new(testMsg)) - require.NoError(t, err) - - intents, err = pub.BuildAckIntents() - require.NoError(t, err) - writeIntents(t, ajc, intents) - - actual = seqPump() - require.Equal(t, 2, len(actual)) - require.Equal(t, "five", actual[0].Str) - require.Equal(t, "", actual[1].Str) - - require.NoError(t, fut.Resolve(&testMsg{Str: "wont see six"})) - require.Equal(t, []testMsg(nil), seqPump()) - - _, err = pub.PublishCommitted(mapping, &testMsg{Str: "seven"}) - require.NoError(t, err) - actual = seqPump() - require.Equal(t, 1, len(actual)) - require.Equal(t, "seven", actual[0].Str) -} - func readAllMsgs(t require.TestingT, bk *brokertest.Broker, spec *pb.JournalSpec) (out []testMsg) { var rr = client.NewRetryReader(context.Background(), bk.Client(), pb.ReadRequest{Journal: spec.Name}) var r = NewReadUncommittedIter(rr, newTestMsg) From 3d7972e7ac799e5bbc535bc8bfd1ddcfc3579a54 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 6 Jun 2024 15:57:21 -0500 Subject: [PATCH 7/8] mk: bump Go and builder versions --- .github/workflows/ci-workflow.yaml | 2 +- mk/ci-builder.Dockerfile | 6 +++--- mk/ci-release.Dockerfile | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/ci-workflow.yaml b/.github/workflows/ci-workflow.yaml index 3442df82..4af864c9 100644 --- a/.github/workflows/ci-workflow.yaml +++ b/.github/workflows/ci-workflow.yaml @@ -27,7 +27,7 @@ env: jobs: build: name: "Build" - runs-on: ubuntu-20.04 + runs-on: ubuntu-22.04 steps: - name: "Checkout" uses: actions/checkout@v2 diff --git a/mk/ci-builder.Dockerfile b/mk/ci-builder.Dockerfile index 26b678d6..e341a266 100644 --- a/mk/ci-builder.Dockerfile +++ b/mk/ci-builder.Dockerfile @@ -2,7 +2,7 @@ # this file. This works only so long as there are no _other_ files that go into the final image. # So if you add any ADD or COPY directives, be sure to update the cache key in the github actions # workflow yaml -FROM ubuntu:20.04 +FROM ubuntu:22.04 RUN apt-get update -y \ && apt-get upgrade -y \ @@ -23,8 +23,8 @@ RUN apt-get update -y \ zip \ && rm -rf /var/lib/apt/lists/* -ARG GOLANG_VERSION=1.19.13 -ARG GOLANG_SHA256=4643d4c29c55f53fa0349367d7f1bb5ca554ea6ef528c146825b0f8464e2e668 +ARG GOLANG_VERSION=1.22.4 +ARG GOLANG_SHA256=ba79d4526102575196273416239cca418a651e049c2b099f3159db85e7bade7d ARG DOCKER_VERSION=19.03.8 ARG DOCKER_SHA256=7f4115dc6a3c19c917f8b9664d7b51c904def1c984e082c4600097433323cf6f diff --git a/mk/ci-release.Dockerfile b/mk/ci-release.Dockerfile index 02365593..74bbab62 100644 --- a/mk/ci-release.Dockerfile +++ b/mk/ci-release.Dockerfile @@ -1,4 +1,4 @@ -FROM ubuntu:20.04 +FROM ubuntu:22.04 # Pick run-time library packages which match the development packages # used by the ci-builder image. "curl" is included, to allow node-zone.sh From ff3183e6cec48cdfb806b9bb749eb96fc7d0c600 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Fri, 23 Aug 2024 12:13:03 -0500 Subject: [PATCH 8/8] consumer: clean up shard working directory if NewStore() fails Previously we failed to clean these up if NewStore() errored out. --- consumer/recovery.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/consumer/recovery.go b/consumer/recovery.go index a4aa3196..ee089b7c 100644 --- a/consumer/recovery.go +++ b/consumer/recovery.go @@ -4,7 +4,7 @@ import ( "context" "encoding/json" "fmt" - "io/ioutil" + "os" "strings" "time" @@ -219,7 +219,7 @@ func beginRecovery(s *shard) error { // Create local temporary directory into which we recover. var dir string - if dir, err = ioutil.TempDir("", strings.ReplaceAll(spec.Id.String(), "/", "_")+"-"); err != nil { + if dir, err = os.MkdirTemp("", strings.ReplaceAll(spec.Id.String(), "/", "_")+"-"); err != nil { return errors.WithMessage(err, "creating shard working directory") } @@ -278,6 +278,12 @@ func completeRecovery(s *shard) (_ pc.Checkpoint, err error) { } if s.store, err = s.svc.App.NewStore(s, s.recovery.recorder); err != nil { + if s.store == nil { + // s.store is nil, ergo its Destroy() will not be invoked by + // waitAndTearDown(), and we are responsible for best-effort + // cleanup of the playback directory now. + _ = os.RemoveAll(s.recovery.recorder.Dir()) + } return cp, errors.WithMessage(err, "app.NewStore") } else if cp, err = s.store.RestoreCheckpoint(s); err != nil { return cp, errors.WithMessage(err, "store.RestoreCheckpoint")