diff --git a/authentication/authentication.go b/authentication/authentication.go new file mode 100644 index 00000000..0b6647cf --- /dev/null +++ b/authentication/authentication.go @@ -0,0 +1,177 @@ +package authentication + +import ( + "bytes" + "crypto/ecdsa" + "crypto/elliptic" + "crypto/rand" + "crypto/sha256" + "crypto/x509" + "encoding/pem" + "fmt" + "net" +) + +// Elliptic Curve Cryptography (ECC) is a key-based technique for encrypting data. +// ECC focuses on pairs of public and private keys for decryption and encryption of web traffic. +// ECC is frequently discussed in the context of the Rivest–Shamir–Adleman (RSA) cryptographic algorithm. +// +// https://pkg.go.dev/github.com/katzenpost/core/crypto/eddsa +type EllipticCurve struct { + addr net.Addr // used to identify self + pubKeyCurve elliptic.Curve // http://golang.org/pkg/crypto/elliptic/#P256 + privateKey *ecdsa.PrivateKey + publicKey *ecdsa.PublicKey +} + +// New EllipticCurve instance +func New(curve elliptic.Curve) *EllipticCurve { + return &EllipticCurve{ + pubKeyCurve: curve, + privateKey: new(ecdsa.PrivateKey), + } +} + +// GenerateKeys EllipticCurve public and private keys +func (ec *EllipticCurve) GenerateKeys() error { + privKey, err := ecdsa.GenerateKey(ec.pubKeyCurve, rand.Reader) + if err != nil { + return err + } + ec.privateKey = privKey + ec.publicKey = &privKey.PublicKey + return nil +} + +// RegisterKeys EllipticCurve public and private keys +func (ec *EllipticCurve) RegisterKeys(addr net.Addr, privKey *ecdsa.PrivateKey, pubKey *ecdsa.PublicKey) { + ec.addr = addr + ec.privateKey = privKey + ec.publicKey = pubKey +} + +// Returns the EllipticCurve public and private keys +func (ec *EllipticCurve) Keys() (*ecdsa.PrivateKey, *ecdsa.PublicKey) { + return ec.privateKey, ec.publicKey +} + +// Returns the address +func (ec *EllipticCurve) Addr() string { + return ec.addr.String() +} + +// EncodePrivate private key +func (ec *EllipticCurve) EncodePrivate() (string, error) { + encoded, err := x509.MarshalECPrivateKey(ec.privateKey) + if err != nil { + return "", err + } + pemEncoded := pem.EncodeToMemory(&pem.Block{Type: "EC PRIVATE KEY", Bytes: encoded}) + key := string(pemEncoded) + return key, nil +} + +// EncodePublic public key +func (ec *EllipticCurve) EncodePublic() (string, error) { + encoded, err := x509.MarshalPKIXPublicKey(ec.publicKey) + if err != nil { + return "", err + } + pemEncodedPub := pem.EncodeToMemory(&pem.Block{Type: "PUBLIC KEY", Bytes: encoded}) + key := string(pemEncodedPub) + return key, nil +} + +// DecodePrivate private key +func (ec *EllipticCurve) DecodePrivate(pemEncodedPriv string) (*ecdsa.PrivateKey, error) { + blockPriv, _ := pem.Decode([]byte(pemEncodedPriv)) + x509EncodedPriv := blockPriv.Bytes + return x509.ParseECPrivateKey(x509EncodedPriv) +} + +// DecodePublic public key +func (ec *EllipticCurve) DecodePublic(pemEncodedPub string) (*ecdsa.PublicKey, error) { + blockPub, _ := pem.Decode([]byte(pemEncodedPub)) + if blockPub == nil { + return nil, fmt.Errorf("invalid publicKey") + } + x509EncodedPub := blockPub.Bytes + genericPublicKey, err := x509.ParsePKIXPublicKey(x509EncodedPub) + publicKey := genericPublicKey.(*ecdsa.PublicKey) + return publicKey, err +} + +func (ec *EllipticCurve) Sign(msg []byte) ([]byte, error) { + h := sha256.Sum256(msg) + hash := h[:] + signature, err := ecdsa.SignASN1(rand.Reader, ec.privateKey, hash) + if err != nil { + return nil, err + } + return signature, nil +} + +func (ec *EllipticCurve) Hash(msg []byte) []byte { + h := sha256.Sum256(msg) + hash := h[:] + return hash +} + +// VerifySignature sign ecdsa style and verify signature +func (ec *EllipticCurve) VerifySignature(pemEncodedPub string, msg, signature []byte) (bool, error) { + h := sha256.Sum256(msg) + hash := h[:] + pubKey, err := ec.DecodePublic(pemEncodedPub) + if err != nil { + return false, err + } + ok := ecdsa.VerifyASN1(pubKey, hash, signature) + return ok, nil +} + +// VerifySignature sign ecdsa style and verify signature +func (ec *EllipticCurve) SignAndVerify(privKey *ecdsa.PrivateKey, pubKey *ecdsa.PublicKey) ([]byte, bool, error) { + h := sha256.Sum256([]byte("test")) + hash := h[:] + signature, err := ecdsa.SignASN1(rand.Reader, privKey, hash) + if err != nil { + return nil, false, err + } + ok := ecdsa.VerifyASN1(pubKey, hash, signature) + return signature, ok, nil +} + +func (ec *EllipticCurve) EncodeMsg(msg any) ([]byte, error) { + return []byte(fmt.Sprintf("%v", msg)), nil + /*var encodedMsg bytes.Buffer + gob.Register(msg) + enc := gob.NewEncoder(&encodedMsg) + err := enc.Encode(msg) + if err != nil { + return nil, err + } + return encodedMsg.Bytes(), nil*/ +} + +func encodeMsg(msg any) ([]byte, error) { + return []byte(fmt.Sprintf("%v", msg)), nil +} + +func Verify(pemEncodedPub string, signature, digest []byte, msg any) (bool, error) { + encodedMsg, err := encodeMsg(msg) + if err != nil { + return false, err + } + ec := New(elliptic.P256()) + h := sha256.Sum256(encodedMsg) + hash := h[:] + if !bytes.Equal(hash, digest) { + return false, fmt.Errorf("wrong digest") + } + pubKey, err := ec.DecodePublic(pemEncodedPub) + if err != nil { + return false, err + } + ok := ecdsa.VerifyASN1(pubKey, hash, signature) + return ok, nil +} diff --git a/authentication/authentication_test.go b/authentication/authentication_test.go new file mode 100644 index 00000000..579f3045 --- /dev/null +++ b/authentication/authentication_test.go @@ -0,0 +1,128 @@ +package authentication + +import ( + "crypto/elliptic" + "errors" + "reflect" + "testing" +) + +func TestAuthentication(t *testing.T) { + ec := New(elliptic.P256()) + _ = ec.GenerateKeys() + err := ec.test() + if err != nil { + t.Error(err) + } +} + +func TestSignAndVerify(t *testing.T) { + ec1 := New(elliptic.P256()) + err := ec1.GenerateKeys() + if err != nil { + t.Fatal(err) + } + + ec2 := New(elliptic.P256()) + err = ec2.GenerateKeys() + if err != nil { + t.Fatal(err) + } + + message := "This is a message" + + encodedMsg1, err := ec1.EncodeMsg(message) + if err != nil { + t.Error(err) + } + signature, err := ec1.Sign(encodedMsg1) + if err != nil { + t.Error(err) + } + pemEncodedPub, err := ec1.EncodePublic() + if err != nil { + t.Error(err) + } + + encodedMsg2, err := ec2.EncodeMsg(message) + if err != nil { + t.Error(err) + } + ok, err := ec2.VerifySignature(pemEncodedPub, encodedMsg2, signature) + if err != nil { + t.Error(err) + } + if !ok { + t.Error("signature not ok!") + } +} + +func TestVerifyWithWrongPubKey(t *testing.T) { + ec1 := New(elliptic.P256()) + err := ec1.GenerateKeys() + if err != nil { + t.Fatal(err) + } + + ec2 := New(elliptic.P256()) + err = ec2.GenerateKeys() + if err != nil { + t.Fatal(err) + } + + message := "This is a message" + encodedMsg1, err := ec1.EncodeMsg(message) + if err != nil { + t.Error(err) + } + signature, err := ec1.Sign(encodedMsg1) + if err != nil { + t.Error(err) + } + + // encoding ec2 public key instead of ec1 (which was used in signing) + pemEncodedPub, err := ec2.EncodePublic() + if err != nil { + t.Error(err) + } + + encodedMsg2, err := ec2.EncodeMsg(message) + if err != nil { + t.Error(err) + } + ok, err := ec2.VerifySignature(pemEncodedPub, encodedMsg2, signature) + if err != nil { + t.Error(err) + } + if ok { + t.Error("signature should not be ok!") + } +} + +// Can be used in _test.go +// Test encode, decode and test it with deep equal +func (ec *EllipticCurve) test() error { + encPriv, err := ec.EncodePrivate() + if err != nil { + return err + } + encPub, err := ec.EncodePublic() + if err != nil { + return err + } + priv2, err := ec.DecodePrivate(encPriv) + if err != nil { + return err + } + pub2, err := ec.DecodePublic(encPub) + if err != nil { + return err + } + if !reflect.DeepEqual(ec.privateKey, priv2) { + return errors.New("private keys do not match") + } + if !reflect.DeepEqual(ec.publicKey, pub2) { + return errors.New("public keys do not match") + } + return nil +} diff --git a/broadcast.go b/broadcast.go new file mode 100644 index 00000000..1c534fb9 --- /dev/null +++ b/broadcast.go @@ -0,0 +1,209 @@ +package gorums + +import ( + "context" + "crypto/elliptic" + "hash/fnv" + "log/slog" + "strings" + "sync" + "time" + + "github.com/relab/gorums/authentication" + "github.com/relab/gorums/broadcast" + "github.com/relab/gorums/logging" + "github.com/relab/gorums/ordering" + "google.golang.org/protobuf/reflect/protoreflect" +) + +// exposing the log entry struct used for structured logging to the user +type LogEntry logging.LogEntry + +// exposing the ellipticCurve struct for the user +func NewAuth(curve elliptic.Curve) *authentication.EllipticCurve { + return authentication.New(curve) +} + +type broadcastServer struct { + viewMutex sync.RWMutex + id uint32 + addr string + machineID uint64 + view RawConfiguration + createBroadcaster func(m BroadcastMetadata, o *BroadcastOrchestrator, b EnqueueBroadcast) Broadcaster + orchestrator *BroadcastOrchestrator + manager broadcast.Manager + logger *slog.Logger +} + +func newBroadcastServer(serverOpts *serverOptions) *broadcastServer { + h := fnv.New32a() + _, _ = h.Write([]byte(serverOpts.listenAddr)) + id := h.Sum32() + srv := &broadcastServer{ + id: id, + addr: serverOpts.listenAddr, + logger: serverOpts.logger, + machineID: serverOpts.machineID, + } + srv.manager = broadcast.NewBroadcastManager(serverOpts.logger, createClient, srv.canceler, serverOpts.executionOrder, serverOpts.clientDialTimeout, serverOpts.reqTTL, serverOpts.shardBuffer, serverOpts.sendBuffer, serverOpts.allowList, serverOpts.grpcDialOpts...) + srv.manager.AddAddr(srv.id, serverOpts.listenAddr, srv.machineID) + return srv +} + +func (srv *broadcastServer) stop() { + srv.manager.Close() +} + +type Snowflake interface { + NewBroadcastID() uint64 +} + +const ( + BroadcastID string = "broadcastID" +) + +type BroadcastHandlerFunc func(method string, req protoreflect.ProtoMessage, broadcastID uint64, enqueueBroadcast EnqueueBroadcast, options ...broadcast.BroadcastOptions) error +type BroadcastForwardHandlerFunc func(req protoreflect.ProtoMessage, method string, broadcastID uint64, forwardAddr, originAddr string) +type BroadcastServerHandlerFunc func(method string, req protoreflect.ProtoMessage, options ...broadcast.BroadcastOptions) +type BroadcastSendToClientHandlerFunc func(broadcastID uint64, resp protoreflect.ProtoMessage, err error, enqueueBroadcast EnqueueBroadcast) error +type CancelHandlerFunc func(broadcastID uint64, srvAddrs []string, enqueueBroadcast EnqueueBroadcast) error +type DoneHandlerFunc func(broadcastID uint64, enqueueBroadcast EnqueueBroadcast) +type EnqueueBroadcast func(*broadcast.Msg) error + +type defaultImplementationFunc[T protoreflect.ProtoMessage, V protoreflect.ProtoMessage] func(ServerCtx, T) (V, error) +type clientImplementationFunc[T protoreflect.ProtoMessage, V protoreflect.ProtoMessage] func(context.Context, T, uint64) (V, error) + +type implementationFunc[T protoreflect.ProtoMessage, V Broadcaster] func(ServerCtx, T, V) + +func CancelFunc(ServerCtx, protoreflect.ProtoMessage, Broadcaster) {} + +const Cancellation string = "cancel" + +// The BroadcastOrchestrator is used as a container for all +// broadcast handlers. The BroadcastHandler takes in a method +// and schedules it for broadcasting. SendToClientHandler works +// similarly but it sends the message to the calling client. +// +// It is necessary to use an orchestrator to hide certain +// implementation details, such as internal methods on the +// broadcast struct. The BroadcastOrchestrator will thus +// be an unimported field in the broadcast struct in the +// generated code. +type BroadcastOrchestrator struct { + BroadcastHandler BroadcastHandlerFunc + ForwardHandler BroadcastForwardHandlerFunc + SendToClientHandler BroadcastSendToClientHandlerFunc + ServerBroadcastHandler BroadcastServerHandlerFunc + CancelHandler CancelHandlerFunc + DoneHandler DoneHandlerFunc +} + +func NewBroadcastOrchestrator(srv *Server) *BroadcastOrchestrator { + return &BroadcastOrchestrator{ + BroadcastHandler: srv.broadcastSrv.broadcastHandler, + ForwardHandler: srv.broadcastSrv.forwardHandler, + ServerBroadcastHandler: srv.broadcastSrv.serverBroadcastHandler, + SendToClientHandler: srv.broadcastSrv.sendToClientHandler, + CancelHandler: srv.broadcastSrv.cancelHandler, + DoneHandler: srv.broadcastSrv.doneHandler, + } +} + +type BroadcastOption func(*broadcast.BroadcastOptions) + +// WithSubset enables broadcasting to a subset of the servers in the view. +// It has the same function as broadcast.To(). +func WithSubset(srvAddrs ...string) BroadcastOption { + return func(b *broadcast.BroadcastOptions) { + b.ServerAddresses = srvAddrs + } +} + +// WithoutSelf prevents the server from broadcasting to itself. +func WithoutSelf() BroadcastOption { + return func(b *broadcast.BroadcastOptions) { + b.SkipSelf = true + } +} + +// ProgressTo allows the server to accept messages to the given method. +// Should only be used if the ServerOption WithOrder() is used. +func ProgressTo(method string) BroadcastOption { + return func(b *broadcast.BroadcastOptions) { + b.ProgressTo = method + } +} + +// AllowDuplication allows the server to broadcast more than once +// to the same RPC method for a particular broadcast request. +func AllowDuplication() BroadcastOption { + return func(b *broadcast.BroadcastOptions) { + b.AllowDuplication = true + } +} + +// WithRelationToRequest allows for broadcasting outside a +// server handler related to a specific broadcastID. +// It is not recommended to use this method. Use the broadcast +// struct provided with a broadcast request instead. +func WithRelationToRequest(broadcastID uint64) BroadcastOption { + return func(b *broadcast.BroadcastOptions) { + b.RelatedToReq = broadcastID + } +} + +func NewBroadcastOptions() broadcast.BroadcastOptions { + return broadcast.BroadcastOptions{ + ServerAddresses: make([]string, 0), // to prevent nil errors + } +} + +type Broadcaster interface{} + +type BroadcastMetadata struct { + BroadcastID uint64 + IsBroadcastClient bool // type of sender, could be: Client or Server + SenderAddr string // address of last hop + OriginAddr string // address of the origin + OriginMethod string // the first method called by the origin + Method string // the current method + Timestamp time.Time // timestamp in seconds when the broadcast request was issued by the client/server + ShardID uint16 // ID of the shard handling the broadcast request + MachineID uint16 // ID of the client/server that issued the broadcast request + SequenceNo uint32 // sequence number of the broadcast request from that particular client/server. Will roll over when reaching max. + OriginDigest []byte + OriginSignature []byte + OriginPubKey string +} + +func newBroadcastMetadata(md *ordering.Metadata) BroadcastMetadata { + if md == nil { + return BroadcastMetadata{} + } + tmp := strings.Split(md.Method, ".") + m := "" + if len(tmp) >= 1 { + m = tmp[len(tmp)-1] + } + timestamp, shardID, machineID, sequenceNo := broadcast.DecodeBroadcastID(md.BroadcastMsg.BroadcastID) + return BroadcastMetadata{ + BroadcastID: md.BroadcastMsg.BroadcastID, + IsBroadcastClient: md.BroadcastMsg.IsBroadcastClient, + SenderAddr: md.BroadcastMsg.SenderAddr, + OriginAddr: md.BroadcastMsg.OriginAddr, + OriginMethod: md.BroadcastMsg.OriginMethod, + OriginDigest: md.BroadcastMsg.OriginDigest, + OriginSignature: md.BroadcastMsg.OriginSignature, + OriginPubKey: md.BroadcastMsg.OriginPubKey, + Method: m, + Timestamp: broadcast.Epoch().Add(time.Duration(timestamp) * time.Second), + ShardID: shardID, + MachineID: machineID, + SequenceNo: sequenceNo, + } +} + +func (md BroadcastMetadata) Verify(msg protoreflect.ProtoMessage) (bool, error) { + return authentication.Verify(md.OriginPubKey, md.OriginSignature, md.OriginDigest, msg) +} diff --git a/broadcast/README.md b/broadcast/README.md new file mode 100644 index 00000000..554bb267 --- /dev/null +++ b/broadcast/README.md @@ -0,0 +1,264 @@ +# Broadcast + +### Preliminary + +The main contribution to Gorums in this Master's thesis are the following files and folders. The root directory is gorums (github.com/relab/gorums) and is not specified in the list for brevity: + +- All files in this folder. (github.com/relab/gorums/broadcast) +- broadcast.go +- handler.go +- clientserver.go +- tests/broadcast +- authentication +- logging + +Additionally, we have made contributions to most of the other files. These changes are presented in this draft pull request: + + https://github.com/relab/gorums/pull/176 + +## Documentation + +We will use an example when presenting the broadcast framework. This example is Eager Reliable Broadcast and is inspired by the implementation on page 124 in this [book](https://link.springer.com/book/10.1007/978-3-642-15260-3). + +#### Prerequisites + +There are no additional prerequisites needed to enable the broadcast framework functionality. The functionality is compatible with the current version of Gorums. If you are using Gorums for the first time, we refer you to the README file in the root directory. + +### Proto file + +The broadcast framework provides two proto options: + +- broadcastcall: Used as entrypoint for clients. +- broadcast: Used by servers to communicate to each others. + +```proto3 +import "gorums.proto"; + +service ReliableBroadcast { + rpc Broadcast(Message) returns (Message) { + option (gorums.broadcastcall) = true; + } + rpc Deliver(Message) returns (Empty) { + option (gorums.broadcast) = true; + } +} + +message Message { + string Data = 1; +} + +message Empty {} +``` + +Notice that the return type of the RPC method `Deliver`. The return type is not used because servers only communicate by broadcasting to each other. The method with broadcastcall does, however, have a return type. This is the type that the servers will reply with when the client invokes `Broadcast`. + +### Client + +After generating the proto files we can define the client in a file named `client.go`: + +```go +type Client struct { + mgr *pb.Manager + config *pb.Configuration +} + +func New(addr string, srvAddresses []string, qSize int) *Client { + mgr := pb.NewManager( + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + lis, _ := net.Listen("tcp", addr) + mgr.AddClientServer(lis, lis.Addr()) + config, _ := mgr.NewConfiguration( + NewQSpec(qSize), + gorums.WithNodeList(srvAddresses), + ) + return &Client{ + mgr: mgr, + config: config, + } +} +``` + +The only addition the broadcast framework brings to how the client is created is the two lines: + +```go +lis, _ := net.Listen("tcp", addr) +mgr.AddClientServer(lis, lis.Addr()) +``` + +The first line creates a listener and the second creates a client-side server. This is necessary in order to accept replies from servers not added to the Gorums configuration. + +Next we can create a function on the `Client` that can be used to invoke broadcasts on the configuration: + +```go +func (c *Client) Broadcast(ctx context.Context, value string) (*pb.Message, error) { + req := &pb.Message{ + Data: value, + } + return c.config.Broadcast(ctx, req) +} +``` + +To be able to collect responses it is also necessary to create a quorum function. When generating the proto files, Gorums will create a QuorumSpec interface containing all quorum functions. In our example this QuorumSpec is generated: + +```go +// QuorumSpec is the interface of quorum functions for ReliableBroadcast. +type QuorumSpec interface { + gorums.ConfigOption + + BroadcastQF(in *Message, replies []*Message) (*Message, bool) +} +``` + +We can then proceed to implement the interface by creating a struct named QSpec that contains all the methods in the QuorumSpec: + +```go +type QSpec struct { + quorumSize int +} + +func NewQSpec(qSize int) pb.QuorumSpec { + return &QSpec{ + quorumSize: qSize, + } +} + +func (qs *QSpec) BroadcastQF(in *pb.Message, replies []*pb.Message) (*pb.Message, bool) { + if len(replies) < qs.quorumSize { + return nil, false + } + return replies[0], true +} +``` + +This `QSpec` struct is used when the Gorums configuration is created. This can be seen in the code example above when we created the client. Here we provide the `NewQSpec` as one of the arguments to the `mgr.NewConfiguration()` function. + +### Server + +To create a server that uses the broadcast functionality we can define a file `server.go` containing the server implementation: + +```go +type Server struct { + *pb.Server + mut sync.Mutex + delivered []*pb.Message + mgr *pb.Manager + addr string + srvAddrs []string +} + +func New(addr string, srvAddrs []string) *Server { + lis, _ := net.Listen("tcp", s.addr) + srv := Server{ + Server: pb.NewServer(gorums.WithListenAddr(lis.Addr())), + addr: addr, + srvAddrs: srvAddrs, + delivered: make([]*pb.Message, 0), + } + srv.configureView() + pb.RegisterReliableBroadcastServer(srv.Server, &srv) + go srv.Serve(lis) + return &srv +} +``` + +The first addition by the broadcast framework when creating the server is that we need to provide the option `gorums.WithListenAddr(lis.Addr())`. This is important because the address of the server is used in the messages sent by the server. Furthermore, we also invoke a function named `configureView()`: + +```go +func (srv *Server) configureView() { + srv.mgr = pb.NewManager( + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + view, err := srv.mgr.NewConfiguration(gorums.WithNodeList(srv.srvAddrs)) + if err != nil { + panic(err) + } + srv.SetView(view) +} +``` + +By creating a Gorums configuration and providing it to the generated method `SetView()` we enable server-to-server communication. We use the term `view` when refering to the Gorums configuration on the server side to distinguish it from the configuration created on the client-side. + +When we have created the server, we can define the server handlers: + +```go +func (s *Server) Broadcast(ctx gorums.ServerCtx, request *pb.Message, broadcast *pb.Broadcast) { + broadcast.Deliver(request) +} + +func (s *Server) Deliver(ctx gorums.ServerCtx, request *pb.Message, broadcast *pb.Broadcast) { + s.mut.Lock() + defer s.mut.Unlock() + if !s.isDelivered(request) { + s.delivered = append(s.delivered, request) + broadcast.Deliver(request) + broadcast.SendToClient(request, nil) + } +} + +func (s *Server) isDelivered(message *pb.Message) bool { + for _, msg := range s.delivered { + if msg.Data == message.Data { + return true + } + } + return false +} +``` + +The server handler signatures have changed a little, as evident from the code. The broadcast framework removes the return types and introduces a new argument named `broadcast`. This struct is the main interface for interacting with the broadcast framework. Each RPC method in the protofile with the option `gorums.broadcast = true` will be generated on the `broadcast struct`. + +The server handler `Broadcast` is quite simple and only contains a single invocation `broadcast.Deliver(request)`. This invocation will broadcast the request to all servers added to the view. + +The next server handler `Deliver` first checks whether the request has already been delivered. If not, it broadcasts `Deliver` to the other servers with the request and sends a reply to the client. + +The broadcast framework handles issues related to late client messages, duplicated broadcasts, and routing of messages. Hence, this is a complete code example that is correct according to the [description of the algorithm](https://link.springer.com/book/10.1007/978-3-642-15260-3). + +## Options + +We have implemented a set of options that can be used to configure the broadcasting functionality. These will be presented in short here: + +#### Broadcast Server + +- `WithShardBuffer(shardBuffer) ServerOption`: Enables the user to specify the buffer size of each shard. A shard stores a map of broadcast requests. A higher buffer size may increase throughput but at the cost of higher memory consumption. The default is 200 broadcast requests. +- `WithSendBuffer(sendBuffer) ServerOption`: Enables the user to specify the buffer size of the communication channels to the broadcast processor. A higher buffer size may increase throughput but at the cost of higher memory consumption. The default is 30 messages. +- `WithBroadcastReqTTL(ttl) ServerOption`: Configures the duration a broadcast request should live on a server, setting the lifetime of a broadcast processor. The default is 5 minutes. + +#### Broadcasting + +- `WithSubset(srvAddrs) BroadcastOption`: Allows the user to specify a subset of servers to broadcast to. The server addresses given must be a subset of the addresses in the server view. +- `WithoutSelf() BroadcastOption`: Prevents the server from broadcasting to itself. +- `AllowDuplication() BroadcastOption`: Allows the user to broadcast to the same RPC method more than once for a particular broadcast request. + +#### Identification + +- `WithMachineID(id) ManagerOption`: Enables the user to set a unique ID for the client. This ID will be embedded in broadcast requests sent from the client, making the requests trackable by the whole cluster. A random ID will be generated if not set, which can cause collisions if there are many clients. The ID is bounded between 0 and 4095. +- `WithSrvID(id) ServerOption`: Enables the user to set a unique ID on the broadcast server. This ID is used to generate BroadcastIDs. +- `WithListenAddr(addr) ServerOption`: Sets the IP address of the broadcast server, which will be used in messages sent by the server. The network of the address must be a TCP network name. + +#### Connection + +- `WithSendRetries(maxRetries) ManagerOption`: Allows the user to specify how many times the node will try to send a message. The message will be dropped if it fails to send more than the specified number of times. Providing `maxRetries = -1` will retry indefinitely. +- `WithConnRetries(maxRetries) ManagerOption`: Allows the user to specify how many times the node will try to reconnect to a node. The default is no limit, but it will follow a backoff strategy. +- `WithClientDialTimeout(timeout) ServerOption`: Enables the user to set a dial timeout for servers when sending replies back to the client in a BroadcastCall. The default is 10 seconds. +- `WithServerGrpcDialOptions(opts) ServerOption`: Enables the user to set gRPC dial options that the Broadcast Router uses when connecting to a client. + +#### Logging + +- `WithLogger(logger) ManagerOption`: Enables the user to provide a structured logger for the Manager. This will log events regarding the creation of nodes and the transmission of messages. +- `WithSLogger(logger) ServerOption`: Enables the user to set a structured logger for the Server. This will log internal events regarding broadcast requests. + +#### Authentication + +- `WithAllowList(allowed) ServerOption`: Enables the user to provide a list of (address, publicKey) pairs which will be used to validate messages. Only nodes on the allow list are permitted to send messages to the server, and the server is only allowed to send replies to nodes on the allow list. +- `EnforceAuthentication() ServerOption`: Requires that messages are signed and validated; otherwise, the server will drop them. +- `WithAuthentication() ManagerOption`: Enables digital signatures for messages. + +#### Execution Ordering + +- `WithOrder(method_1, ..., method_n) ServerOption`: Enables the user to specify the order in which methods should be executed. This option does not order messages but caches messages meant for processing at a later stage. For example, in PBFT, it caches all commit messages if the state is not prepared yet. +- `ProgressTo(method_i) BroadcastOption`: Allows the server to accept messages for the given method or for methods prior in the execution order. diff --git a/broadcast/consts.go b/broadcast/consts.go new file mode 100644 index 00000000..0979b127 --- /dev/null +++ b/broadcast/consts.go @@ -0,0 +1,23 @@ +package broadcast + +import ( + "context" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type BroadcastOptions struct { + ServerAddresses []string + AllowDuplication bool + SkipSelf bool + ProgressTo string + RelatedToReq uint64 +} + +const ( + BroadcastID string = "broadcastID" + // special origin addr used in creating a broadcast request from a server + ServerOriginAddr string = "server" +) + +type ServerHandler func(ctx context.Context, in protoreflect.ProtoMessage, broadcastID uint64, originAddr, originMethod string, options BroadcastOptions, id uint32, addr string, originDigest, originSignature []byte, originPubKey string) diff --git a/broadcast/errors.go b/broadcast/errors.go new file mode 100644 index 00000000..a66975c0 --- /dev/null +++ b/broadcast/errors.go @@ -0,0 +1,57 @@ +package broadcast + +type BroadcastIDErr struct{} + +func (err BroadcastIDErr) Error() string { + return "wrong broadcastID" +} + +type MissingClientReqErr struct{} + +func (err MissingClientReqErr) Error() string { + return "has not received client req yet" +} + +type AlreadyProcessedErr struct{} + +func (err AlreadyProcessedErr) Error() string { + return "already processed request" +} + +type ReqFinishedErr struct{} + +func (err ReqFinishedErr) Error() string { + return "request has terminated" +} + +type ClientReqAlreadyReceivedErr struct{} + +func (err ClientReqAlreadyReceivedErr) Error() string { + return "the client req has already been received. The forward req is thus dropped." +} + +type MissingReqErr struct{} + +func (err MissingReqErr) Error() string { + return "a request has not been created yet." +} + +type OutOfOrderErr struct{} + +func (err OutOfOrderErr) Error() string { + return "the message is out of order" +} + +type ShardDownErr struct{} + +func (err ShardDownErr) Error() string { + return "the shard is down" +} + +type InvalidAddrErr struct { + addr string +} + +func (err InvalidAddrErr) Error() string { + return "provided addr is invalid. got: " + err.addr +} diff --git a/broadcast/manager.go b/broadcast/manager.go new file mode 100644 index 00000000..be28222d --- /dev/null +++ b/broadcast/manager.go @@ -0,0 +1,164 @@ +package broadcast + +import ( + "context" + "log/slog" + "time" + + "google.golang.org/grpc" + "google.golang.org/protobuf/reflect/protoreflect" +) + +type Manager interface { + Process(*Content) (context.Context, func(*Msg) error, error) + Broadcast(uint64, protoreflect.ProtoMessage, string, func(*Msg) error, ...BroadcastOptions) error + SendToClient(uint64, protoreflect.ProtoMessage, error, func(*Msg) error) error + Cancel(uint64, []string, func(*Msg) error) error + Done(uint64, func(*Msg) error) + NewBroadcastID() uint64 + AddAddr(id uint32, addr string, machineID uint64) + AddHandler(method string, handler any) + Close() error + ResetState() +} + +type manager struct { + state *BroadcastState + router *BroadcastRouter + logger *slog.Logger +} + +func NewBroadcastManager(logger *slog.Logger, createClient func(addr string, dialOpts []grpc.DialOption) (*Client, error), canceler func(broadcastID uint64, srvAddrs []string), order map[string]int, dialTimeout, reqTTL time.Duration, shardBuffer, sendBuffer int, allowList map[string]string, dialOpts ...grpc.DialOption) Manager { + router := NewRouter(logger, createClient, canceler, dialTimeout, allowList, dialOpts...) + state := NewState(logger, router, order, reqTTL, shardBuffer, sendBuffer) + router.registerState(state) + return &manager{ + state: state, + router: router, + logger: logger, + } +} + +func (mgr *manager) Process(msg *Content) (context.Context, func(*Msg) error, error) { + _, shardID, _, _ := DecodeBroadcastID(msg.BroadcastID) + shardID = shardID % NumShards + shard := mgr.state.shards[shardID] + + // we only need a single response + receiveChan := make(chan shardResponse, 1) + msg.ReceiveChan = receiveChan + resp := shard.handleMsg(msg) + return resp.reqCtx, resp.enqueueBroadcast, resp.err +} + +func (mgr *manager) Broadcast(broadcastID uint64, req protoreflect.ProtoMessage, method string, enqueueBroadcast func(*Msg) error, opts ...BroadcastOptions) error { + var options BroadcastOptions + if len(opts) > 0 { + options = opts[0] + } + msg := &Msg{ + MsgType: BroadcastMsg, + Msg: NewMsg(broadcastID, req, method, options), + Method: method, + BroadcastID: broadcastID, + } + // fast path: communicate directly with the broadcast request + if enqueueBroadcast != nil { + return enqueueBroadcast(msg) + } + // slow path: communicate with the shard first + _, shardID, _, _ := DecodeBroadcastID(broadcastID) + shardID = shardID % NumShards + shard := mgr.state.getShard(shardID) + shard.handleBMsg(msg) + return nil +} + +func (mgr *manager) SendToClient(broadcastID uint64, resp protoreflect.ProtoMessage, err error, enqueueBroadcast func(*Msg) error) error { + msg := &Msg{ + MsgType: ReplyMsg, + Reply: &reply{ + Response: resp, + Err: err, + }, + BroadcastID: broadcastID, + } + // fast path: communicate directly with the broadcast request + if enqueueBroadcast != nil { + return enqueueBroadcast(msg) + } + // slow path: communicate with the shard first + _, shardID, _, _ := DecodeBroadcastID(broadcastID) + shardID = shardID % NumShards + shard := mgr.state.getShard(shardID) + shard.handleBMsg(msg) + return nil +} + +func (mgr *manager) Cancel(broadcastID uint64, srvAddrs []string, enqueueBroadcast func(*Msg) error) error { + msg := &Msg{ + MsgType: CancellationMsg, + Cancellation: &cancellation{ + end: false, // should NOT stop the request + srvAddrs: srvAddrs, + }, + BroadcastID: broadcastID, + } + if enqueueBroadcast != nil { + return enqueueBroadcast(msg) + } + _, shardID, _, _ := DecodeBroadcastID(broadcastID) + shardID = shardID % NumShards + shard := mgr.state.getShard(shardID) + shard.handleBMsg(msg) + return nil +} + +func (mgr *manager) Done(broadcastID uint64, enqueueBroadcast func(*Msg) error) { + msg := &Msg{ + MsgType: CancellationMsg, + Cancellation: &cancellation{ + end: true, // should stop the request + }, + BroadcastID: broadcastID, + } + if enqueueBroadcast != nil { + // no need to check error because the processor + // is stopped. + _ = enqueueBroadcast(msg) + return + } + _, shardID, _, _ := DecodeBroadcastID(broadcastID) + shardID = shardID % NumShards + shard := mgr.state.getShard(shardID) + shard.handleBMsg(msg) +} + +func (mgr *manager) NewBroadcastID() uint64 { + return mgr.state.snowflake.NewBroadcastID() +} + +func (mgr *manager) AddAddr(id uint32, addr string, machineID uint64) { + mgr.router.id = id + mgr.router.addr = addr + mgr.state.snowflake = NewSnowflake(machineID) +} + +func (mgr *manager) AddHandler(method string, handler any) { + switch h := handler.(type) { + case ServerHandler: + mgr.router.serverHandlers[method] = h + default: + // only needs to know whether the handler exists. routing is done + // client-side using the provided metadata in the request. + mgr.router.clientHandlers[method] = struct{}{} + } +} + +func (mgr *manager) Close() error { + return mgr.state.Close() +} + +func (mgr *manager) ResetState() { + mgr.state.reset() +} diff --git a/broadcast/processor.go b/broadcast/processor.go new file mode 100644 index 00000000..55f22e5a --- /dev/null +++ b/broadcast/processor.go @@ -0,0 +1,552 @@ +package broadcast + +import ( + "context" + "log/slog" + "time" + + "github.com/relab/gorums/logging" + "google.golang.org/protobuf/reflect/protoreflect" +) + +type BroadcastProcessor struct { + broadcastID uint64 + router Router + broadcastChan chan *Msg + sendChan chan *Content + ctx context.Context + cancelFunc context.CancelFunc + started time.Time + ended time.Time + logger *slog.Logger + + cancellationCtx context.Context + cancellationCtxCancel context.CancelFunc + + // ordering + executionOrder map[string]int + orderIndex int + outOfOrderMsgs map[string][]*Content +} + +type metadata struct { + OriginAddr string + OriginMethod string + Sent bool + ResponseMsg protoreflect.ProtoMessage + ResponseErr error + SendFn func(protoreflect.ProtoMessage, error) error + IsBroadcastClient bool + SentCancellation bool + HasReceivedClientReq bool + OriginDigest []byte + OriginPubKey string + OriginSignature []byte +} + +func (p *BroadcastProcessor) run(msg *Content) { + // defining metadata and methods here to prevent allocation on the heap + metadata := &metadata{ + OriginAddr: msg.OriginAddr, + OriginMethod: msg.OriginMethod, + IsBroadcastClient: msg.IsBroadcastClient, + SendFn: msg.SendFn, + Sent: false, + SentCancellation: false, + OriginDigest: msg.OriginDigest, + OriginSignature: msg.OriginSignature, + OriginPubKey: msg.OriginPubKey, + } + // methods is placed here and not in the metadata as an optimization strategy. + // Testing shows that it does not allocate memory for it on the heap. + methods := make([]string, 0, 3) + p.initialize(msg, metadata) + defer p.cleanup(metadata) + for { + select { + case <-p.ctx.Done(): + // processor is done + return + case bMsg := <-p.broadcastChan: + // we have received an outgoing message from a server handler + if p.broadcastID != bMsg.BroadcastID { + p.log("broadcast: wrong BroadcastID", BroadcastIDErr{}, logging.MsgType(bMsg.MsgType.String()), logging.Stopping(false)) + continue + } + switch bMsg.MsgType { + case CancellationMsg: + if p.handleCancellation(bMsg, metadata) { + return + } + case BroadcastMsg: + if p.handleBroadcast(bMsg, methods, metadata) { + // methods keeps track of which methods has been broadcasted to. + // This prevents duplicate broadcasts. + methods = append(methods, bMsg.Method) + } + case ReplyMsg: + if p.handleReply(bMsg, metadata) { + // request is done if a reply is sent to the client. + return + } + } + case new := <-p.sendChan: + // we have received an incoming message from a server or client + if p.handleMsg(new, metadata) { + return + } + } + } +} + +func (p *BroadcastProcessor) handleCancellation(bMsg *Msg, metadata *metadata) bool { + if bMsg.Cancellation.end { + p.log("broadcast: broadcast.Done() called", nil, logging.MsgType(bMsg.MsgType.String()), logging.Stopping(true)) + return true + } + if !metadata.SentCancellation { + p.log("broadcast: sent cancellation", nil, logging.MsgType(bMsg.MsgType.String()), logging.Stopping(false)) + metadata.SentCancellation = true + go func(broadcastID uint64, cancellationMsg *cancellation) { + _ = p.router.Send(broadcastID, "", "", nil, nil, "", cancellationMsg) + }(p.broadcastID, bMsg.Cancellation) + } + return false +} + +func (p *BroadcastProcessor) handleBroadcast(bMsg *Msg, methods []string, metadata *metadata) bool { + // check if msg has already been broadcasted for this method + //if alreadyBroadcasted(p.metadata.Methods, bMsg.Method) { + if !bMsg.Msg.options.AllowDuplication && alreadyBroadcasted(methods, bMsg.Method) { + return false + } + err := p.router.Send(p.broadcastID, metadata.OriginAddr, metadata.OriginMethod, metadata.OriginDigest, metadata.OriginSignature, metadata.OriginPubKey, bMsg.Msg) + p.log("broadcast: sending broadcast", err, logging.MsgType(bMsg.MsgType.String()), logging.Method(bMsg.Method), logging.Stopping(false), logging.IsBroadcastCall(metadata.isBroadcastCall())) + + p.updateOrder(bMsg.Method, bMsg.Msg.options.ProgressTo) + p.dispatchOutOfOrderMsgs() + return true +} + +func (p *BroadcastProcessor) handleReply(bMsg *Msg, metadata *metadata) bool { + // BroadcastCall if origin addr is non-empty. + if metadata.isBroadcastCall() { + go func(broadcastID uint64, originAddr, originMethod string, replyMsg *reply) { + err := p.router.Send(broadcastID, originAddr, originMethod, metadata.OriginDigest, metadata.OriginSignature, metadata.OriginPubKey, replyMsg) + p.log("broadcast: sent reply to client", err, logging.Method(originMethod), logging.MsgType(bMsg.MsgType.String()), logging.Stopping(true), logging.IsBroadcastCall(metadata.isBroadcastCall())) + }(p.broadcastID, metadata.OriginAddr, metadata.OriginMethod, bMsg.Reply) + // the request is done becuase we have sent a reply to the client + p.log("broadcast: sending reply to client", nil, logging.Method(metadata.OriginMethod), logging.MsgType(bMsg.MsgType.String()), logging.Stopping(true), logging.IsBroadcastCall(metadata.isBroadcastCall())) + return true + } + // QuorumCall if origin addr is empty. + + // this sends a reply back to the client only if the client has + // connected to the server. Otherwise, an error will be returned. + // We thus need to cache the msg until the client has connected to + // the server. + err := metadata.send(bMsg.Reply.Response, bMsg.Reply.Err) + if err != nil { + // add response if not already done + if metadata.ResponseMsg == nil { + metadata.ResponseMsg = bMsg.Reply.Response + metadata.ResponseErr = bMsg.Reply.Err + metadata.Sent = true + } + // the request is not done yet because we have not replied to + // the client. + p.log("broadcast: failed to send reply to client", err, logging.Method(metadata.OriginMethod), logging.MsgType(bMsg.MsgType.String()), logging.Stopping(false), logging.IsBroadcastCall(metadata.isBroadcastCall())) + // we must stop the goroutine if we have received the client req. This can mean that + // the client no longer accepts replies or has gone offline. In any case, the operation + // is done. + return metadata.hasReceivedClientRequest() + } + // the request is done becuase we have sent a reply to the client + p.log("broadcast: sending reply to client", err, logging.Method(metadata.OriginMethod), logging.MsgType(bMsg.MsgType.String()), logging.Stopping(true), logging.IsBroadcastCall(metadata.isBroadcastCall())) + return true +} + +func (p *BroadcastProcessor) handleMsg(new *Content, metadata *metadata) bool { + if p.broadcastID != new.BroadcastID { + new.ReceiveChan <- shardResponse{ + err: BroadcastIDErr{}, + } + p.log("msg: wrong BroadcastID", BroadcastIDErr{}, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return false + } + if new.IsCancellation { + if p.cancellationCtxCancel != nil { + p.cancellationCtxCancel() + } + // the cancellation implementation is just an + // empty function and does not need the ctx or + // broadcastChan. + new.ReceiveChan <- shardResponse{ + err: nil, + } + p.log("msg: received cancellation", nil, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return false + } + + if new.IsBroadcastClient { + if metadata.HasReceivedClientReq { + // this is a duplicate request, possibly from a forward operation. + // the req should simply be dropped. + new.ReceiveChan <- shardResponse{ + err: ClientReqAlreadyReceivedErr{}, + } + p.log("msg: duplicate client req", ClientReqAlreadyReceivedErr{}, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return false + } + // important to set this option to prevent duplicate client reqs. + // this can be the result if a server forwards the req but the + // leader has already received the client req. + metadata.HasReceivedClientReq = true + go func() { + // new.Ctx will correspond to the streamCtx between the client and this server. + // We can thus listen to it and signal a cancellation if the client goes offline + // or cancels the request. We also have to listen to the p.ctx to prevent leaking + // the goroutine. + select { + case <-p.ctx.Done(): + case <-new.Ctx.Done(): + } + // when the processor returns it sets p.cancellationCtxCancel = nil. + // Hence, it is important to check if it is nil. Also, the processor + // calls this function when it returns. + if p.cancellationCtxCancel != nil { + p.cancellationCtxCancel() + } + }() + p.log("msg: received client req", nil, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + } + + metadata.update(new) + // this only pertains to requests where the server has a + // direct connection to the client, e.g. QuorumCall. + if metadata.Sent && !metadata.isBroadcastCall() { + // we must return an error to prevent executing the implementation func. + // This is because the server has finished the request and tried to reply + // to the client previously. The msg we have just received is from the client, + // meaning we can finally return the cached response. + err := metadata.send(metadata.ResponseMsg, metadata.ResponseErr) + if err == nil { + err = AlreadyProcessedErr{} + } + new.ReceiveChan <- shardResponse{ + err: err, + } + p.log("msg: late msg", err, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return true + } + if !p.isInOrder(new.CurrentMethod) { + // save the message and execute it later + p.addToOutOfOrder(new) + new.ReceiveChan <- shardResponse{ + err: OutOfOrderErr{}, + } + p.log("msg: out of order", OutOfOrderErr{}, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return false + } + new.ReceiveChan <- shardResponse{ + err: nil, + reqCtx: p.cancellationCtx, + enqueueBroadcast: p.enqueueBroadcast, + } + p.log("msg: processed", nil, logging.Method(new.CurrentMethod), logging.From(new.SenderAddr)) + return false +} + +func (p *BroadcastProcessor) log(msg string, err error, args ...slog.Attr) { + if p.logger != nil { + args = append(args, logging.Err(err), logging.Type("broadcast processor")) + level := slog.LevelInfo + if err != nil { + level = slog.LevelError + } + p.logger.LogAttrs(context.Background(), level, msg, args...) + } +} + +func (m *metadata) update(new *Content) { + if m.OriginAddr == "" && new.OriginAddr != "" { + m.OriginAddr = new.OriginAddr + } + if m.OriginMethod == "" && new.OriginMethod != "" { + m.OriginMethod = new.OriginMethod + } + if m.OriginPubKey == "" && new.OriginPubKey != "" { + m.OriginPubKey = new.OriginPubKey + } + if m.OriginSignature == nil && new.OriginSignature != nil { + m.OriginSignature = new.OriginSignature + } + if m.OriginDigest == nil && new.OriginDigest != nil { + m.OriginDigest = new.OriginDigest + } + if m.SendFn == nil && new.SendFn != nil { + m.SendFn = new.SendFn + m.IsBroadcastClient = new.IsBroadcastClient + } +} + +func (m *metadata) isBroadcastCall() bool { + return m.OriginAddr != "" +} + +func (m *metadata) send(resp protoreflect.ProtoMessage, err error) error { + if !m.hasReceivedClientRequest() { + return MissingClientReqErr{} + } + // error is intentionally ignored. We have not setup retry logic for failed + // deliveries to clients. Responding with nil will stop the broadcast request + // which is needed to prevent many stale goroutines. + _ = m.SendFn(resp, err) + return nil +} + +func (m *metadata) hasReceivedClientRequest() bool { + return m.IsBroadcastClient && m.SendFn != nil +} + +func (p *BroadcastProcessor) emptyChannels(metadata *metadata) { + for { + select { + case msg := <-p.broadcastChan: + if p.broadcastID != msg.BroadcastID { + continue + } + switch msg.MsgType { + case CancellationMsg: + // it is possible to call SendToClient() before Cancel() in the same + // server handler. Since SendToClient() will stop the processor, we need + // to handle the cancellation here. We don't want to send duplicate + // cancellations and thus we only want to send a cancellation if the + // request has been stopped. + p.handleCancellation(msg, metadata) + case BroadcastMsg: + // broadcasts are not performed after a reply to the client is sent. + // this is to prevent duplication and processing of old messages. + case ReplyMsg: + // a reply should not be sent after the processor is done. This is + // because either: + // 1. A reply has already been sent + // 2. Done() has been called. SendToClient() should not be used together + // with this method + } + default: + return + } + } +} + +func (r *BroadcastProcessor) initOrder() { + // the implementer has not specified an execution order + if r.executionOrder == nil || len(r.executionOrder) <= 0 { + return + } + r.outOfOrderMsgs = make(map[string][]*Content) +} + +func (r *BroadcastProcessor) isInOrder(method string) bool { + // the implementer has not specified an execution order + if r.executionOrder == nil || len(r.executionOrder) <= 0 { + return true + } + order, ok := r.executionOrder[method] + // accept all methods without a specified order + if !ok { + return true + } + // the first method should always be allowed to be executed + if r.executionOrder[method] <= 0 { + return true + } + return order <= r.orderIndex +} + +func (r *BroadcastProcessor) addToOutOfOrder(msg *Content) { + // the implementer has not specified an execution order + if r.executionOrder == nil || len(r.executionOrder) <= 0 { + return + } + var ( + msgs []*Content + ok bool + ) + if msgs, ok = r.outOfOrderMsgs[msg.CurrentMethod]; ok { + msgs = append(msgs, msg) + } else { + msgs = []*Content{msg} + } + r.outOfOrderMsgs[msg.CurrentMethod] = msgs +} + +func (r *BroadcastProcessor) updateOrder(method string, progressTo string) { + // the implementer has not specified an execution order + if r.executionOrder == nil || len(r.executionOrder) <= 0 { + return + } + if progressTo != "" { + method = progressTo + } + order, ok := r.executionOrder[method] + // do nothing for methods without specified order + if !ok { + return + } + if order > r.orderIndex { + r.orderIndex = order + } +} + +func (r *BroadcastProcessor) dispatchOutOfOrderMsgs() { + // the implementer has not specified an execution order + if r.executionOrder == nil || len(r.executionOrder) <= 0 { + return + } + // return early if there are no cached msgs + if len(r.outOfOrderMsgs) <= 0 { + return + } + handledMethods := make([]string, 0, len(r.outOfOrderMsgs)) + for method, msgs := range r.outOfOrderMsgs { + order, ok := r.executionOrder[method] + if !ok { + // this should not be possible unless the execution order + // is changed during operation, which is prohibited. + panic("how did you get here?") + } + if order <= r.orderIndex { + for _, msg := range msgs { + msg.Run(r.cancellationCtx, r.enqueueBroadcast) + r.log("msg: dispatching out of order msg", nil, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + } + handledMethods = append(handledMethods, method) + } + } + // cleanup after dispatching the cached messages + for _, m := range handledMethods { + delete(r.outOfOrderMsgs, m) + } +} + +func alreadyBroadcasted(methods []string, method string) bool { + for _, m := range methods { + if m == method { + return true + } + } + return false +} + +func (p *BroadcastProcessor) initialize(msg *Content, metadata *metadata) { + p.log("processor: started", nil, logging.Started(p.started)) + p.broadcastID = msg.BroadcastID + p.initOrder() + // connect to client immediately to potentially save some time + go p.router.Connect(metadata.OriginAddr) + if msg.IsBroadcastClient { + // important to set this option to prevent duplicate client reqs. + // this can be the result if a server forwards the req but the + // leader has already received the client req. + metadata.HasReceivedClientReq = true + go func() { + if msg.Ctx == nil { + return + } + // new.Ctx will correspond to the streamCtx between the client and this server. + // We can thus listen to it and signal a cancellation if the client goes offline + // or cancels the request. We also have to listen to the p.ctx to prevent leaking + // the goroutine. + select { + case <-p.ctx.Done(): + case <-msg.Ctx.Done(): + } + // when the processor returns it sets p.cancellationCtxCancel = nil. + // Hence, it is important to check if it is nil. Also, the processor + // calls this function when it returns. + if p.cancellationCtxCancel != nil { + p.cancellationCtxCancel() + } + }() + p.log("msg: received client req", nil, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + } + // all messages should always have a ReceiveChan + if msg.ReceiveChan != nil { + if !p.isInOrder(msg.CurrentMethod) { + // save the message and execute it later + p.addToOutOfOrder(msg) + msg.ReceiveChan <- shardResponse{ + err: OutOfOrderErr{}, + } + p.log("msg: out of order", OutOfOrderErr{}, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + } else { + msg.ReceiveChan <- shardResponse{ + err: nil, + reqCtx: p.cancellationCtx, + enqueueBroadcast: p.enqueueBroadcast, + } + p.log("msg: processed", nil, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + } + } +} + +func (p *BroadcastProcessor) cleanup(metadata *metadata) { + p.ended = time.Now() + p.cancelFunc() + p.cancellationCtxCancel() + // make sure the context is cancelled before closing the channels + <-p.ctx.Done() + p.emptyChannels(metadata) + // mark allocations ready for GC and minimize memory footprint of finished broadcast requests. + // this is safe to do because all send operations listen to the cancelled p.ctx thus preventing + // deadlocks/goroutine leaks. + p.outOfOrderMsgs = nil + p.executionOrder = nil + /* + These two lines would greatly impact the amount of memory used by a processor: + 1. p.broadcastChan = nil + 2. p.sendChan = nil + + However, the shard uses them and thus we get a race condition when setting them to nil. + Using mutexes will remove the benefit of setting them to nil. It is safe to set them to + nil because sending on a nil channel will block forever. We always listen to ctx.Done(), + ensuring that we don't get deadlocks. Additionally, queued msgs will by dropped in any + case and new msgs will not even get enqueued. As a concluding remark, we comment the lines + out to not get hits when running tests with the race detector. + */ + p.log("processor: stopped", nil, logging.Started(p.started), logging.Ended(p.ended)) +} + +// this method is used to enqueue messages onto the broadcast channel +// of a broadcast processor. The messages enqueued are then transmitted +// to the other servers or the client depending on the type of message. +// Currently there are three types: +// - BroadcastMsg +// - ClientReply +// - Cancellation +func (p *BroadcastProcessor) enqueueBroadcast(msg *Msg) error { + // we want to prevent queueing messages on the buffered broadcastChan + // because it can potentially lead to concurrency bugs. These include: + // - buffering a message on the channel and requiring that it is processed. + // this can happen with cancellation when SendToClient() is called first. + // - reaching the end of the buffer (same as not buffering the channel) and + // closing the broadcastChan at the same time. This will cause an error. + select { + case <-p.ctx.Done(): + p.log("broadcast: already processed", AlreadyProcessedErr{}, logging.Method(msg.Method), logging.MsgType(msg.MsgType.String())) + return AlreadyProcessedErr{} + default: + } + // this is not an optimal solution regarding cancellations. The cancellation + // msg can be discarded if the buffer is fully populated. This is because + // ctx.Done() will be called before the msg is queued. + select { + case <-p.ctx.Done(): + p.log("broadcast: already processed", AlreadyProcessedErr{}, logging.Method(msg.Method), logging.MsgType(msg.MsgType.String())) + return AlreadyProcessedErr{} + case p.broadcastChan <- msg: + return nil + } +} diff --git a/broadcast/processor_test.go b/broadcast/processor_test.go new file mode 100644 index 00000000..d38e7cc1 --- /dev/null +++ b/broadcast/processor_test.go @@ -0,0 +1,324 @@ +package broadcast + +import ( + "context" + "fmt" + "testing" + "time" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type mockResp struct{} + +func (mockResp) ProtoReflect() protoreflect.Message { + return nil +} + +type mockRouter struct { + returnError bool + reqType string + resp protoreflect.ProtoMessage +} + +func (r *mockRouter) Send(broadcastID uint64, addr, method string, originDigest, originSignature []byte, originPubKey string, req msg) error { + switch val := req.(type) { + case *broadcastMsg: + r.reqType = "Broadcast" + case *reply: + r.reqType = "SendToClient" + r.resp = val.Response + } + if r.returnError { + return fmt.Errorf("router: send error") + } + return nil +} + +func (r *mockRouter) Connect(addr string) {} + +func TestHandleBroadcastOption1(t *testing.T) { + snowflake := NewSnowflake(0) + broadcastID := snowflake.NewBroadcastID() + + var tests = []struct { + in *Content + out error + }{ + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse), + }, + out: nil, + }, + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: snowflake.NewBroadcastID(), + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse), + }, + out: BroadcastIDErr{}, + }, + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse), + }, + out: nil, + }, + } + + msg := &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + OriginMethod: "testMethod", + ReceiveChan: make(chan shardResponse), + } + + router := &mockRouter{ + returnError: false, + } + + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), 1*time.Minute) + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + defer cancel() + defer cancelCancel() + req := &BroadcastProcessor{ + ctx: ctx, + cancelFunc: cancel, + sendChan: make(chan *Content), + broadcastChan: make(chan *Msg, 5), + started: time.Now(), + cancellationCtx: cancelCtx, + cancellationCtxCancel: cancelCancel, + router: router, + } + go req.run(msg) + resp := <-msg.ReceiveChan + if resp.err != nil { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", nil, resp.err) + } + + for _, tt := range tests { + req.sendChan <- tt.in + resp := <-tt.in.ReceiveChan + if resp.err != tt.out { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", tt.out, resp.err) + } + } + + select { + case <-time.After(100 * time.Millisecond): + case <-req.ctx.Done(): + t.Fatalf("the request is not done yet. SendToClient has not been called.") + } + + req.broadcastChan <- &Msg{ + MsgType: ReplyMsg, + Reply: &reply{ + Response: mockResp{}, + Err: nil, + }, + BroadcastID: broadcastID, + } + + select { + case <-time.After(100 * time.Millisecond): + case <-req.ctx.Done(): + t.Fatalf("the request is not done yet. SendToClient has been called, but the client request has not arrived yet.") + } + + clientMsg := &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: true, + SendFn: func(resp protoreflect.ProtoMessage, err error) error { return nil }, + ReceiveChan: make(chan shardResponse), + } + req.sendChan <- clientMsg + resp = <-clientMsg.ReceiveChan + expectedErr := AlreadyProcessedErr{} + if resp.err != expectedErr { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", resp.err, expectedErr) + } + + select { + case <-time.After(3 * time.Second): + t.Fatalf("the request should have been stopped. Both SendToClient has been called and the client request has arrived.") + case <-req.ctx.Done(): + } +} + +func TestHandleBroadcastCall1(t *testing.T) { + snowflake := NewSnowflake(0) + broadcastID := snowflake.NewBroadcastID() + + var tests = []struct { + in *Content + out error + }{ + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse, 1), + }, + out: nil, + }, + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: snowflake.NewBroadcastID(), + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse, 1), + }, + out: BroadcastIDErr{}, + }, + { + in: &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse, 1), + }, + out: nil, + }, + } + + msg := &Content{ + Ctx: context.Background(), + BroadcastID: broadcastID, + IsBroadcastClient: false, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + ReceiveChan: make(chan shardResponse), + } + + router := &mockRouter{ + returnError: false, + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), 1*time.Minute) + defer cancel() + defer cancelCancel() + req := &BroadcastProcessor{ + ctx: ctx, + cancelFunc: cancel, + sendChan: make(chan *Content), + broadcastChan: make(chan *Msg, 5), + started: time.Now(), + cancellationCtx: cancelCtx, + cancellationCtxCancel: cancelCancel, + router: router, + } + go req.run(msg) + resp := <-msg.ReceiveChan + if resp.err != nil { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", nil, resp.err) + } + + for _, tt := range tests { + req.sendChan <- tt.in + resp := <-tt.in.ReceiveChan + if resp.err != tt.out { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", tt.out, resp.err) + } + } + + select { + case <-time.After(100 * time.Millisecond): + case <-req.ctx.Done(): + t.Fatalf("the request is not done yet. SendToClient has not been called.") + } + + req.broadcastChan <- &Msg{ + MsgType: ReplyMsg, + Reply: &reply{ + Response: mockResp{}, + Err: nil, + }, + BroadcastID: broadcastID, + } + + select { + case <-time.After(1 * time.Second): + t.Fatalf("the request is done. SendToClient has been called and this is a BroadcastCall, meaning it should respond regardless of the client request.") + case <-req.ctx.Done(): + } + + clientMsg := &Content{ + BroadcastID: broadcastID, + IsBroadcastClient: true, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + ReceiveChan: make(chan shardResponse), + } + select { + case <-req.ctx.Done(): + case req.sendChan <- clientMsg: + t.Fatalf("the request is done. SendToClient has been called so this message should be dropped.") + } +} + +func BenchmarkHandleProcessor(b *testing.B) { + snowflake := NewSnowflake(0) + originMethod := "testMethod" + router := &mockRouter{ + returnError: false, + } + // not important to use unique broadcastID because we are + // not using shards in this test + broadcastID := snowflake.NewBroadcastID() + resp := Msg{ + MsgType: ReplyMsg, + Reply: &reply{ + Response: mockResp{}, + Err: nil, + }, + BroadcastID: broadcastID, + } + sendFn := func(resp protoreflect.ProtoMessage, err error) error { return nil } + ctx := context.Background() + + b.ResetTimer() + b.Run("ProcessorHandler", func(b *testing.B) { + for i := 0; i < b.N; i++ { + msg := &Content{ + BroadcastID: broadcastID, + IsBroadcastClient: true, + SendFn: sendFn, + OriginMethod: originMethod, + ReceiveChan: nil, + Ctx: ctx, + } + + cancelCtx, cancelCancel := context.WithTimeout(ctx, 1*time.Minute) + ctx, cancel := context.WithTimeout(ctx, 1*time.Minute) + proc := &BroadcastProcessor{ + ctx: ctx, + cancelFunc: cancel, + cancellationCtx: cancelCtx, + cancellationCtxCancel: cancelCancel, + sendChan: make(chan *Content), + broadcastChan: make(chan *Msg, 5), + started: time.Now(), + router: router, + } + go proc.run(msg) + + proc.broadcastChan <- &resp + + <-proc.ctx.Done() + cancel() + cancelCancel() + } + }) +} diff --git a/broadcast/router.go b/broadcast/router.go new file mode 100644 index 00000000..420bbfd7 --- /dev/null +++ b/broadcast/router.go @@ -0,0 +1,234 @@ +package broadcast + +import ( + "context" + "errors" + "log/slog" + "sync" + "time" + + "github.com/relab/gorums/logging" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/protobuf/reflect/protoreflect" +) + +type Client struct { + Addr string + SendMsg func(broadcastID uint64, method string, msg protoreflect.ProtoMessage, timeout time.Duration, originDigest, originSignature []byte, originPubKey string) error + Close func() error +} + +type Router interface { + Send(broadcastID uint64, addr, method string, originDigest, originSignature []byte, originPubKey string, req msg) error + Connect(addr string) +} + +type BroadcastRouter struct { + mut sync.RWMutex + id uint32 + addr string + serverHandlers map[string]ServerHandler // handlers on other servers + clientHandlers map[string]struct{} // specifies what handlers a client has implemented. Used only for BroadcastCalls. + createClient func(addr string, dialOpts []grpc.DialOption) (*Client, error) + canceler func(broadcastID uint64, srvAddrs []string) + dialOpts []grpc.DialOption + dialTimeout time.Duration + logger *slog.Logger + state *BroadcastState + allowList map[string]string // whitelist of (address, pubKey) pairs the server can reply to +} + +func NewRouter(logger *slog.Logger, createClient func(addr string, dialOpts []grpc.DialOption) (*Client, error), canceler func(broadcastID uint64, srvAddrs []string), dialTimeout time.Duration, allowList map[string]string, dialOpts ...grpc.DialOption) *BroadcastRouter { + if len(dialOpts) <= 0 { + dialOpts = []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + } + } + return &BroadcastRouter{ + serverHandlers: make(map[string]ServerHandler), + clientHandlers: make(map[string]struct{}), + createClient: createClient, + canceler: canceler, + dialOpts: dialOpts, + dialTimeout: dialTimeout, + logger: logger, + allowList: allowList, + } +} + +func (r *BroadcastRouter) registerState(state *BroadcastState) { + r.state = state +} + +type msg interface{} + +func (r *BroadcastRouter) Send(broadcastID uint64, addr, method string, originDigest, originSignature []byte, originPubKey string, req msg) error { + if r.addr == "" { + panic("The listen addr on the broadcast server cannot be empty. Use the WithListenAddr() option when creating the server.") + } + switch val := req.(type) { + case *broadcastMsg: + return r.routeBroadcast(broadcastID, addr, method, val, originDigest, originSignature, originPubKey) + case *reply: + return r.routeClientReply(broadcastID, addr, method, val, originDigest, originSignature, originPubKey) + case *cancellation: + r.canceler(broadcastID, val.srvAddrs) + return nil + } + err := errors.New("wrong req type") + r.log("router: malformed msg", err, logging.BroadcastID(broadcastID)) + return err +} + +func (r *BroadcastRouter) Connect(addr string) { + _, _ = r.getClient(addr) +} + +func (r *BroadcastRouter) routeBroadcast(broadcastID uint64, addr, method string, msg *broadcastMsg, originDigest, originSignature []byte, originPubKey string) error { + if handler, ok := r.serverHandlers[msg.method]; ok { + // it runs an interceptor prior to broadcastCall, hence a different signature. + // see: (srv *broadcastServer) registerBroadcastFunc(method string). + handler(msg.ctx, msg.request, broadcastID, addr, method, msg.options, r.id, r.addr, originDigest, originSignature, originPubKey) + return nil + } + err := errors.New("handler not found") + r.log("router (broadcast): could not find handler", err, logging.BroadcastID(broadcastID), logging.NodeAddr(addr), logging.Method(method)) + return err +} + +func (r *BroadcastRouter) routeClientReply(broadcastID uint64, addr, method string, resp *reply, originDigest, originSignature []byte, originPubKey string) error { + // the client has initiated a broadcast call and the reply should be sent as an RPC + if _, ok := r.clientHandlers[method]; ok && addr != "" { + client, err := r.getClient(addr) + if err != nil { + r.log("router (reply): could not get client", err, logging.BroadcastID(broadcastID), logging.NodeAddr(addr), logging.Method(method)) + return err + } + err = client.SendMsg(broadcastID, method, resp.getResponse(), r.dialTimeout, originDigest, originSignature, originPubKey) + r.log("router (reply): sending reply to client", err, logging.BroadcastID(broadcastID), logging.NodeAddr(addr), logging.Method(method)) + return err + } + // the server can receive a broadcast from another server before a client sends a direct message. + // it should thus wait for a potential message from the client. otherwise, it should be removed. + err := errors.New("not routed") + r.log("router (reply): could not find handler", err, logging.BroadcastID(broadcastID), logging.NodeAddr(addr), logging.Method(method)) + return err +} + +func (r *BroadcastRouter) validAddr(addr string) bool { + if addr == "" { + return false + } + if addr == ServerOriginAddr { + return false + } + if r.allowList != nil { + _, ok := r.allowList[addr] + return ok + } + return true +} + +func (r *BroadcastRouter) getClient(addr string) (*Client, error) { + if !r.validAddr(addr) { + return nil, InvalidAddrErr{addr: addr} + } + // fast path: + // read lock because it is likely that we will send many + // messages to the same client. + r.mut.RLock() + if client, ok := r.state.getClient(addr); ok { + r.mut.RUnlock() + return client, nil + } + r.mut.RUnlock() + // slow path: + // we need a write lock when adding a new client. This only process + // one at a time and is thus necessary to check if the client has + // already been added again. Otherwise, we can end up creating multiple + // clients. + r.mut.Lock() + defer r.mut.Unlock() + if client, ok := r.state.getClient(addr); ok { + return client, nil + } + client, err := r.createClient(addr, r.dialOpts) + if err != nil { + return nil, err + } + r.state.addClient(addr, client) + return client, nil +} + +func (r *BroadcastRouter) log(msg string, err error, args ...slog.Attr) { + if r.logger != nil { + args = append(args, logging.Err(err), logging.Type("router")) + level := slog.LevelInfo + if err != nil { + level = slog.LevelError + } + r.logger.LogAttrs(context.Background(), level, msg, args...) + } +} + +type msgType int + +const ( + BroadcastMsg msgType = iota + ReplyMsg + CancellationMsg +) + +func (m msgType) String() string { + switch m { + case BroadcastMsg: + return "BroadcastMsg" + case ReplyMsg: + return "ReplyMsg" + case CancellationMsg: + return "CancellationMsg" + } + return "unkown" +} + +type Msg struct { + MsgType msgType + BroadcastID uint64 + Msg *broadcastMsg + Method string + Reply *reply + Cancellation *cancellation +} + +type broadcastMsg struct { + request protoreflect.ProtoMessage + method string + broadcastID uint64 + options BroadcastOptions + ctx context.Context +} + +func NewMsg(broadcastID uint64, req protoreflect.ProtoMessage, method string, options BroadcastOptions) *broadcastMsg { + return &broadcastMsg{ + request: req, + method: method, + broadcastID: broadcastID, + options: options, + ctx: context.Background(), + } +} + +type reply struct { + Response protoreflect.ProtoMessage + Err error +} + +func (r *reply) getResponse() protoreflect.ProtoMessage { + return r.Response +} + +type cancellation struct { + srvAddrs []string + end bool // end is used to stop the request. +} diff --git a/broadcast/shard.go b/broadcast/shard.go new file mode 100644 index 00000000..e888c6df --- /dev/null +++ b/broadcast/shard.go @@ -0,0 +1,211 @@ +package broadcast + +import ( + "context" + "log/slog" + "sync" + "time" + + "github.com/relab/gorums/logging" +) + +const NumShards = 16 + +type shard struct { + mut sync.RWMutex + id int + parentCtx context.Context + procs map[uint64]*BroadcastProcessor + reqTTL time.Duration + router Router + nextGC time.Time + shardBuffer int + sendBuffer int + logger *slog.Logger + + preserveOrdering bool + order map[string]int +} + +func createShards(ctx context.Context, shardBuffer, sendBuffer int, router Router, order map[string]int, reqTTL time.Duration, logger *slog.Logger) []*shard { + shards := make([]*shard, NumShards) + for i := range shards { + shards[i] = &shard{ + id: i, + parentCtx: ctx, + procs: make(map[uint64]*BroadcastProcessor, shardBuffer), + shardBuffer: shardBuffer, + sendBuffer: sendBuffer, + reqTTL: reqTTL, + router: router, + preserveOrdering: order != nil, + order: order, + logger: logger, + } + } + return shards +} + +func (s *shard) handleMsg(msg *Content) shardResponse { + // Optimization: first check with a read lock if the processor already exists + if p, ok := s.getProcessor(msg.BroadcastID); ok { + return s.process(p, msg) + } else { + if msg.IsCancellation { + // ignore cancellations if a broadcast request + // has not been created yet + return shardResponse{ + err: MissingReqErr{}, + } + } + proc, alreadyExists := s.addProcessor(msg) + if alreadyExists { + return s.process(proc, msg) + } + select { + case resp := <-msg.ReceiveChan: + return resp + case <-proc.ctx.Done(): + return shardResponse{ + err: ReqFinishedErr{}, + } + } + } +} + +func (s *shard) process(p *BroadcastProcessor, msg *Content) shardResponse { + // must check if the req is done first to prevent + // unecessarily running the server handler. + select { + case <-p.ctx.Done(): + p.log("msg: already processed", AlreadyProcessedErr{}, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + return shardResponse{ + err: AlreadyProcessedErr{}, + } + default: + } + if msg.IsCancellation { + p.cancellationCtxCancel() + return shardResponse{ + err: nil, + } + } + if !msg.IsBroadcastClient && !s.preserveOrdering { + p.log("msg: processed", nil, logging.Method(msg.CurrentMethod), logging.From(msg.SenderAddr)) + // no need to send it to the broadcast request goroutine. + // the first request should contain all info needed + // except for the routing info given in the client req. + return shardResponse{ + err: nil, + reqCtx: p.cancellationCtx, + enqueueBroadcast: p.enqueueBroadcast, + } + } + // must check if the req is done to prevent deadlock + select { + case <-p.ctx.Done(): + return shardResponse{ + err: AlreadyProcessedErr{}, + } + case p.sendChan <- msg: + select { + case resp := <-msg.ReceiveChan: + return resp + case <-p.ctx.Done(): + return shardResponse{ + err: ReqFinishedErr{}, + } + } + } + +} + +func (s *shard) handleBMsg(msg *Msg) { + if req, ok := s.getProcessor(msg.BroadcastID); ok { + select { + case <-req.ctx.Done(): + case req.broadcastChan <- msg: + } + } +} + +func (s *shard) getProcessor(broadcastID uint64) (*BroadcastProcessor, bool) { + s.mut.RLock() + defer s.mut.RUnlock() + p, ok := s.procs[broadcastID] + return p, ok +} + +func (s *shard) addProcessor(msg *Content) (*BroadcastProcessor, bool) { + s.mut.Lock() + defer s.mut.Unlock() + if p, ok := s.procs[msg.BroadcastID]; ok { + return p, true + } + if time.Since(s.nextGC) > 0 { + // make sure the current request is done before running the GC. + // This is to prevent running the GC in vain. + t := s.reqTTL + 5*time.Second + s.nextGC = time.Now().Add(t) + go s.gc(t) + } + if msg.IsBroadcastClient { + // msg.Ctx will correspond to the streamCtx between the client and this server, + // meaning the ctx will cancel when the client cancels or disconnects. + msg.Ctx, msg.CancelCtx = context.WithCancel(msg.Ctx) + } else { + msg.Ctx, msg.CancelCtx = context.WithCancel(context.Background()) + } + // check size of s.reqs. If too big, then perform necessary cleanup. + // should only affect the current shard and not the others. + ctx, cancel := context.WithTimeout(s.parentCtx, s.reqTTL) + var logger *slog.Logger + if s.logger != nil { + logger = s.logger.With(logging.BroadcastID(msg.BroadcastID)) + } + proc := &BroadcastProcessor{ + ctx: ctx, + cancelFunc: cancel, + sendChan: make(chan *Content, s.sendBuffer), + broadcastChan: make(chan *Msg, s.sendBuffer), + started: time.Now(), + router: s.router, + cancellationCtx: msg.Ctx, + cancellationCtxCancel: msg.CancelCtx, + executionOrder: s.order, + logger: logger, + } + s.procs[msg.BroadcastID] = proc + go proc.run(msg) + return proc, false +} + +func (s *shard) gc(nextGC time.Duration) { + // make sure there is overlap between GC's + time.Sleep(nextGC + 1*time.Second) + s.mut.Lock() + defer s.mut.Unlock() + newReqs := make(map[uint64]*BroadcastProcessor, s.shardBuffer) + for broadcastID, req := range s.procs { + // stale requests should be cancelled and removed immediately + if time.Since(req.started) > s.reqTTL { + req.cancelFunc() + continue + } + select { + case <-req.ctx.Done(): + // if the request has finished early then it has + // probably executed successfully on all servers. + // we can thus assume it is safe to remove the req + // after a short period after it has finished because + // it will likely not receive any msg related to this + // broadcast request. + if time.Since(req.ended) > s.reqTTL/5 { + continue + } + default: + } + newReqs[broadcastID] = req + } + s.procs = newReqs +} diff --git a/broadcast/shard_test.go b/broadcast/shard_test.go new file mode 100644 index 00000000..f2fa0525 --- /dev/null +++ b/broadcast/shard_test.go @@ -0,0 +1,135 @@ +package broadcast + +import ( + "context" + "errors" + "fmt" + "testing" + "time" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type slowRouter struct { + returnError bool + reqType string + resp protoreflect.ProtoMessage +} + +func (r *slowRouter) Send(broadcastID uint64, addr, method string, originDigest, originSignature []byte, originPubKey string, req msg) error { + time.Sleep(1 * time.Second) + switch val := req.(type) { + case *broadcastMsg: + r.reqType = "Broadcast" + case *reply: + r.reqType = "SendToClient" + r.resp = val.Response + } + if r.returnError { + return fmt.Errorf("router: send error") + } + return nil +} + +func (r *slowRouter) Connect(addr string) {} + +func TestShard(t *testing.T) { + snowflake := NewSnowflake(0) + broadcastID := snowflake.NewBroadcastID() + router := &slowRouter{ + returnError: false, + } + shardBuffer := 100 + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + shard := &shard{ + id: 0, + parentCtx: ctx, + procs: make(map[uint64]*BroadcastProcessor, shardBuffer), + router: router, + reqTTL: 5 * time.Minute, + } + + var tests = []struct { + in *Content + out error + }{ + { + in: &Content{ + BroadcastID: broadcastID, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + IsBroadcastClient: true, + ReceiveChan: make(chan shardResponse), + Ctx: context.Background(), + }, + out: nil, + }, + { + in: &Content{ + BroadcastID: broadcastID, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse), + Ctx: context.Background(), + }, + out: nil, + }, + { + in: &Content{ + BroadcastID: broadcastID, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + IsBroadcastClient: false, + ReceiveChan: make(chan shardResponse), + Ctx: context.Background(), + }, + out: nil, + }, + } + + for _, tt := range tests { + resp := shard.handleMsg(tt.in) + if resp.err != tt.out { + t.Fatalf("wrong error returned.\n\tgot: %v, want: %v", tt.out, resp.err) + } + } + + shard.handleBMsg(&Msg{ + MsgType: ReplyMsg, + Reply: &reply{ + Response: mockResp{}, + Err: nil, + }, + BroadcastID: broadcastID, + }) + + clientMsg := &Content{ + BroadcastID: broadcastID, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + IsBroadcastClient: true, + ReceiveChan: make(chan shardResponse, 1), + Ctx: context.Background(), + } + resp := shard.handleMsg(clientMsg) + if !errors.Is(resp.err, AlreadyProcessedErr{}) { + t.Fatalf("the request should have been stopped. SendToClient has been called.") + } + + // wait for the request to finish + msgShouldBeDropped := &Content{ + BroadcastID: broadcastID, + OriginAddr: "127.0.0.1:8080", + OriginMethod: "testMethod", + IsBroadcastClient: true, + ReceiveChan: make(chan shardResponse, 1), + Ctx: context.Background(), + } + // this will panic if the request sendChan is closed + resp = shard.handleMsg(msgShouldBeDropped) + if !errors.Is(resp.err, AlreadyProcessedErr{}) { + t.Fatalf("the request should have been stopped. SendToClient has been called.") + } +} diff --git a/broadcast/snowflake.go b/broadcast/snowflake.go new file mode 100644 index 00000000..d3665f0e --- /dev/null +++ b/broadcast/snowflake.go @@ -0,0 +1,79 @@ +package broadcast + +import ( + "math/rand" + "sync" + "time" +) + +type Snowflake struct { + mut sync.Mutex + MachineID uint64 + SequenceNum uint64 + lastT uint64 + lastS uint64 + epoch time.Time +} + +const ( + MaxMachineID = uint16(1 << 12) + maxShard = uint8(1 << 4) + maxSequenceNum = uint32(1 << 18) + bitMaskTimestamp = uint64((1<<30)-1) << 34 + bitMaskShardID = uint64((1<<4)-1) << 30 + bitMaskMachineID = uint64((1<<12)-1) << 18 + bitMaskSequenceNum = uint64((1 << 18) - 1) + epoch = "2024-01-01T00:00:00" +) + +func Epoch() time.Time { + timestamp, _ := time.Parse("2006-01-02T15:04:05", epoch) + return timestamp +} + +func NewSnowflake(id uint64) *Snowflake { + if id >= uint64(MaxMachineID) { + id = uint64(rand.Int31n(int32(MaxMachineID))) + } + return &Snowflake{ + MachineID: id, + SequenceNum: 0, + epoch: Epoch(), + } +} + +func (s *Snowflake) NewBroadcastID() uint64 { + // timestamp: 30 bit -> seconds since 01.01.2024 + // shardID: 4 bit -> 16 different shards + // machineID: 12 bit -> 4096 clients + // sequenceNum: 18 bit -> 262 144 messages +start: + s.mut.Lock() + timestamp := uint64(time.Since(s.epoch).Seconds()) + l := (s.SequenceNum + 1) % uint64(maxSequenceNum) + if timestamp-s.lastT <= 0 && l == s.lastS { + s.mut.Unlock() + time.Sleep(10 * time.Millisecond) + goto start + } + if timestamp > s.lastT { + s.lastT = timestamp + s.lastS = l + } + s.SequenceNum = l + s.mut.Unlock() + + t := (timestamp << 34) & bitMaskTimestamp + shard := (uint64(rand.Int31n(int32(maxShard))) << 30) & bitMaskShardID + m := uint64(s.MachineID<<18) & bitMaskMachineID + n := l & bitMaskSequenceNum + return t | shard | m | n +} + +func DecodeBroadcastID(broadcastID uint64) (timestamp uint32, shardID uint16, machineID uint16, sequenceNo uint32) { + t := (broadcastID & bitMaskTimestamp) >> 34 + shard := (broadcastID & bitMaskShardID) >> 30 + m := (broadcastID & bitMaskMachineID) >> 18 + n := (broadcastID & bitMaskSequenceNum) + return uint32(t), uint16(shard), uint16(m), uint32(n) +} diff --git a/broadcast/snowflake_test.go b/broadcast/snowflake_test.go new file mode 100644 index 00000000..b39dadc4 --- /dev/null +++ b/broadcast/snowflake_test.go @@ -0,0 +1,48 @@ +package broadcast + +import ( + "testing" +) + +func TestBroadcastID(t *testing.T) { + if MaxMachineID != 4096 { + t.Errorf("maxMachineID is hardcoded in test. want: %v, got: %v", 4096, MaxMachineID) + } + if maxSequenceNum != 262144 { + t.Errorf("maxSequenceNum is hardcoded in test. want: %v, got: %v", 262144, maxSequenceNum) + } + if maxShard != 16 { + t.Errorf("maxShard is hardcoded in test. want: %v, got: %v", 16, maxShard) + } + // intentionally provide an illegal machineID. A random machineID should be given instead. + snowflake := NewSnowflake(8000) + machineID := snowflake.MachineID + timestampDistribution := make(map[uint32]int) + maxN := 262144 // = 2^18 + for j := 1; j < 3*maxN; j++ { + i := j % maxN + broadcastID := snowflake.NewBroadcastID() + timestamp, shard, m, n := DecodeBroadcastID(broadcastID) + if i != int(n) { + t.Errorf("wrong sequence number. want: %v, got: %v", i, n) + } + if m >= 4096 { + t.Errorf("machine ID cannot be higher than max. want: %v, got: %v", 4095, m) + } + if m != uint16(machineID) { + t.Errorf("wrong machine ID. want: %v, got: %v", machineID, m) + } + if shard >= 16 { + t.Errorf("cannot have higher shard than max. want: %v, got: %v", 15, shard) + } + if n >= uint32(maxN) { + t.Errorf("sequence number cannot be higher than max. want: %v, got: %v", maxN, n) + } + timestampDistribution[timestamp]++ + } + for k, v := range timestampDistribution { + if v > maxN { + t.Errorf("cannot have more than maxN in a second. want: %v, got: %v", maxN, k) + } + } +} diff --git a/broadcast/state.go b/broadcast/state.go new file mode 100644 index 00000000..f9c98146 --- /dev/null +++ b/broadcast/state.go @@ -0,0 +1,122 @@ +package broadcast + +import ( + "context" + "log/slog" + "sync" + "time" + + "google.golang.org/protobuf/reflect/protoreflect" +) + +type BroadcastState struct { + mut sync.Mutex + shardMut sync.RWMutex // RW because we often read and very seldom write to the state + parentCtx context.Context + parentCtxCancelFunc context.CancelFunc + logger *slog.Logger + reqTTL time.Duration + sendBuffer int + shardBuffer int + snowflake *Snowflake + clients map[string]*Client + router Router + order map[string]int + + shards []*shard +} + +func NewState(logger *slog.Logger, router Router, order map[string]int, reqTTL time.Duration, shardBuffer, sendBuffer int) *BroadcastState { + ctx, cancel := context.WithCancel(context.Background()) + shards := createShards(ctx, shardBuffer, sendBuffer, router, order, reqTTL, logger) + state := &BroadcastState{ + parentCtx: ctx, + parentCtxCancelFunc: cancel, + shards: shards, + logger: logger, + reqTTL: reqTTL, + sendBuffer: sendBuffer, + shardBuffer: shardBuffer, + router: router, + order: order, + clients: make(map[string]*Client), + } + return state +} + +func (s *BroadcastState) Close() error { + s.mut.Lock() + defer s.mut.Unlock() + if s.logger != nil { + s.logger.Debug("broadcast: closing state") + } + s.parentCtxCancelFunc() + var err error + for _, client := range s.clients { + clientErr := client.Close() + if clientErr != nil { + err = clientErr + } + } + return err +} + +func (s *BroadcastState) reset() { + s.parentCtxCancelFunc() + s.mut.Lock() + s.parentCtx, s.parentCtxCancelFunc = context.WithCancel(context.Background()) + for _, client := range s.clients { + _ = client.Close() + } + s.clients = make(map[string]*Client) + shards := createShards(s.parentCtx, s.shardBuffer, s.sendBuffer, s.router, s.order, s.reqTTL, s.logger) + s.mut.Unlock() + // unlocking because we don't want to end up with a deadlock. + s.shardMut.Lock() + s.shards = shards + s.shardMut.Unlock() +} + +func (s *BroadcastState) getClient(addr string) (*Client, bool) { + s.mut.Lock() + defer s.mut.Unlock() + client, ok := s.clients[addr] + return client, ok +} + +func (s *BroadcastState) addClient(addr string, client *Client) { + s.mut.Lock() + defer s.mut.Unlock() + s.clients[addr] = client +} + +func (s *BroadcastState) getShard(i uint16) *shard { + s.shardMut.RLock() + defer s.shardMut.RUnlock() + return s.shards[i] +} + +type shardResponse struct { + err error + reqCtx context.Context + enqueueBroadcast func(*Msg) error +} + +type Content struct { + BroadcastID uint64 + IsBroadcastClient bool + IsCancellation bool + OriginAddr string + OriginMethod string + OriginPubKey string + OriginSignature []byte + OriginDigest []byte + ViewNumber uint64 + SenderAddr string + CurrentMethod string + ReceiveChan chan shardResponse + SendFn func(resp protoreflect.ProtoMessage, err error) error + Ctx context.Context + CancelCtx context.CancelFunc + Run func(context.Context, func(*Msg) error) +} diff --git a/broadcastcall.go b/broadcastcall.go new file mode 100644 index 00000000..364d8420 --- /dev/null +++ b/broadcastcall.go @@ -0,0 +1,107 @@ +package gorums + +import ( + "context" + + "github.com/relab/gorums/ordering" + "google.golang.org/protobuf/reflect/protoreflect" +) + +// BroadcastCallData holds the message, destination nodes, method identifier, +// and other information necessary to perform the various quorum call types +// supported by Gorums. +type BroadcastCallData struct { + Message protoreflect.ProtoMessage + Method string + BroadcastID uint64 // a unique identifier for the current broadcast request + IsBroadcastClient bool + SenderAddr string + OriginAddr string + OriginMethod string + OriginPubKey string + OriginSignature []byte + OriginDigest []byte + ServerAddresses []string + SkipSelf bool +} + +// checks whether the given address is contained in the given subset +// of server addresses. Will return true if a subset is not given. +func (bcd *BroadcastCallData) inSubset(addr string) bool { + if bcd.ServerAddresses == nil || len(bcd.ServerAddresses) <= 0 { + return true + } + for _, srvAddr := range bcd.ServerAddresses { + if addr == srvAddr { + return true + } + } + return false +} + +// BroadcastCall performs a broadcast on the configuration. +// +// This method should be used by generated code only. +func (c RawConfiguration) BroadcastCall(ctx context.Context, d BroadcastCallData, opts ...CallOption) { + md := &ordering.Metadata{MessageID: c.getMsgID(), Method: d.Method, BroadcastMsg: &ordering.BroadcastMsg{ + IsBroadcastClient: d.IsBroadcastClient, + BroadcastID: d.BroadcastID, + SenderAddr: d.SenderAddr, + OriginAddr: d.OriginAddr, + OriginMethod: d.OriginMethod, + OriginPubKey: d.OriginPubKey, + OriginSignature: d.OriginSignature, + OriginDigest: d.OriginDigest, + }} + msg := &Message{Metadata: md, Message: d.Message} + o := getCallOptions(E_Broadcast, opts) + c.sign(msg, o.signOrigin) + + var replyChan chan response + if !o.noSendWaiting { + replyChan = make(chan response, len(c)) + } + sentMsgs := 0 + notEnqueued := make([]*RawNode, 0, len(c)) + for _, n := range c { + if d.SkipSelf && n.Address() == d.SenderAddr { + // the node will not send the message to itself + continue + } + // skip nodes not specified in subset + if !d.inSubset(n.addr) { + continue + } + sentMsgs++ + // do NOT enqueue in a goroutine. This inhibits ordering constraints. + // the message will only be enqueued if the channel has enough capacity + // or if the receiver is ready. This prevents a slow node from limiting the + // enqueueing of messages to other nodes while still ensuring correct + // ordering of messages. + // + // NOTE: the slow path will be invoked even though we buffer the channel. Hence, + // the enqueueFast will provide a small performance benefit. + enqueued := n.channel.enqueueFast(request{ctx: ctx, msg: msg, opts: o}, replyChan, false) + if !enqueued { + notEnqueued = append(notEnqueued, n) + } + } + + // it is important to retry the enqueueing for slow nodes. the method + // will block until the message is enqueued. + // NOTE: enqueueFast() creates a responseRouter and thus it is not + // necessary to provide the replyChan to enqueueSlow(). + for _, n := range notEnqueued { + n.channel.enqueueSlow(request{ctx: ctx, msg: msg, opts: o}) + } + + // if noSendWaiting is set, we will not wait for confirmation from the channel before returning. + if o.noSendWaiting { + return + } + + // wait until all requests have been sent + for ; sentMsgs > 0; sentMsgs-- { + <-replyChan + } +} diff --git a/callopts.go b/callopts.go index 1ce5c79c..689f3c95 100644 --- a/callopts.go +++ b/callopts.go @@ -5,6 +5,7 @@ import "google.golang.org/protobuf/runtime/protoimpl" type callOptions struct { callType *protoimpl.ExtensionInfo noSendWaiting bool + signOrigin bool } // CallOption is a function that sets a value in the given callOptions struct @@ -25,3 +26,11 @@ func WithNoSendWaiting() CallOption { o.noSendWaiting = true } } + +// WithOriginAuthentication is a CallOption that makes BroadcastCall methods +// digitally sign messages. +func WithOriginAuthentication() CallOption { + return func(o *callOptions) { + o.signOrigin = true + } +} diff --git a/channel.go b/channel.go index c6243961..da5e25fb 100644 --- a/channel.go +++ b/channel.go @@ -3,12 +3,14 @@ package gorums import ( "context" "fmt" + "log/slog" "math" "math/rand" "sync" "sync/atomic" "time" + "github.com/relab/gorums/logging" "github.com/relab/gorums/ordering" "google.golang.org/grpc" "google.golang.org/grpc/backoff" @@ -20,9 +22,10 @@ import ( var streamDownErr = status.Error(codes.Unavailable, "stream is down") type request struct { - ctx context.Context - msg *Message - opts callOptions + ctx context.Context + msg *Message + opts callOptions + numFailed int } // waitForSend returns true if the WithNoSendWaiting call option is not set. @@ -30,6 +33,33 @@ func (req request) waitForSend() bool { return req.opts.callType != nil && !req.opts.noSendWaiting } +// relatedToBroadcast returns true if the request is related to a broadcast request. +func (req request) relatedToBroadcast() bool { + if req.msg.Metadata == nil { + return false + } + if req.msg.Metadata.BroadcastMsg == nil { + return false + } + if req.msg.Metadata.BroadcastMsg.BroadcastID <= 0 { + return false + } + return true +} + +// getLogArgs returns the args given to the structured logger. +func (req request) getLogArgs(c *channel) []slog.Attr { + // no need to do processing if logging is not enabled + if c.logger == nil { + return nil + } + args := []slog.Attr{logging.MsgID(req.msg.Metadata.MessageID), logging.Method(req.msg.Metadata.Method), logging.NumFailed(req.numFailed), logging.MaxRetries(c.maxSendRetries)} + if req.relatedToBroadcast() { + args = append(args, logging.BroadcastID(req.msg.Metadata.BroadcastMsg.BroadcastID)) + } + return args +} + type response struct { nid uint32 msg protoreflect.ProtoMessage @@ -52,6 +82,7 @@ type channel struct { gorumsClient ordering.GorumsClient gorumsStream ordering.Gorums_NodeStreamClient streamMut sync.RWMutex + dialMut sync.RWMutex streamBroken atomicFlag connEstablished atomicFlag parentCtx context.Context @@ -59,6 +90,9 @@ type channel struct { cancelStream context.CancelFunc responseRouters map[uint64]responseRouter responseMut sync.Mutex + maxSendRetries int // number of times we try to resend a failed msg + maxConnRetries int // number of times we try to reconnect to a node + logger *slog.Logger } // newChannel creates a new channel for the given node and starts the sending goroutine. @@ -68,6 +102,10 @@ type channel struct { // deadlock when invoking a call type, as the goroutine will // block on the sendQ until a connection has been established. func newChannel(n *RawNode) *channel { + var logger *slog.Logger + if n.mgr.logger != nil { + logger = n.mgr.logger.With(slog.Uint64("nodeID", uint64(n.ID())), slog.String("nodeAddr", n.Address())) + } c := &channel{ sendQ: make(chan request, n.mgr.opts.sendBuffer), backoffCfg: n.mgr.opts.backoff, @@ -75,6 +113,9 @@ func newChannel(n *RawNode) *channel { latency: -1 * time.Second, rand: rand.New(rand.NewSource(time.Now().UnixNano())), responseRouters: make(map[uint64]responseRouter), + maxSendRetries: n.mgr.opts.maxSendRetries, + maxConnRetries: n.mgr.opts.maxConnRetries, + logger: logger, } // parentCtx controls the channel and is used to shut it down c.parentCtx = n.newContext() @@ -117,6 +158,7 @@ func (c *channel) cancelPendingMsgs() { defer c.responseMut.Unlock() for msgID, router := range c.responseRouters { router.c <- response{nid: c.node.ID(), err: streamDownErr} + c.log("channel: cancelling pending msg", streamDownErr, slog.LevelError, logging.MsgID(msgID)) // delete the router if we are only expecting a single reply message if !router.streaming { delete(c.responseRouters, msgID) @@ -143,7 +185,36 @@ func (c *channel) enqueue(req request, responseChan chan<- response, streaming b c.responseMut.Unlock() } // either enqueue the request on the sendQ or respond - // with error if the node is closed + // with error if the node is closed. + select { + case <-c.parentCtx.Done(): + c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("channel closed")}) + return + case c.sendQ <- req: + } +} + +func (c *channel) enqueueFast(req request, responseChan chan<- response, streaming bool) bool { + if responseChan != nil { + c.responseMut.Lock() + c.responseRouters[req.msg.Metadata.MessageID] = responseRouter{responseChan, streaming} + c.responseMut.Unlock() + } + // only enqueue the request on the sendQ if it is available and + // the node is not closed. + select { + case <-c.parentCtx.Done(): + c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("channel closed")}) + case c.sendQ <- req: + default: + return false + } + return true +} + +func (c *channel) enqueueSlow(req request) { + // either enqueue the request on the sendQ or respond + // with error if the node is closed. select { case <-c.parentCtx.Done(): c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("channel closed")}) @@ -166,6 +237,11 @@ func (c *channel) sendMsg(req request) (err error) { // that is, waitForSend is true. Conversely, if the call option is set, the call type // will not block on the response channel, and the "receiver" goroutine below will // eventually clean up the responseRouter map by calling routeResponse. + // + // It is important to note that waitForSend() should NOT return true if a response + // is expected from the node at the other end, e.g. when using RPCCall or QuorumCall. + // CallOptions are not provided with the requests from these types and thus waitForSend() + // returns false. This design should maybe be revised? if req.waitForSend() { // unblock the caller and clean up the responseRouter map c.routeResponse(req.msg.Metadata.MessageID, response{}) @@ -197,12 +273,24 @@ func (c *channel) sendMsg(req request) (err error) { case <-done: // false alarm default: + // CANCELLING HERE CAN HAVE DESTRUCTIVE EFFECTS! + // Imagine the client has sent several requests and is waiting + // for a response on each individual request. Furthermore, let's + // say the client has sent a message to two different handlers: + // 1. A handler that does a lot of work and thus long response times are expected. + // 2. A handler that is normally very fast. + // + // If the client is impatient and cancels a request sent to a handler in scenario 2, + // then all requests sent to the handler in scenario 1 will also be cancelled because + // the stream is taken down. + // trigger reconnect - c.cancelStream() + //c.streamMut.Lock() + //c.cancelStream() + //c.streamMut.Unlock() } } }() - err = c.gorumsStream.SendMsg(req.msg) if err != nil { c.setLastErr(err) @@ -226,18 +314,23 @@ func (c *channel) sender() { // have failed or if the node has disconnected if !c.isConnected() { // streamBroken will be set if the reconnection fails - c.connect() + err := c.connect() + if err != nil { + c.setLastErr(err) + c.streamBroken.set() + } } - // return error if stream is broken + // retry if stream is broken if c.streamBroken.get() { - c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: streamDownErr}) + go c.retryMsg(req, streamDownErr) continue } // else try to send message err := c.sendMsg(req) if err != nil { - // return the error - c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: err}) + go c.retryMsg(req, err) + } else { + c.log("channel: successfully sent msg", nil, slog.LevelInfo, req.getLogArgs(c)...) } } } @@ -246,7 +339,17 @@ func (c *channel) receiver() { for { resp := newMessage(responseType) c.streamMut.RLock() - err := c.gorumsStream.RecvMsg(resp) + var err error + // the gorumsStream can be nil because this method + // runs in a goroutine. If the stream goes down after + // the streamMut is unlocked, then we can have a scenario + // where the gorumsStream is set to nil in the reconnect + // method by another goroutine. + if c.gorumsStream != nil { + err = c.gorumsStream.RecvMsg(resp) + } else { + err = streamDownErr + } if err != nil { c.streamBroken.set() c.streamMut.RUnlock() @@ -255,6 +358,7 @@ func (c *channel) receiver() { // was sent and we are waiting for a reply. We thus need to respond // with a stream is down error on all pending messages. c.cancelPendingMsgs() + c.log("channel: lost connection", err, slog.LevelError, logging.Reconnect(true)) // attempt to reconnect indefinitely until the node is closed. // This is necessary when streaming is enabled. c.reconnect(-1) @@ -262,6 +366,11 @@ func (c *channel) receiver() { c.streamMut.RUnlock() err := status.FromProto(resp.Metadata.GetStatus()).Err() c.routeResponse(resp.Metadata.MessageID, response{nid: c.node.ID(), msg: resp.Message, err: err}) + if err != nil { + c.log("channel: got response", err, slog.LevelError, logging.MsgID(resp.Metadata.MessageID), logging.Method(resp.Metadata.Method)) + } else { + c.log("channel: got response", nil, slog.LevelInfo, logging.MsgID(resp.Metadata.MessageID), logging.Method(resp.Metadata.Method)) + } } select { @@ -274,19 +383,23 @@ func (c *channel) receiver() { func (c *channel) connect() error { if !c.connEstablished.get() { + c.dialMut.Lock() + defer c.dialMut.Unlock() // a connection has not yet been established; i.e., // a previous dial attempt could have failed. // try dialing again. err := c.node.dial() if err != nil { - c.streamBroken.set() return err } err = c.newNodeStream(c.node.conn) if err != nil { - c.streamBroken.set() return err } + // return early because streamBroken will be cleared if no error occurs. + // also works a preemptive measure to a deadlock since dialMut.Unlock() + // is deferred. + return nil } // the node was previously connected but is now disconnected if c.streamBroken.get() { @@ -297,12 +410,28 @@ func (c *channel) connect() error { return nil } +type retry float64 + +func (r retry) exceeds(maxRetries int) bool { + return r >= retry(maxRetries) && maxRetries > 0 +} + // reconnect tries to reconnect to the node using an exponential backoff strategy. // maxRetries = -1 represents infinite retries. -func (c *channel) reconnect(maxRetries float64) { +func (c *channel) reconnect(maxRetries int) { + select { + case <-c.parentCtx.Done(): + // no need to try to reconnect if the node is closed. + // make sure to cancel the previous ctx to prevent context leakage + if c.cancelStream != nil { + c.cancelStream() + } + return + default: + } backoffCfg := c.backoffCfg - var retries float64 + var retries retry for { var err error c.streamMut.Lock() @@ -312,17 +441,23 @@ func (c *channel) reconnect(maxRetries float64) { c.streamMut.Unlock() return } + // make sure to cancel the previous ctx to prevent context leakage + if c.cancelStream != nil { + c.cancelStream() + } c.streamCtx, c.cancelStream = context.WithCancel(c.parentCtx) c.gorumsStream, err = c.gorumsClient.NodeStream(c.streamCtx) if err == nil { + c.log("channel: restored connection", nil, slog.LevelInfo) c.streamBroken.clear() c.streamMut.Unlock() return } + c.log("channel: reconnection failed", err, slog.LevelWarn, logging.RetryNum(float64(retries)), logging.Reconnect(!(retries.exceeds(maxRetries) || retries.exceeds(c.maxConnRetries)))) c.cancelStream() c.streamMut.Unlock() c.setLastErr(err) - if retries >= maxRetries && maxRetries > 0 { + if retries.exceeds(maxRetries) || retries.exceeds(c.maxConnRetries) { c.streamBroken.set() return } @@ -342,6 +477,39 @@ func (c *channel) reconnect(maxRetries float64) { } } +// This method should always be run in a goroutine. It will +// enqueue a msg if it has previously failed. The message will +// be dropped if it fails more than maxRetries or if the ctx +// is cancelled. +func (c *channel) retryMsg(req request, err error) { + req.numFailed++ + c.log("channel: failed to send msg", err, slog.LevelError, req.getLogArgs(c)...) + // c.maxRetries = -1, is the same as infinite retries. + if req.numFailed > c.maxSendRetries && c.maxSendRetries != -1 { + c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("max retries exceeded. err=%e", err)}) + return + } + delay := float64(c.backoffCfg.BaseDelay) + //delay := float64(10 * time.Millisecond) + max := float64(c.backoffCfg.MaxDelay) + for r := req.numFailed; delay < max && r > 0; r-- { + delay *= c.backoffCfg.Multiplier + } + delay = math.Min(delay, max) + delay *= 1 + c.backoffCfg.Jitter*(rand.Float64()*2-1) + select { + case <-c.parentCtx.Done(): + c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("channel closed")}) + return + case <-req.ctx.Done(): + c.routeResponse(req.msg.Metadata.MessageID, response{nid: c.node.ID(), err: fmt.Errorf("context cancelled")}) + return + case <-time.After(time.Duration(delay)): + // enqueue the request again + } + c.enqueueSlow(req) +} + func (c *channel) setLastErr(err error) { c.mu.Lock() defer c.mu.Unlock() @@ -370,6 +538,13 @@ func (c *channel) isConnected() bool { return c.connEstablished.get() && !c.streamBroken.get() } +func (c *channel) log(msg string, err error, level slog.Level, args ...slog.Attr) { + if c.logger != nil { + args = append(args, logging.Err(err), logging.Type("channel")) + c.logger.LogAttrs(context.Background(), level, msg, args...) + } +} + type atomicFlag struct { flag int32 } diff --git a/channel_test.go b/channel_test.go index 8919ca0d..7e54597a 100644 --- a/channel_test.go +++ b/channel_test.go @@ -1,7 +1,10 @@ package gorums import ( + "bytes" "context" + "crypto/elliptic" + "net" "testing" "time" @@ -34,7 +37,7 @@ func dummySrv() *Server { req := in.Message.(*mock.Request) defer ctx.Release() resp, err := mockSrv.Test(ctx, req) - SendMessage(ctx, finished, WrapMessage(in.Metadata, resp, err)) + _ = SendMessage(ctx, finished, WrapMessage(in.Metadata, resp, err)) }) return srv } @@ -47,7 +50,7 @@ func TestChannelCreation(t *testing.T) { mgr := dummyMgr() defer mgr.Close() // a proper connection should NOT be established here - node.connect(mgr) + _ = node.connect(mgr) replyChan := make(chan response, 1) go func() { @@ -110,7 +113,7 @@ func TestChannelUnsuccessfulConnection(t *testing.T) { } func TestChannelReconnection(t *testing.T) { - srvAddr := "127.0.0.1:5000" + srvAddr := "127.0.0.1:5005" // wait to start the server startServer, stopServer := testServerSetup(t, srvAddr, dummySrv()) node, err := NewRawNode(srvAddr) @@ -120,7 +123,7 @@ func TestChannelReconnection(t *testing.T) { mgr := dummyMgr() defer mgr.Close() // a proper connection should NOT be established here because server is not started - node.connect(mgr) + _ = node.connect(mgr) // send first message when server is down replyChan1 := make(chan response, 1) @@ -180,3 +183,72 @@ func TestChannelReconnection(t *testing.T) { t.Fatal("deadlock: impossible to enqueue messages to the node") } } + +func TestAuthentication(t *testing.T) { + srv := NewServer(EnforceAuthentication(elliptic.P256())) + node, err := NewRawNode("127.0.0.1:5000") + if err != nil { + t.Fatal(err) + } + addr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:5000") + if err != nil { + t.Fatal(err) + } + auth := NewAuth(elliptic.P256()) + _ = auth.GenerateKeys() + privKey, pubKey := auth.Keys() + auth.RegisterKeys(addr, privKey, pubKey) + mgr := NewRawManager(WithAuthentication(auth)) + defer mgr.Close() + node.mgr = mgr + config := make(RawConfiguration, 0) + config = append(config, node) + + md := &ordering.Metadata{MessageID: 0, Method: "test", BroadcastMsg: &ordering.BroadcastMsg{ + IsBroadcastClient: true, + BroadcastID: 0, + SenderAddr: "127.0.0.1:5000", + OriginAddr: "127.0.0.1:5000", + OriginMethod: "test", + }} + msg := &Message{Metadata: md, Message: &mock.Request{}} + msg1 := &Message{Metadata: md, Message: &mock.Request{}} + + chEncodedMsg, _ := config.encodeMsg(msg1) + srvEncodedMsg, _ := srv.srv.encodeMsg(msg1) + if !bytes.Equal(chEncodedMsg, srvEncodedMsg) { + t.Fatalf("wrong encoding. want: %x, got: %x", chEncodedMsg, srvEncodedMsg) + } + pemEncodedPub, err := auth.EncodePublic() + if err != nil { + t.Fatal(err) + } + signature, err := auth.Sign(chEncodedMsg) + if err != nil { + t.Fatal(err) + } + valid, err := auth.VerifySignature(pemEncodedPub, chEncodedMsg, signature) + if err != nil { + t.Fatal(err) + } + if !valid { + t.Fatal("channel encoded msg not valid") + } + valid, err = auth.VerifySignature(pemEncodedPub, srvEncodedMsg, signature) + if err != nil { + t.Fatal(err) + } + if !valid { + t.Fatal("srv encoded msg not valid") + } + + config.sign(msg) + + if pemEncodedPub != msg.Metadata.AuthMsg.PublicKey { + t.Fatalf("wrong pub key. want: %s, got: %s", pemEncodedPub, msg.Metadata.AuthMsg.PublicKey) + } + + if err := srv.srv.verify(msg); err != nil { + t.Fatalf("authentication failed. want: nil, got: %s", err) + } +} diff --git a/clientserver.go b/clientserver.go new file mode 100644 index 00000000..7d2374e2 --- /dev/null +++ b/clientserver.go @@ -0,0 +1,344 @@ +package gorums + +import ( + "context" + "fmt" + "log/slog" + "net" + "sync" + "time" + + "github.com/relab/gorums/authentication" + "github.com/relab/gorums/broadcast" + "github.com/relab/gorums/logging" + "github.com/relab/gorums/ordering" + "google.golang.org/grpc" + "google.golang.org/protobuf/reflect/protoreflect" +) + +type ReplySpecHandler func(req protoreflect.ProtoMessage, replies []protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) + +type csr struct { + ctx context.Context + cancel context.CancelFunc + respChan chan protoreflect.ProtoMessage +} + +type ClientServer struct { + id uint64 // should correpond to the MachineID given to the manager + addr string + mu sync.Mutex + csr map[uint64]*csr + lis net.Listener + ctx context.Context + cancelCtx context.CancelFunc + grpcServer *grpc.Server + handlers map[string]requestHandler + logger *slog.Logger + auth *authentication.EllipticCurve + allowList map[string]string + ordering.UnimplementedGorumsServer +} + +func (srv *ClientServer) Stop() { + if srv.logger != nil { + srv.logger.Info("clientserver: stopped") + } + if srv.cancelCtx != nil { + srv.cancelCtx() + } + if srv.grpcServer != nil { + srv.grpcServer.Stop() + } +} + +func (srv *ClientServer) AddRequest(broadcastID uint64, clientCtx context.Context, in protoreflect.ProtoMessage, handler ReplySpecHandler, method string) (chan protoreflect.ProtoMessage, BroadcastCallData) { + cd := BroadcastCallData{ + Message: in, + Method: method, + + BroadcastID: broadcastID, + IsBroadcastClient: true, + OriginAddr: srv.addr, + } + // we expect one response when we are done + doneChan := make(chan protoreflect.ProtoMessage, 1) + // we should buffer this channel according to the number of servers. + // most configs hopefully contain less than 7 servers. + respChan := make(chan protoreflect.ProtoMessage, 7) + ctx, cancel := context.WithCancel(srv.ctx) + + srv.mu.Lock() + srv.csr[broadcastID] = &csr{ + ctx: ctx, + cancel: cancel, + respChan: respChan, + } + srv.mu.Unlock() + + var logger *slog.Logger + if srv.logger != nil { + logger = srv.logger.With(logging.BroadcastID(broadcastID)) + } + go createReq(ctx, clientCtx, cancel, in, doneChan, respChan, handler, logger) + + return doneChan, cd +} + +func createReq(ctx, clientCtx context.Context, cancel context.CancelFunc, req protoreflect.ProtoMessage, doneChan chan protoreflect.ProtoMessage, respChan chan protoreflect.ProtoMessage, handler ReplySpecHandler, logger *slog.Logger) { + // make sure to cancel the req ctx when returning to + // prevent a leaking ctx. + defer cancel() + resps := make([]protoreflect.ProtoMessage, 0, 3) + for { + select { + case <-clientCtx.Done(): + // client provided ctx + if logger != nil { + logger.Warn("clientserver: stopped by client", logging.Cancelled(true)) + } + return + case <-ctx.Done(): + // request ctx. this is a child to the server ctx. + // hence, guaranteeing that all reqs are finished + // when the server is stopped. + + // we must send on channel to prevent deadlock on + // the receiving end. this can happen if the client + // chooses not to timeout the request and the server + // goes down. + close(doneChan) + if logger != nil { + logger.Warn("clientserver: stopped by server", logging.Cancelled(true)) + } + return + case resp := <-respChan: + // keep track of all responses thus far + resps = append(resps, resp) + // handler is the QSpec method provided by the implementer. + response, done := handler(req, resps) + if done { + select { + case doneChan <- response: + if logger != nil { + logger.Info("clientserver: req done", logging.Cancelled(false)) + } + case <-ctx.Done(): + if logger != nil { + logger.Warn("clientserver: req done but stopped by server", logging.Cancelled(true)) + } + case <-clientCtx.Done(): + if logger != nil { + logger.Warn("clientserver: req done but cancelled by client", logging.Cancelled(true)) + } + } + close(doneChan) + return + } + } + } + +} + +func (srv *ClientServer) AddResponse(ctx context.Context, resp protoreflect.ProtoMessage, broadcastID uint64) error { + if broadcastID == 0 { + return fmt.Errorf("no broadcastID") + } + srv.mu.Lock() + csr, ok := srv.csr[broadcastID] + srv.mu.Unlock() + + if !ok { + return fmt.Errorf("doesn't exist") + } + if srv.logger != nil { + srv.logger.Info("clientserver: got a reply", logging.BroadcastID(broadcastID)) + } + select { + case <-ctx.Done(): + return ctx.Err() + case <-csr.ctx.Done(): + return csr.ctx.Err() + case csr.respChan <- resp: + } + return nil +} + +func ConvertToType[T, U protoreflect.ProtoMessage](handler func(U, []T) (T, bool)) ReplySpecHandler { + return func(req protoreflect.ProtoMessage, replies []protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + data := make([]T, len(replies)) + for i, elem := range replies { + data[i] = elem.(T) + } + return handler(req.(U), data) + } +} + +// NodeStream handles a connection to a single client. The stream is aborted if there +// is any error with sending or receiving. +func (s *ClientServer) NodeStream(srv ordering.Gorums_NodeStreamServer) error { + var mut sync.Mutex // used to achieve mutex between request handlers + ctx := srv.Context() + // Start with a locked mutex + mut.Lock() + defer mut.Unlock() + for { + req := newMessage(responseType) + err := srv.RecvMsg(req) + if err != nil { + return err + } + err = s.verify(req) + if err != nil { + continue + } + if handler, ok := s.handlers[req.Metadata.Method]; ok { + go handler(ServerCtx{Context: ctx, once: new(sync.Once), mut: &mut}, req, nil) + mut.Lock() + } + } +} + +// NewClientServer returns a new instance of ClientServer. +// This function is intended for internal Gorums use. +// You should call `NewServer` in the generated code instead. +func NewClientServer(lis net.Listener, opts ...ServerOption) *ClientServer { + var serverOpts serverOptions + for _, opt := range opts { + opt(&serverOpts) + } + if serverOpts.listenAddr == "" { + panic("The listen addr cannot be empty. Provide the WithListenAddr() option to AddClientServer().") + } + var logger *slog.Logger + if serverOpts.logger != nil { + logger = serverOpts.logger.With(logging.MachineID(serverOpts.machineID)) + } + ctx, cancel := context.WithCancel(context.Background()) + srv := &ClientServer{ + id: serverOpts.machineID, + addr: serverOpts.listenAddr, + ctx: ctx, + cancelCtx: cancel, + csr: make(map[uint64]*csr), + grpcServer: grpc.NewServer(serverOpts.grpcOpts...), + handlers: make(map[string]requestHandler), + logger: logger, + } + ordering.RegisterGorumsServer(srv.grpcServer, srv) + srv.lis = lis + if serverOpts.auth != nil { + srv.auth = serverOpts.auth + } + if serverOpts.allowList != nil { + srv.allowList = serverOpts.allowList + } + return srv +} + +// RegisterHandler registers a request handler for the specified method name. +// +// This function should only be used by generated code. +func (srv *ClientServer) RegisterHandler(method string, handler requestHandler) { + srv.handlers[method] = handler +} + +// Serve starts serving on the listener. +func (srv *ClientServer) Serve(listener net.Listener) error { + if srv.addr == "" { + srv.addr = listener.Addr().String() + } + return srv.grpcServer.Serve(listener) +} + +func (srv *ClientServer) encodeMsg(req *Message) ([]byte, error) { + // we must not consider the signature field when validating. + // also the msgType must be set to requestType. + signature := make([]byte, len(req.Metadata.AuthMsg.Signature)) + copy(signature, req.Metadata.AuthMsg.Signature) + reqType := req.msgType + req.Metadata.AuthMsg.Signature = nil + req.msgType = 0 + encodedMsg, err := srv.auth.EncodeMsg(*req) + req.Metadata.AuthMsg.Signature = make([]byte, len(signature)) + copy(req.Metadata.AuthMsg.Signature, signature) + req.msgType = reqType + return encodedMsg, err +} + +func (srv *ClientServer) verify(req *Message) error { + if srv.auth == nil { + return nil + } + if req.Metadata.AuthMsg == nil { + return fmt.Errorf("missing authMsg") + } + if req.Metadata.AuthMsg.Signature == nil { + return fmt.Errorf("missing signature") + } + if req.Metadata.AuthMsg.PublicKey == "" { + return fmt.Errorf("missing publicKey") + } + authMsg := req.Metadata.AuthMsg + if srv.allowList != nil { + pemEncodedPub, ok := srv.allowList[authMsg.Sender] + if !ok { + return fmt.Errorf("not allowed") + } + if pemEncodedPub != authMsg.PublicKey { + return fmt.Errorf("publicKey did not match") + } + } + encodedMsg, err := srv.encodeMsg(req) + if err != nil { + return err + } + valid, err := srv.auth.VerifySignature(authMsg.PublicKey, encodedMsg, authMsg.Signature) + if err != nil { + return err + } + if !valid { + return fmt.Errorf("invalid signature") + } + return nil +} + +func createClient(addr string, dialOpts []grpc.DialOption) (*broadcast.Client, error) { + // necessary to ensure correct marshalling and unmarshalling of gorums messages + dialOpts = append(dialOpts, grpc.WithDefaultCallOptions(grpc.CallContentSubtype(ContentSubtype))) + opts := newManagerOptions() + opts.grpcDialOpts = dialOpts + mgr := &RawManager{ + opts: opts, + } + node, err := NewRawNode(addr) + if err != nil { + return nil, err + } + err = node.connect(mgr) + if err != nil { + return nil, err + } + return &broadcast.Client{ + Addr: node.Address(), + SendMsg: func(broadcastID uint64, method string, msg protoreflect.ProtoMessage, timeout time.Duration, originDigest, originSignature []byte, originPubKey string) error { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + cd := CallData{ + Method: method, + Message: msg, + BroadcastID: broadcastID, + OriginDigest: originDigest, + OriginSignature: originSignature, + OriginPubKey: originPubKey, + } + _, err := node.RPCCall(ctx, cd) + return err + }, + Close: func() error { + mgr.Close() + node.close() + return nil + }, + }, nil +} diff --git a/cmd/protoc-gen-gorums/dev/config.go b/cmd/protoc-gen-gorums/dev/config.go index 99e8e86e..bb8bc9b3 100644 --- a/cmd/protoc-gen-gorums/dev/config.go +++ b/cmd/protoc-gen-gorums/dev/config.go @@ -10,8 +10,10 @@ import ( // procedure calls may be invoked. type Configuration struct { gorums.RawConfiguration - qspec QuorumSpec - nodes []*Node + qspec QuorumSpec + srv *clientServerImpl + snowflake gorums.Snowflake + nodes []*Node } // ConfigurationFromRaw returns a new Configuration from the given raw configuration and QuorumSpec. diff --git a/cmd/protoc-gen-gorums/dev/mgr.go b/cmd/protoc-gen-gorums/dev/mgr.go index 3180a912..56f64583 100644 --- a/cmd/protoc-gen-gorums/dev/mgr.go +++ b/cmd/protoc-gen-gorums/dev/mgr.go @@ -2,6 +2,7 @@ package dev import ( "fmt" + "net" "github.com/relab/gorums" "google.golang.org/grpc/encoding" @@ -17,6 +18,7 @@ func init() { // which quorum calls can be performed. type Manager struct { *gorums.RawManager + srv *clientServerImpl } // NewManager returns a new Manager for managing connection to nodes added @@ -28,6 +30,35 @@ func NewManager(opts ...gorums.ManagerOption) *Manager { } } +func (mgr *Manager) Close() { + if mgr.RawManager != nil { + mgr.RawManager.Close() + } + if mgr.srv != nil { + mgr.srv.stop() + } +} + +// AddClientServer starts a lightweight client-side server. This server only accepts responses +// to broadcast requests sent by the client. +// +// It is important to provide the listenAddr because this will be used to advertise the IP the +// servers should reply back to. +func (mgr *Manager) AddClientServer(lis net.Listener, clientAddr net.Addr, opts ...gorums.ServerOption) error { + options := []gorums.ServerOption{gorums.WithListenAddr(clientAddr)} + options = append(options, opts...) + srv := gorums.NewClientServer(lis, options...) + srvImpl := &clientServerImpl{ + ClientServer: srv, + } + registerClientServerHandlers(srvImpl) + go func() { + _ = srvImpl.Serve(lis) + }() + mgr.srv = srvImpl + return nil +} + // NewConfiguration returns a configuration based on the provided list of nodes (required) // and an optional quorum specification. The QuorumSpec is necessary for call types that // must process replies. For configurations only used for unicast or multicast call types, @@ -54,11 +85,16 @@ func (m *Manager) NewConfiguration(opts ...gorums.ConfigOption) (c *Configuratio return nil, fmt.Errorf("config: unknown option type: %v", v) } } - // return an error if the QuorumSpec interface is not empty and no implementation was provided. - var test interface{} = struct{}{} - if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { - return nil, fmt.Errorf("config: missing required QuorumSpec") + // register the client server if it exists. + // used to collect responses in BroadcastCalls + if m.srv != nil { + c.srv = m.srv } + c.snowflake = m.Snowflake() + //var test interface{} = struct{}{} + //if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { + // return nil, fmt.Errorf("config: missing required QuorumSpec") + //} // initialize the nodes slice c.nodes = make([]*Node, c.Size()) for i, n := range c.RawConfiguration { diff --git a/cmd/protoc-gen-gorums/dev/server.go b/cmd/protoc-gen-gorums/dev/server.go new file mode 100644 index 00000000..eaf6ae0e --- /dev/null +++ b/cmd/protoc-gen-gorums/dev/server.go @@ -0,0 +1,120 @@ +package dev + +import ( + "fmt" + + "github.com/relab/gorums" + grpc "google.golang.org/grpc" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" +) + +type Server struct { + *gorums.Server + broadcast *Broadcast + View *Configuration +} + +func NewServer(opts ...gorums.ServerOption) *Server { + srv := &Server{ + Server: gorums.NewServer(opts...), + } + b := &Broadcast{ + orchestrator: gorums.NewBroadcastOrchestrator(srv.Server), + } + srv.broadcast = b + srv.RegisterBroadcaster(newBroadcaster) + return srv +} + +func newBroadcaster(m gorums.BroadcastMetadata, o *gorums.BroadcastOrchestrator, e gorums.EnqueueBroadcast) gorums.Broadcaster { + return &Broadcast{ + orchestrator: o, + metadata: m, + srvAddrs: make([]string, 0), + enqueueBroadcast: e, + } +} + +func (srv *Server) SetView(config *Configuration) { + srv.View = config + srv.RegisterConfig(config.RawConfiguration) +} + +type Broadcast struct { + orchestrator *gorums.BroadcastOrchestrator + metadata gorums.BroadcastMetadata + srvAddrs []string + enqueueBroadcast gorums.EnqueueBroadcast +} + +// Returns a readonly struct of the metadata used in the broadcast. +// +// Note: Some of the data are equal across the cluster, such as BroadcastID. +// Other fields are local, such as SenderAddr. +func (b *Broadcast) GetMetadata() gorums.BroadcastMetadata { + return b.metadata +} + +type clientServerImpl struct { + *gorums.ClientServer + grpcServer *grpc.Server +} + +func (c *clientServerImpl) stop() { + c.ClientServer.Stop() + if c.grpcServer != nil { + c.grpcServer.Stop() + } +} + +func (b *Broadcast) To(addrs ...string) *Broadcast { + if len(addrs) <= 0 { + return b + } + b.srvAddrs = append(b.srvAddrs, addrs...) + return b +} + +func (b *Broadcast) Forward(req protoreflect.ProtoMessage, addr string) error { + if addr == "" { + return fmt.Errorf("cannot forward to empty addr, got: %s", addr) + } + if !b.metadata.IsBroadcastClient { + return fmt.Errorf("can only forward client requests") + } + go b.orchestrator.ForwardHandler(req, b.metadata.OriginMethod, b.metadata.BroadcastID, addr, b.metadata.OriginAddr) + return nil +} + +// Done signals the end of a broadcast request. It is necessary to call +// either Done() or SendToClient() to properly terminate a broadcast request +// and free up resources. Otherwise, it could cause poor performance. +func (b *Broadcast) Done() { + b.orchestrator.DoneHandler(b.metadata.BroadcastID, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (b *Broadcast) SendToClient(resp protoreflect.ProtoMessage, err error) error { + return b.orchestrator.SendToClientHandler(b.metadata.BroadcastID, resp, err, b.enqueueBroadcast) +} + +// Cancel is a non-destructive method call that will transmit a cancellation +// to all servers in the view. It will not stop the execution but will cause +// the given ServerCtx to be cancelled, making it possible to listen for +// cancellations. +// +// Could be used together with either SendToClient() or Done(). +func (b *Broadcast) Cancel() error { + return b.orchestrator.CancelHandler(b.metadata.BroadcastID, b.srvAddrs, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (srv *Server) SendToClient(resp protoreflect.ProtoMessage, err error, broadcastID uint64) error { + return srv.SendToClientHandler(resp, err, broadcastID, nil) +} diff --git a/cmd/protoc-gen-gorums/dev/zorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums.pb.go index 8b4bbc2c..8d5111e8 100644 --- a/cmd/protoc-gen-gorums/dev/zorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums.pb.go @@ -1,16 +1,16 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.33.0 -// protoc v4.25.3 +// protoc-gen-go v1.32.0 +// protoc v3.12.4 // source: zorums.proto package dev import ( + empty "github.com/golang/protobuf/ptypes/empty" _ "github.com/relab/gorums" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" - emptypb "google.golang.org/protobuf/types/known/emptypb" reflect "reflect" sync "sync" ) @@ -176,7 +176,7 @@ var file_zorums_proto_rawDesc = []byte{ 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x22, 0x0a, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x32, 0x86, 0x11, 0x0a, 0x0d, 0x5a, 0x6f, 0x72, 0x75, + 0x52, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x32, 0xae, 0x14, 0x0a, 0x0d, 0x5a, 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x29, 0x0a, 0x08, 0x47, 0x52, 0x50, 0x43, 0x43, 0x61, 0x6c, 0x6c, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, @@ -204,118 +204,144 @@ var file_zorums_proto_rawDesc = []byte{ 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x12, 0x2e, 0x0a, 0x09, 0x4d, - 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3c, 0x0a, 0x13, 0x4d, - 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, - 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x08, 0x98, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0x12, 0x2f, 0x0a, 0x0a, 0x4d, 0x75, 0x6c, - 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x38, 0x0a, 0x0a, 0x4d, 0x75, - 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x33, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, - 0x98, 0xb5, 0x18, 0x01, 0x12, 0x42, 0x0a, 0x0a, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, - 0x74, 0x34, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x12, 0x40, 0x0a, 0x17, 0x51, + 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x42, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x48, 0x0a, + 0x16, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x57, 0x69, 0x74, 0x68, 0x42, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x08, 0x98, + 0xb5, 0x18, 0x01, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x3f, 0x0a, 0x11, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x0c, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x38, 0x0a, 0x0f, 0x51, 0x75, 0x6f, 0x72, - 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x0c, 0x2e, 0x64, 0x65, - 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, - 0x18, 0x01, 0x12, 0x46, 0x0a, 0x19, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, - 0x41, 0x73, 0x79, 0x6e, 0x63, 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, - 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, - 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x0c, 0xa0, 0xb5, - 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0x12, 0x56, 0x0a, 0x1f, 0x51, 0x75, - 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x43, 0x75, 0x73, - 0x74, 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x2e, - 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, - 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0xa0, 0xb5, 0x18, 0x01, - 0xd0, 0xb5, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x4f, 0x0a, 0x14, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, - 0x41, 0x73, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, + 0x74, 0x79, 0x22, 0x04, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x40, 0x0a, 0x1b, 0x42, 0x72, 0x6f, 0x61, + 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x48, + 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x31, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, 0x01, 0x12, 0x40, 0x0a, 0x1b, 0x42, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x74, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, 0x01, 0x12, 0x55, 0x0a, 0x2c, + 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x41, 0x6e, 0x64, 0x42, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0c, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, + 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xb0, 0xb5, 0x18, 0x01, 0x90, + 0xb8, 0x18, 0x01, 0x12, 0x2e, 0x0a, 0x09, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, + 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, + 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0x98, + 0xb5, 0x18, 0x01, 0x12, 0x3c, 0x0a, 0x13, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, + 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1a, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, - 0x01, 0xa0, 0xb6, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x10, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, - 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0x98, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, + 0x01, 0x12, 0x2f, 0x0a, 0x0a, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x32, 0x12, + 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, + 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0x98, 0xb5, + 0x18, 0x01, 0x12, 0x38, 0x0a, 0x0a, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x33, + 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x42, 0x0a, 0x0a, + 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x34, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, + 0x12, 0x38, 0x0a, 0x0f, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, + 0x79, 0x6e, 0x63, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x46, 0x0a, 0x19, 0x51, 0x75, + 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x50, 0x65, 0x72, + 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x46, - 0x0a, 0x14, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, - 0x63, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x08, 0xa0, 0xb5, - 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x48, 0x0a, 0x15, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, - 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x12, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, - 0x12, 0x30, 0x0a, 0x0b, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x0c, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0xa0, 0xb6, + 0x18, 0x01, 0x12, 0x56, 0x0a, 0x1f, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, + 0x41, 0x73, 0x79, 0x6e, 0x63, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, + 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x16, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, + 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4f, 0x0a, 0x14, 0x51, 0x75, + 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x43, 0x6f, 0x6d, + 0x62, 0x6f, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x1a, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0xf2, 0xb6, 0x18, + 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x39, 0x0a, 0x10, 0x51, + 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, - 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, - 0x18, 0x01, 0x12, 0x3e, 0x0a, 0x15, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, - 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, - 0x18, 0x01, 0x12, 0x4e, 0x0a, 0x1b, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x12, - 0xa8, 0xb5, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x47, 0x0a, 0x10, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x16, 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0xf2, 0xb6, 0x18, - 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x10, 0x43, - 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, - 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x12, 0x40, 0x0a, 0x11, 0x43, - 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, - 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x12, 0x38, 0x0a, - 0x11, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, + 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x46, 0x0a, 0x14, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, + 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, 0x63, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x0c, + 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, + 0x6d, 0x70, 0x74, 0x79, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x48, + 0x0a, 0x15, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x41, 0x73, 0x79, 0x6e, + 0x63, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, + 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, + 0xa0, 0xb5, 0x18, 0x01, 0xd0, 0xb5, 0x18, 0x01, 0x12, 0x30, 0x0a, 0x0b, 0x43, 0x6f, 0x72, 0x72, + 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x12, 0x3e, 0x0a, 0x15, 0x43, 0x6f, + 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, + 0x41, 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x08, 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0x12, 0x4e, 0x0a, 0x1b, 0x43, 0x6f, + 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x52, + 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x12, 0xa8, 0xb5, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, + 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x10, 0x43, 0x6f, + 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x12, 0x0c, + 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0xa8, 0xb5, 0x18, + 0x01, 0xa0, 0xb6, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x3e, 0x0a, 0x10, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0xa8, + 0xb5, 0x18, 0x01, 0x12, 0x40, 0x0a, 0x11, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x30, 0x01, 0x12, 0x46, 0x0a, 0x1b, 0x43, 0x6f, 0x72, 0x72, 0x65, - 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x65, 0x72, 0x4e, - 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0x30, 0x01, 0x12, - 0x56, 0x0a, 0x21, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x43, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, - 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x12, 0xa8, 0xb5, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x4f, 0x0a, 0x16, 0x43, 0x6f, 0x72, 0x72, 0x65, - 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x62, - 0x6f, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, - 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x46, 0x0a, 0x16, 0x43, 0x6f, 0x72, 0x72, + 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x12, 0x38, 0x0a, 0x11, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, + 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x30, 0x01, 0x12, + 0x46, 0x0a, 0x1b, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x50, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x12, 0x0c, + 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa8, 0xb5, 0x18, + 0x01, 0xa0, 0xb6, 0x18, 0x01, 0x30, 0x01, 0x12, 0x56, 0x0a, 0x21, 0x43, 0x6f, 0x72, 0x72, 0x65, + 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x75, 0x73, 0x74, + 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0c, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, + 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x12, 0xa8, 0xb5, 0x18, 0x01, 0xf2, + 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, + 0x4f, 0x0a, 0x16, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0xa8, 0xb5, 0x18, 0x01, 0xa0, 0xb6, 0x18, 0x01, + 0xf2, 0xb6, 0x18, 0x0a, 0x4d, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, + 0x12, 0x46, 0x0a, 0x16, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, + 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x30, 0x01, 0x12, 0x48, 0x0a, 0x17, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x30, 0x01, - 0x12, 0x48, 0x0a, 0x17, 0x43, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, - 0x74, 0x72, 0x65, 0x61, 0x6d, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x12, 0x16, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, - 0x70, 0x74, 0x79, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, 0x30, 0x01, 0x12, 0x2c, 0x0a, 0x07, 0x55, 0x6e, - 0x69, 0x63, 0x61, 0x73, 0x74, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x04, 0x90, 0xb5, 0x18, 0x01, 0x12, 0x36, 0x0a, 0x08, 0x55, 0x6e, 0x69, 0x63, - 0x61, 0x73, 0x74, 0x32, 0x12, 0x0c, 0x2e, 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, 0xb5, 0x18, 0x01, - 0x42, 0x1b, 0x5a, 0x19, 0x63, 0x6d, 0x64, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, - 0x65, 0x6e, 0x2d, 0x67, 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x2f, 0x64, 0x65, 0x76, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x74, 0x79, 0x32, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x0d, 0x2e, 0x64, 0x65, + 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa8, 0xb5, 0x18, 0x01, + 0x30, 0x01, 0x12, 0x2c, 0x0a, 0x07, 0x55, 0x6e, 0x69, 0x63, 0x61, 0x73, 0x74, 0x12, 0x0c, 0x2e, + 0x64, 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x64, 0x65, + 0x76, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0x90, 0xb5, 0x18, 0x01, + 0x12, 0x36, 0x0a, 0x08, 0x55, 0x6e, 0x69, 0x63, 0x61, 0x73, 0x74, 0x32, 0x12, 0x0c, 0x2e, 0x64, + 0x65, 0x76, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, + 0x74, 0x79, 0x22, 0x04, 0x90, 0xb5, 0x18, 0x01, 0x42, 0x1b, 0x5a, 0x19, 0x63, 0x6d, 0x64, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x2d, 0x67, 0x65, 0x6e, 0x2d, 0x67, 0x6f, 0x72, 0x75, 0x6d, + 0x73, 0x2f, 0x64, 0x65, 0x76, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -332,10 +358,10 @@ func file_zorums_proto_rawDescGZIP() []byte { var file_zorums_proto_msgTypes = make([]protoimpl.MessageInfo, 3) var file_zorums_proto_goTypes = []interface{}{ - (*Request)(nil), // 0: dev.Request - (*Response)(nil), // 1: dev.Response - (*MyResponse)(nil), // 2: dev.MyResponse - (*emptypb.Empty)(nil), // 3: google.protobuf.Empty + (*Request)(nil), // 0: dev.Request + (*Response)(nil), // 1: dev.Response + (*MyResponse)(nil), // 2: dev.MyResponse + (*empty.Empty)(nil), // 3: google.protobuf.Empty } var file_zorums_proto_depIdxs = []int32{ 0, // 0: dev.ZorumsService.GRPCCall:input_type -> dev.Request @@ -345,67 +371,79 @@ var file_zorums_proto_depIdxs = []int32{ 0, // 4: dev.ZorumsService.QuorumCallCombo:input_type -> dev.Request 3, // 5: dev.ZorumsService.QuorumCallEmpty:input_type -> google.protobuf.Empty 0, // 6: dev.ZorumsService.QuorumCallEmpty2:input_type -> dev.Request - 0, // 7: dev.ZorumsService.Multicast:input_type -> dev.Request - 0, // 8: dev.ZorumsService.MulticastPerNodeArg:input_type -> dev.Request - 0, // 9: dev.ZorumsService.Multicast2:input_type -> dev.Request - 0, // 10: dev.ZorumsService.Multicast3:input_type -> dev.Request - 3, // 11: dev.ZorumsService.Multicast4:input_type -> google.protobuf.Empty - 0, // 12: dev.ZorumsService.QuorumCallAsync:input_type -> dev.Request - 0, // 13: dev.ZorumsService.QuorumCallAsyncPerNodeArg:input_type -> dev.Request - 0, // 14: dev.ZorumsService.QuorumCallAsyncCustomReturnType:input_type -> dev.Request - 0, // 15: dev.ZorumsService.QuorumCallAsyncCombo:input_type -> dev.Request - 0, // 16: dev.ZorumsService.QuorumCallAsync2:input_type -> dev.Request - 0, // 17: dev.ZorumsService.QuorumCallAsyncEmpty:input_type -> dev.Request - 3, // 18: dev.ZorumsService.QuorumCallAsyncEmpty2:input_type -> google.protobuf.Empty - 0, // 19: dev.ZorumsService.Correctable:input_type -> dev.Request - 0, // 20: dev.ZorumsService.CorrectablePerNodeArg:input_type -> dev.Request - 0, // 21: dev.ZorumsService.CorrectableCustomReturnType:input_type -> dev.Request - 0, // 22: dev.ZorumsService.CorrectableCombo:input_type -> dev.Request - 0, // 23: dev.ZorumsService.CorrectableEmpty:input_type -> dev.Request - 3, // 24: dev.ZorumsService.CorrectableEmpty2:input_type -> google.protobuf.Empty - 0, // 25: dev.ZorumsService.CorrectableStream:input_type -> dev.Request - 0, // 26: dev.ZorumsService.CorrectableStreamPerNodeArg:input_type -> dev.Request - 0, // 27: dev.ZorumsService.CorrectableStreamCustomReturnType:input_type -> dev.Request - 0, // 28: dev.ZorumsService.CorrectableStreamCombo:input_type -> dev.Request - 0, // 29: dev.ZorumsService.CorrectableStreamEmpty:input_type -> dev.Request - 3, // 30: dev.ZorumsService.CorrectableStreamEmpty2:input_type -> google.protobuf.Empty - 0, // 31: dev.ZorumsService.Unicast:input_type -> dev.Request - 0, // 32: dev.ZorumsService.Unicast2:input_type -> dev.Request - 1, // 33: dev.ZorumsService.GRPCCall:output_type -> dev.Response - 1, // 34: dev.ZorumsService.QuorumCall:output_type -> dev.Response - 1, // 35: dev.ZorumsService.QuorumCallPerNodeArg:output_type -> dev.Response - 1, // 36: dev.ZorumsService.QuorumCallCustomReturnType:output_type -> dev.Response - 1, // 37: dev.ZorumsService.QuorumCallCombo:output_type -> dev.Response - 1, // 38: dev.ZorumsService.QuorumCallEmpty:output_type -> dev.Response - 3, // 39: dev.ZorumsService.QuorumCallEmpty2:output_type -> google.protobuf.Empty - 1, // 40: dev.ZorumsService.Multicast:output_type -> dev.Response - 1, // 41: dev.ZorumsService.MulticastPerNodeArg:output_type -> dev.Response - 1, // 42: dev.ZorumsService.Multicast2:output_type -> dev.Response - 3, // 43: dev.ZorumsService.Multicast3:output_type -> google.protobuf.Empty - 3, // 44: dev.ZorumsService.Multicast4:output_type -> google.protobuf.Empty - 1, // 45: dev.ZorumsService.QuorumCallAsync:output_type -> dev.Response - 1, // 46: dev.ZorumsService.QuorumCallAsyncPerNodeArg:output_type -> dev.Response - 1, // 47: dev.ZorumsService.QuorumCallAsyncCustomReturnType:output_type -> dev.Response - 1, // 48: dev.ZorumsService.QuorumCallAsyncCombo:output_type -> dev.Response - 1, // 49: dev.ZorumsService.QuorumCallAsync2:output_type -> dev.Response - 3, // 50: dev.ZorumsService.QuorumCallAsyncEmpty:output_type -> google.protobuf.Empty - 1, // 51: dev.ZorumsService.QuorumCallAsyncEmpty2:output_type -> dev.Response - 1, // 52: dev.ZorumsService.Correctable:output_type -> dev.Response - 1, // 53: dev.ZorumsService.CorrectablePerNodeArg:output_type -> dev.Response - 1, // 54: dev.ZorumsService.CorrectableCustomReturnType:output_type -> dev.Response - 1, // 55: dev.ZorumsService.CorrectableCombo:output_type -> dev.Response - 3, // 56: dev.ZorumsService.CorrectableEmpty:output_type -> google.protobuf.Empty - 1, // 57: dev.ZorumsService.CorrectableEmpty2:output_type -> dev.Response - 1, // 58: dev.ZorumsService.CorrectableStream:output_type -> dev.Response - 1, // 59: dev.ZorumsService.CorrectableStreamPerNodeArg:output_type -> dev.Response - 1, // 60: dev.ZorumsService.CorrectableStreamCustomReturnType:output_type -> dev.Response - 1, // 61: dev.ZorumsService.CorrectableStreamCombo:output_type -> dev.Response - 3, // 62: dev.ZorumsService.CorrectableStreamEmpty:output_type -> google.protobuf.Empty - 1, // 63: dev.ZorumsService.CorrectableStreamEmpty2:output_type -> dev.Response - 1, // 64: dev.ZorumsService.Unicast:output_type -> dev.Response - 3, // 65: dev.ZorumsService.Unicast2:output_type -> google.protobuf.Empty - 33, // [33:66] is the sub-list for method output_type - 0, // [0:33] is the sub-list for method input_type + 0, // 7: dev.ZorumsService.QuorumCallWithBroadcast:input_type -> dev.Request + 0, // 8: dev.ZorumsService.MulticastWithBroadcast:input_type -> dev.Request + 0, // 9: dev.ZorumsService.BroadcastInternal:input_type -> dev.Request + 0, // 10: dev.ZorumsService.BroadcastWithClientHandler1:input_type -> dev.Request + 0, // 11: dev.ZorumsService.BroadcastWithClientHandler2:input_type -> dev.Request + 0, // 12: dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption:input_type -> dev.Request + 0, // 13: dev.ZorumsService.Multicast:input_type -> dev.Request + 0, // 14: dev.ZorumsService.MulticastPerNodeArg:input_type -> dev.Request + 0, // 15: dev.ZorumsService.Multicast2:input_type -> dev.Request + 0, // 16: dev.ZorumsService.Multicast3:input_type -> dev.Request + 3, // 17: dev.ZorumsService.Multicast4:input_type -> google.protobuf.Empty + 0, // 18: dev.ZorumsService.QuorumCallAsync:input_type -> dev.Request + 0, // 19: dev.ZorumsService.QuorumCallAsyncPerNodeArg:input_type -> dev.Request + 0, // 20: dev.ZorumsService.QuorumCallAsyncCustomReturnType:input_type -> dev.Request + 0, // 21: dev.ZorumsService.QuorumCallAsyncCombo:input_type -> dev.Request + 0, // 22: dev.ZorumsService.QuorumCallAsync2:input_type -> dev.Request + 0, // 23: dev.ZorumsService.QuorumCallAsyncEmpty:input_type -> dev.Request + 3, // 24: dev.ZorumsService.QuorumCallAsyncEmpty2:input_type -> google.protobuf.Empty + 0, // 25: dev.ZorumsService.Correctable:input_type -> dev.Request + 0, // 26: dev.ZorumsService.CorrectablePerNodeArg:input_type -> dev.Request + 0, // 27: dev.ZorumsService.CorrectableCustomReturnType:input_type -> dev.Request + 0, // 28: dev.ZorumsService.CorrectableCombo:input_type -> dev.Request + 0, // 29: dev.ZorumsService.CorrectableEmpty:input_type -> dev.Request + 3, // 30: dev.ZorumsService.CorrectableEmpty2:input_type -> google.protobuf.Empty + 0, // 31: dev.ZorumsService.CorrectableStream:input_type -> dev.Request + 0, // 32: dev.ZorumsService.CorrectableStreamPerNodeArg:input_type -> dev.Request + 0, // 33: dev.ZorumsService.CorrectableStreamCustomReturnType:input_type -> dev.Request + 0, // 34: dev.ZorumsService.CorrectableStreamCombo:input_type -> dev.Request + 0, // 35: dev.ZorumsService.CorrectableStreamEmpty:input_type -> dev.Request + 3, // 36: dev.ZorumsService.CorrectableStreamEmpty2:input_type -> google.protobuf.Empty + 0, // 37: dev.ZorumsService.Unicast:input_type -> dev.Request + 0, // 38: dev.ZorumsService.Unicast2:input_type -> dev.Request + 1, // 39: dev.ZorumsService.GRPCCall:output_type -> dev.Response + 1, // 40: dev.ZorumsService.QuorumCall:output_type -> dev.Response + 1, // 41: dev.ZorumsService.QuorumCallPerNodeArg:output_type -> dev.Response + 1, // 42: dev.ZorumsService.QuorumCallCustomReturnType:output_type -> dev.Response + 1, // 43: dev.ZorumsService.QuorumCallCombo:output_type -> dev.Response + 1, // 44: dev.ZorumsService.QuorumCallEmpty:output_type -> dev.Response + 3, // 45: dev.ZorumsService.QuorumCallEmpty2:output_type -> google.protobuf.Empty + 1, // 46: dev.ZorumsService.QuorumCallWithBroadcast:output_type -> dev.Response + 3, // 47: dev.ZorumsService.MulticastWithBroadcast:output_type -> google.protobuf.Empty + 3, // 48: dev.ZorumsService.BroadcastInternal:output_type -> google.protobuf.Empty + 1, // 49: dev.ZorumsService.BroadcastWithClientHandler1:output_type -> dev.Response + 1, // 50: dev.ZorumsService.BroadcastWithClientHandler2:output_type -> dev.Response + 1, // 51: dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption:output_type -> dev.Response + 1, // 52: dev.ZorumsService.Multicast:output_type -> dev.Response + 1, // 53: dev.ZorumsService.MulticastPerNodeArg:output_type -> dev.Response + 1, // 54: dev.ZorumsService.Multicast2:output_type -> dev.Response + 3, // 55: dev.ZorumsService.Multicast3:output_type -> google.protobuf.Empty + 3, // 56: dev.ZorumsService.Multicast4:output_type -> google.protobuf.Empty + 1, // 57: dev.ZorumsService.QuorumCallAsync:output_type -> dev.Response + 1, // 58: dev.ZorumsService.QuorumCallAsyncPerNodeArg:output_type -> dev.Response + 1, // 59: dev.ZorumsService.QuorumCallAsyncCustomReturnType:output_type -> dev.Response + 1, // 60: dev.ZorumsService.QuorumCallAsyncCombo:output_type -> dev.Response + 1, // 61: dev.ZorumsService.QuorumCallAsync2:output_type -> dev.Response + 3, // 62: dev.ZorumsService.QuorumCallAsyncEmpty:output_type -> google.protobuf.Empty + 1, // 63: dev.ZorumsService.QuorumCallAsyncEmpty2:output_type -> dev.Response + 1, // 64: dev.ZorumsService.Correctable:output_type -> dev.Response + 1, // 65: dev.ZorumsService.CorrectablePerNodeArg:output_type -> dev.Response + 1, // 66: dev.ZorumsService.CorrectableCustomReturnType:output_type -> dev.Response + 1, // 67: dev.ZorumsService.CorrectableCombo:output_type -> dev.Response + 3, // 68: dev.ZorumsService.CorrectableEmpty:output_type -> google.protobuf.Empty + 1, // 69: dev.ZorumsService.CorrectableEmpty2:output_type -> dev.Response + 1, // 70: dev.ZorumsService.CorrectableStream:output_type -> dev.Response + 1, // 71: dev.ZorumsService.CorrectableStreamPerNodeArg:output_type -> dev.Response + 1, // 72: dev.ZorumsService.CorrectableStreamCustomReturnType:output_type -> dev.Response + 1, // 73: dev.ZorumsService.CorrectableStreamCombo:output_type -> dev.Response + 3, // 74: dev.ZorumsService.CorrectableStreamEmpty:output_type -> google.protobuf.Empty + 1, // 75: dev.ZorumsService.CorrectableStreamEmpty2:output_type -> dev.Response + 1, // 76: dev.ZorumsService.Unicast:output_type -> dev.Response + 3, // 77: dev.ZorumsService.Unicast2:output_type -> google.protobuf.Empty + 39, // [39:78] is the sub-list for method output_type + 0, // [0:39] is the sub-list for method input_type 0, // [0:0] is the sub-list for extension type_name 0, // [0:0] is the sub-list for extension extendee 0, // [0:0] is the sub-list for field type_name diff --git a/cmd/protoc-gen-gorums/dev/zorums.proto b/cmd/protoc-gen-gorums/dev/zorums.proto index b4f4e203..02d9271a 100644 --- a/cmd/protoc-gen-gorums/dev/zorums.proto +++ b/cmd/protoc-gen-gorums/dev/zorums.proto @@ -58,6 +58,60 @@ service ZorumsService { option (gorums.quorumcall) = true; } + // QuorumCall with broadcast option enables the server handler to broadcast. + // The handler still works like a regular QuorumCall from the client's + // perpective. + rpc QuorumCallWithBroadcast(Request) returns (Response) { + option (gorums.quorumcall) = true; + option (gorums.broadcast) = true; + } + + // Multicast with broadcast option enables the server handler to broadcast. + // The handler still works like a regular QuorumCall from the client's + // perpective. + rpc MulticastWithBroadcast(Request) returns (google.protobuf.Empty) { + option (gorums.multicast) = true; + option (gorums.broadcast) = true; + } + + // An rpc method with only the broadcast option specified will create a + // server handler which can be used to broadcast. The handler will not + // be exposed to the client, only to server. Hence, it is named internal. + rpc BroadcastInternal(Request) returns (google.protobuf.Empty) { + option (gorums.broadcast) = true; + } + + // The BroadcastCall is similar to a QuorumCall, but it enables the servers + // to send the response to a client-side server handler. I.e. the client + // needs to register a client-side server and register it. + // + // NOTE: this will NOT create a broadcast method, meaning servers cannot + // call this method (only clients) + rpc BroadcastWithClientHandler1(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + + // The BroadcastCall is similar to a QuorumCall, but it enables the servers + // to send the response to a client-side server handler. I.e. the client + // needs to register a client-side server and register it. + // + // NOTE: this will NOT create a broadcast method, meaning servers cannot + // call this method (only clients) + rpc BroadcastWithClientHandler2(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + + // The BroadcastCall is similar to a QuorumCall, but it enables the servers + // to send the response to a client-side server handler. I.e. the client + // needs to register a client-side server and register it. + // + // NOTE: this WILL create a broadcast method, meaning servers (and clients) + // can call this method + rpc BroadcastWithClientHandlerAndBroadcastOption(Request) returns (Response) { + option (gorums.broadcastcall) = true; + option (gorums.broadcast) = true; + } + // ------------------------------------------------------------------ // Multicast variants // The multicast call type does not support the 'custom_return_type' diff --git a/cmd/protoc-gen-gorums/dev/zorums_async_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_async_gorums.pb.go index a32fcacb..9479840b 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_async_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_async_gorums.pb.go @@ -1,16 +1,16 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( context "context" + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" protoreflect "google.golang.org/protobuf/reflect/protoreflect" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -123,9 +123,9 @@ func (c *Configuration) QuorumCallAsyncEmpty(ctx context.Context, in *Request) * Method: "dev.ZorumsService.QuorumCallAsyncEmpty", } cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { - r := make(map[uint32]*emptypb.Empty, len(replies)) + r := make(map[uint32]*empty.Empty, len(replies)) for k, v := range replies { - r[k] = v.(*emptypb.Empty) + r[k] = v.(*empty.Empty) } return c.qspec.QuorumCallAsyncEmptyQF(req.(*Request), r) } @@ -136,7 +136,7 @@ func (c *Configuration) QuorumCallAsyncEmpty(ctx context.Context, in *Request) * // QuorumCallAsyncEmpty2 for testing imported message type; with same return // type as QuorumCallAsync: Response. -func (c *Configuration) QuorumCallAsyncEmpty2(ctx context.Context, in *emptypb.Empty) *AsyncResponse { +func (c *Configuration) QuorumCallAsyncEmpty2(ctx context.Context, in *empty.Empty) *AsyncResponse { cd := gorums.QuorumCallData{ Message: in, Method: "dev.ZorumsService.QuorumCallAsyncEmpty2", @@ -146,7 +146,7 @@ func (c *Configuration) QuorumCallAsyncEmpty2(ctx context.Context, in *emptypb.E for k, v := range replies { r[k] = v.(*Response) } - return c.qspec.QuorumCallAsyncEmpty2QF(req.(*emptypb.Empty), r) + return c.qspec.QuorumCallAsyncEmpty2QF(req.(*empty.Empty), r) } fut := c.RawConfiguration.AsyncCall(ctx, cd) diff --git a/cmd/protoc-gen-gorums/dev/zorums_broadcast_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_broadcast_gorums.pb.go new file mode 100644 index 00000000..0369b6e5 --- /dev/null +++ b/cmd/protoc-gen-gorums/dev/zorums_broadcast_gorums.pb.go @@ -0,0 +1,66 @@ +// Code generated by protoc-gen-gorums. DO NOT EDIT. +// versions: +// protoc-gen-gorums v0.7.0-devel +// protoc v3.12.4 +// source: zorums.proto + +package dev + +import ( + gorums "github.com/relab/gorums" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = gorums.EnforceVersion(7 - gorums.MinVersion) + // Verify that the gorums runtime is sufficiently up-to-date. + _ = gorums.EnforceVersion(gorums.MaxVersion - 7) +) + +func (b *Broadcast) QuorumCallWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastQuorumCallWithBroadcast instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("dev.ZorumsService.QuorumCallWithBroadcast", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) MulticastWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastMulticastWithBroadcast instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("dev.ZorumsService.MulticastWithBroadcast", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) BroadcastInternal(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastBroadcastInternal instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("dev.ZorumsService.BroadcastInternal", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) BroadcastWithClientHandlerAndBroadcastOption(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastBroadcastWithClientHandlerAndBroadcastOption instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} diff --git a/cmd/protoc-gen-gorums/dev/zorums_broadcastcall_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_broadcastcall_gorums.pb.go new file mode 100644 index 00000000..d665aa0d --- /dev/null +++ b/cmd/protoc-gen-gorums/dev/zorums_broadcastcall_gorums.pb.go @@ -0,0 +1,184 @@ +// Code generated by protoc-gen-gorums. DO NOT EDIT. +// versions: +// protoc-gen-gorums v0.7.0-devel +// protoc v3.12.4 +// source: zorums.proto + +package dev + +import ( + context "context" + fmt "fmt" + gorums "github.com/relab/gorums" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + time "time" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = gorums.EnforceVersion(7 - gorums.MinVersion) + // Verify that the gorums runtime is sufficiently up-to-date. + _ = gorums.EnforceVersion(gorums.MaxVersion - 7) +) + +func (srv *clientServerImpl) clientBroadcastWithClientHandler1(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastWithClientHandler1(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastWithClientHandler1QF), "dev.ZorumsService.BroadcastWithClientHandler1") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientBroadcastWithClientHandler2(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastWithClientHandler2(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastWithClientHandler2QF), "dev.ZorumsService.BroadcastWithClientHandler2") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientBroadcastWithClientHandlerAndBroadcastOption(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastWithClientHandlerAndBroadcastOption(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastWithClientHandlerAndBroadcastOptionQF), "dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} diff --git a/cmd/protoc-gen-gorums/dev/zorums_clientserver_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_clientserver_gorums.pb.go new file mode 100644 index 00000000..9532e97e --- /dev/null +++ b/cmd/protoc-gen-gorums/dev/zorums_clientserver_gorums.pb.go @@ -0,0 +1,25 @@ +// Code generated by protoc-gen-gorums. DO NOT EDIT. +// versions: +// protoc-gen-gorums v0.7.0-devel +// protoc v3.12.4 +// source: zorums.proto + +package dev + +import ( + gorums "github.com/relab/gorums" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = gorums.EnforceVersion(7 - gorums.MinVersion) + // Verify that the gorums runtime is sufficiently up-to-date. + _ = gorums.EnforceVersion(gorums.MaxVersion - 7) +) + +func registerClientServerHandlers(srv *clientServerImpl) { + + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandler1", gorums.ClientHandler(srv.clientBroadcastWithClientHandler1)) + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandler2", gorums.ClientHandler(srv.clientBroadcastWithClientHandler2)) + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption", gorums.ClientHandler(srv.clientBroadcastWithClientHandlerAndBroadcastOption)) +} diff --git a/cmd/protoc-gen-gorums/dev/zorums_correctable_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_correctable_gorums.pb.go index 5cb8b8f4..611b4e03 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_correctable_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_correctable_gorums.pb.go @@ -1,16 +1,16 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( context "context" + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" protoreflect "google.golang.org/protobuf/reflect/protoreflect" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -110,9 +110,9 @@ func (c *Configuration) CorrectableEmpty(ctx context.Context, in *Request) *Corr ServerStream: false, } cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, int, bool) { - r := make(map[uint32]*emptypb.Empty, len(replies)) + r := make(map[uint32]*empty.Empty, len(replies)) for k, v := range replies { - r[k] = v.(*emptypb.Empty) + r[k] = v.(*empty.Empty) } return c.qspec.CorrectableEmptyQF(req.(*Request), r) } @@ -123,7 +123,7 @@ func (c *Configuration) CorrectableEmpty(ctx context.Context, in *Request) *Corr // CorrectableEmpty2 for testing imported message type; with same return // type as Correctable: Response. -func (c *Configuration) CorrectableEmpty2(ctx context.Context, in *emptypb.Empty) *CorrectableResponse { +func (c *Configuration) CorrectableEmpty2(ctx context.Context, in *empty.Empty) *CorrectableResponse { cd := gorums.CorrectableCallData{ Message: in, Method: "dev.ZorumsService.CorrectableEmpty2", @@ -134,7 +134,7 @@ func (c *Configuration) CorrectableEmpty2(ctx context.Context, in *emptypb.Empty for k, v := range replies { r[k] = v.(*Response) } - return c.qspec.CorrectableEmpty2QF(req.(*emptypb.Empty), r) + return c.qspec.CorrectableEmpty2QF(req.(*empty.Empty), r) } corr := c.RawConfiguration.CorrectableCall(ctx, cd) @@ -231,9 +231,9 @@ func (c *Configuration) CorrectableStreamEmpty(ctx context.Context, in *Request) ServerStream: true, } cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, int, bool) { - r := make(map[uint32]*emptypb.Empty, len(replies)) + r := make(map[uint32]*empty.Empty, len(replies)) for k, v := range replies { - r[k] = v.(*emptypb.Empty) + r[k] = v.(*empty.Empty) } return c.qspec.CorrectableStreamEmptyQF(req.(*Request), r) } @@ -244,7 +244,7 @@ func (c *Configuration) CorrectableStreamEmpty(ctx context.Context, in *Request) // CorrectableEmpty2 for testing imported message type; with same return // type as Correctable: Response. -func (c *Configuration) CorrectableStreamEmpty2(ctx context.Context, in *emptypb.Empty) *CorrectableStreamResponse { +func (c *Configuration) CorrectableStreamEmpty2(ctx context.Context, in *empty.Empty) *CorrectableStreamResponse { cd := gorums.CorrectableCallData{ Message: in, Method: "dev.ZorumsService.CorrectableStreamEmpty2", @@ -255,7 +255,7 @@ func (c *Configuration) CorrectableStreamEmpty2(ctx context.Context, in *emptypb for k, v := range replies { r[k] = v.(*Response) } - return c.qspec.CorrectableStreamEmpty2QF(req.(*emptypb.Empty), r) + return c.qspec.CorrectableStreamEmpty2QF(req.(*empty.Empty), r) } corr := c.RawConfiguration.CorrectableCall(ctx, cd) diff --git a/cmd/protoc-gen-gorums/dev/zorums_multicast_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_multicast_gorums.pb.go index 005a7c15..20037233 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_multicast_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_multicast_gorums.pb.go @@ -1,16 +1,16 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( context "context" + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" protoreflect "google.golang.org/protobuf/reflect/protoreflect" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -20,6 +20,21 @@ const ( _ = gorums.EnforceVersion(gorums.MaxVersion - 7) ) +// Reference imports to suppress errors if they are not otherwise used. +var _ empty.Empty + +// Multicast with broadcast option enables the server handler to broadcast. +// The handler still works like a regular QuorumCall from the client's +// perpective. +func (c *Configuration) MulticastWithBroadcast(ctx context.Context, in *Request, opts ...gorums.CallOption) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "dev.ZorumsService.MulticastWithBroadcast", + } + + c.RawConfiguration.Multicast(ctx, cd, opts...) +} + // Multicast plain. Response type is not needed here. func (c *Configuration) Multicast(ctx context.Context, in *Request, opts ...gorums.CallOption) { cd := gorums.QuorumCallData{ @@ -55,7 +70,7 @@ func (c *Configuration) Multicast2(ctx context.Context, in *Request, opts ...gor } // Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty +var _ empty.Empty // Multicast3 is testing imported message type. func (c *Configuration) Multicast3(ctx context.Context, in *Request, opts ...gorums.CallOption) { @@ -68,10 +83,10 @@ func (c *Configuration) Multicast3(ctx context.Context, in *Request, opts ...gor } // Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty +var _ empty.Empty // Multicast4 is testing imported message type. -func (c *Configuration) Multicast4(ctx context.Context, in *emptypb.Empty, opts ...gorums.CallOption) { +func (c *Configuration) Multicast4(ctx context.Context, in *empty.Empty, opts ...gorums.CallOption) { cd := gorums.QuorumCallData{ Message: in, Method: "dev.ZorumsService.Multicast4", diff --git a/cmd/protoc-gen-gorums/dev/zorums_qspec_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_qspec_gorums.pb.go index 6b71a306..151cf810 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_qspec_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_qspec_gorums.pb.go @@ -1,14 +1,14 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -54,15 +54,43 @@ type QuorumSpec interface { // quorum call method. The in parameter is the request object // supplied to the QuorumCallEmpty method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed - // you should implement your quorum function with '_ *emptypb.Empty'. - QuorumCallEmptyQF(in *emptypb.Empty, replies map[uint32]*Response) (*Response, bool) + // you should implement your quorum function with '_ *empty.Empty'. + QuorumCallEmptyQF(in *empty.Empty, replies map[uint32]*Response) (*Response, bool) // QuorumCallEmpty2QF is the quorum function for the QuorumCallEmpty2 // quorum call method. The in parameter is the request object // supplied to the QuorumCallEmpty2 method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed // you should implement your quorum function with '_ *Request'. - QuorumCallEmpty2QF(in *Request, replies map[uint32]*emptypb.Empty) (*emptypb.Empty, bool) + QuorumCallEmpty2QF(in *Request, replies map[uint32]*empty.Empty) (*empty.Empty, bool) + + // QuorumCallWithBroadcastQF is the quorum function for the QuorumCallWithBroadcast + // broadcast call method. The in parameter is the request object + // supplied to the QuorumCallWithBroadcast method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + QuorumCallWithBroadcastQF(in *Request, replies map[uint32]*Response) (*Response, bool) + + // BroadcastWithClientHandler1QF is the quorum function for the BroadcastWithClientHandler1 + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastWithClientHandler1 method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastWithClientHandler1QF(in *Request, replies []*Response) (*Response, bool) + + // BroadcastWithClientHandler2QF is the quorum function for the BroadcastWithClientHandler2 + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastWithClientHandler2 method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastWithClientHandler2QF(in *Request, replies []*Response) (*Response, bool) + + // BroadcastWithClientHandlerAndBroadcastOptionQF is the quorum function for the BroadcastWithClientHandlerAndBroadcastOption + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastWithClientHandlerAndBroadcastOption method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastWithClientHandlerAndBroadcastOptionQF(in *Request, replies []*Response) (*Response, bool) // QuorumCallAsyncQF is the quorum function for the QuorumCallAsync // asynchronous quorum call method. The in parameter is the request object @@ -104,14 +132,14 @@ type QuorumSpec interface { // supplied to the QuorumCallAsyncEmpty method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed // you should implement your quorum function with '_ *Request'. - QuorumCallAsyncEmptyQF(in *Request, replies map[uint32]*emptypb.Empty) (*emptypb.Empty, bool) + QuorumCallAsyncEmptyQF(in *Request, replies map[uint32]*empty.Empty) (*empty.Empty, bool) // QuorumCallAsyncEmpty2QF is the quorum function for the QuorumCallAsyncEmpty2 // asynchronous quorum call method. The in parameter is the request object // supplied to the QuorumCallAsyncEmpty2 method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed - // you should implement your quorum function with '_ *emptypb.Empty'. - QuorumCallAsyncEmpty2QF(in *emptypb.Empty, replies map[uint32]*Response) (*Response, bool) + // you should implement your quorum function with '_ *empty.Empty'. + QuorumCallAsyncEmpty2QF(in *empty.Empty, replies map[uint32]*Response) (*Response, bool) // CorrectableQF is the quorum function for the Correctable // correctable quorum call method. The in parameter is the request object @@ -146,14 +174,14 @@ type QuorumSpec interface { // supplied to the CorrectableEmpty method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed // you should implement your quorum function with '_ *Request'. - CorrectableEmptyQF(in *Request, replies map[uint32]*emptypb.Empty) (*emptypb.Empty, int, bool) + CorrectableEmptyQF(in *Request, replies map[uint32]*empty.Empty) (*empty.Empty, int, bool) // CorrectableEmpty2QF is the quorum function for the CorrectableEmpty2 // correctable quorum call method. The in parameter is the request object // supplied to the CorrectableEmpty2 method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed - // you should implement your quorum function with '_ *emptypb.Empty'. - CorrectableEmpty2QF(in *emptypb.Empty, replies map[uint32]*Response) (*Response, int, bool) + // you should implement your quorum function with '_ *empty.Empty'. + CorrectableEmpty2QF(in *empty.Empty, replies map[uint32]*Response) (*Response, int, bool) // CorrectableStreamQF is the quorum function for the CorrectableStream // correctable stream quorum call method. The in parameter is the request object @@ -188,12 +216,12 @@ type QuorumSpec interface { // supplied to the CorrectableStreamEmpty method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed // you should implement your quorum function with '_ *Request'. - CorrectableStreamEmptyQF(in *Request, replies map[uint32]*emptypb.Empty) (*emptypb.Empty, int, bool) + CorrectableStreamEmptyQF(in *Request, replies map[uint32]*empty.Empty) (*empty.Empty, int, bool) // CorrectableStreamEmpty2QF is the quorum function for the CorrectableStreamEmpty2 // correctable stream quorum call method. The in parameter is the request object // supplied to the CorrectableStreamEmpty2 method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed - // you should implement your quorum function with '_ *emptypb.Empty'. - CorrectableStreamEmpty2QF(in *emptypb.Empty, replies map[uint32]*Response) (*Response, int, bool) + // you should implement your quorum function with '_ *empty.Empty'. + CorrectableStreamEmpty2QF(in *empty.Empty, replies map[uint32]*Response) (*Response, int, bool) } diff --git a/cmd/protoc-gen-gorums/dev/zorums_quorumcall_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_quorumcall_gorums.pb.go index 0d4baa37..8f60fd83 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_quorumcall_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_quorumcall_gorums.pb.go @@ -1,16 +1,16 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( context "context" + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" protoreflect "google.golang.org/protobuf/reflect/protoreflect" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -111,7 +111,7 @@ func (c *Configuration) QuorumCallCombo(ctx context.Context, in *Request, f func } // QuorumCallEmpty for testing imported message type. -func (c *Configuration) QuorumCallEmpty(ctx context.Context, in *emptypb.Empty) (resp *Response, err error) { +func (c *Configuration) QuorumCallEmpty(ctx context.Context, in *empty.Empty) (resp *Response, err error) { cd := gorums.QuorumCallData{ Message: in, Method: "dev.ZorumsService.QuorumCallEmpty", @@ -121,7 +121,7 @@ func (c *Configuration) QuorumCallEmpty(ctx context.Context, in *emptypb.Empty) for k, v := range replies { r[k] = v.(*Response) } - return c.qspec.QuorumCallEmptyQF(req.(*emptypb.Empty), r) + return c.qspec.QuorumCallEmptyQF(req.(*empty.Empty), r) } res, err := c.RawConfiguration.QuorumCall(ctx, cd) @@ -132,15 +132,15 @@ func (c *Configuration) QuorumCallEmpty(ctx context.Context, in *emptypb.Empty) } // QuorumCallEmpty2 for testing imported message type. -func (c *Configuration) QuorumCallEmpty2(ctx context.Context, in *Request) (resp *emptypb.Empty, err error) { +func (c *Configuration) QuorumCallEmpty2(ctx context.Context, in *Request) (resp *empty.Empty, err error) { cd := gorums.QuorumCallData{ Message: in, Method: "dev.ZorumsService.QuorumCallEmpty2", } cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { - r := make(map[uint32]*emptypb.Empty, len(replies)) + r := make(map[uint32]*empty.Empty, len(replies)) for k, v := range replies { - r[k] = v.(*emptypb.Empty) + r[k] = v.(*empty.Empty) } return c.qspec.QuorumCallEmpty2QF(req.(*Request), r) } @@ -149,5 +149,31 @@ func (c *Configuration) QuorumCallEmpty2(ctx context.Context, in *Request) (resp if err != nil { return nil, err } - return res.(*emptypb.Empty), err + return res.(*empty.Empty), err +} + +// QuorumCall with broadcast option enables the server handler to broadcast. +// The handler still works like a regular QuorumCall from the client's +// perpective. +func (c *Configuration) QuorumCallWithBroadcast(ctx context.Context, in *Request) (resp *Response, err error) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "dev.ZorumsService.QuorumCallWithBroadcast", + + BroadcastID: c.snowflake.NewBroadcastID(), + IsBroadcastClient: true, + } + cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + r := make(map[uint32]*Response, len(replies)) + for k, v := range replies { + r[k] = v.(*Response) + } + return c.qspec.QuorumCallWithBroadcastQF(req.(*Request), r) + } + + res, err := c.RawConfiguration.QuorumCall(ctx, cd) + if err != nil { + return nil, err + } + return res.(*Response), err } diff --git a/cmd/protoc-gen-gorums/dev/zorums_rpc_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_rpc_gorums.pb.go index 4e57c69e..32a1b77d 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_rpc_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_rpc_gorums.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev diff --git a/cmd/protoc-gen-gorums/dev/zorums_server_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_server_gorums.pb.go index 6feaaef2..5b1d9b01 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_server_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_server_gorums.pb.go @@ -1,16 +1,18 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" ordering "github.com/relab/gorums/ordering" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" proto "google.golang.org/protobuf/proto" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -27,37 +29,161 @@ type ZorumsService interface { QuorumCallPerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) - QuorumCallEmpty(ctx gorums.ServerCtx, request *emptypb.Empty) (response *Response, err error) - QuorumCallEmpty2(ctx gorums.ServerCtx, request *Request) (response *emptypb.Empty, err error) + QuorumCallEmpty(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) + QuorumCallEmpty2(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) + QuorumCallWithBroadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + MulticastWithBroadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastInternal(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastWithClientHandler1(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastWithClientHandler2(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastWithClientHandlerAndBroadcastOption(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) Multicast(ctx gorums.ServerCtx, request *Request) MulticastPerNodeArg(ctx gorums.ServerCtx, request *Request) Multicast2(ctx gorums.ServerCtx, request *Request) Multicast3(ctx gorums.ServerCtx, request *Request) - Multicast4(ctx gorums.ServerCtx, request *emptypb.Empty) + Multicast4(ctx gorums.ServerCtx, request *empty.Empty) QuorumCallAsync(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallAsyncPerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallAsyncCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallAsyncCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) QuorumCallAsync2(ctx gorums.ServerCtx, request *Request) (response *Response, err error) - QuorumCallAsyncEmpty(ctx gorums.ServerCtx, request *Request) (response *emptypb.Empty, err error) - QuorumCallAsyncEmpty2(ctx gorums.ServerCtx, request *emptypb.Empty) (response *Response, err error) + QuorumCallAsyncEmpty(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) + QuorumCallAsyncEmpty2(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) Correctable(ctx gorums.ServerCtx, request *Request) (response *Response, err error) CorrectablePerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) CorrectableCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) CorrectableCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) - CorrectableEmpty(ctx gorums.ServerCtx, request *Request) (response *emptypb.Empty, err error) - CorrectableEmpty2(ctx gorums.ServerCtx, request *emptypb.Empty) (response *Response, err error) + CorrectableEmpty(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) + CorrectableEmpty2(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) CorrectableStream(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error CorrectableStreamPerNodeArg(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error CorrectableStreamCustomReturnType(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error CorrectableStreamCombo(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error - CorrectableStreamEmpty(ctx gorums.ServerCtx, request *Request, send func(response *emptypb.Empty) error) error - CorrectableStreamEmpty2(ctx gorums.ServerCtx, request *emptypb.Empty, send func(response *Response) error) error + CorrectableStreamEmpty(ctx gorums.ServerCtx, request *Request, send func(response *empty.Empty) error) error + CorrectableStreamEmpty2(ctx gorums.ServerCtx, request *empty.Empty, send func(response *Response) error) error Unicast(ctx gorums.ServerCtx, request *Request) Unicast2(ctx gorums.ServerCtx, request *Request) } -func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { +func (srv *Server) GRPCCall(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method GRPCCall not implemented")) +} +func (srv *Server) QuorumCall(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCall not implemented")) +} +func (srv *Server) QuorumCallPerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallPerNodeArg not implemented")) +} +func (srv *Server) QuorumCallCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallCustomReturnType not implemented")) +} +func (srv *Server) QuorumCallCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallCombo not implemented")) +} +func (srv *Server) QuorumCallEmpty(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallEmpty not implemented")) +} +func (srv *Server) QuorumCallEmpty2(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallEmpty2 not implemented")) +} +func (srv *Server) QuorumCallWithBroadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallWithBroadcast not implemented")) +} +func (srv *Server) MulticastWithBroadcast(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method MulticastWithBroadcast not implemented")) +} +func (srv *Server) BroadcastInternal(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastInternal not implemented")) +} +func (srv *Server) BroadcastWithClientHandler1(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastWithClientHandler1 not implemented")) +} +func (srv *Server) BroadcastWithClientHandler2(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastWithClientHandler2 not implemented")) +} +func (srv *Server) BroadcastWithClientHandlerAndBroadcastOption(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastWithClientHandlerAndBroadcastOption not implemented")) +} +func (srv *Server) Multicast(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Multicast not implemented")) +} +func (srv *Server) MulticastPerNodeArg(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method MulticastPerNodeArg not implemented")) +} +func (srv *Server) Multicast2(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Multicast2 not implemented")) +} +func (srv *Server) Multicast3(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Multicast3 not implemented")) +} +func (srv *Server) Multicast4(ctx gorums.ServerCtx, request *empty.Empty) { + panic(status.Errorf(codes.Unimplemented, "method Multicast4 not implemented")) +} +func (srv *Server) QuorumCallAsync(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsync not implemented")) +} +func (srv *Server) QuorumCallAsyncPerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsyncPerNodeArg not implemented")) +} +func (srv *Server) QuorumCallAsyncCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsyncCustomReturnType not implemented")) +} +func (srv *Server) QuorumCallAsyncCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsyncCombo not implemented")) +} +func (srv *Server) QuorumCallAsync2(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsync2 not implemented")) +} +func (srv *Server) QuorumCallAsyncEmpty(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsyncEmpty not implemented")) +} +func (srv *Server) QuorumCallAsyncEmpty2(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallAsyncEmpty2 not implemented")) +} +func (srv *Server) Correctable(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method Correctable not implemented")) +} +func (srv *Server) CorrectablePerNodeArg(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method CorrectablePerNodeArg not implemented")) +} +func (srv *Server) CorrectableCustomReturnType(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method CorrectableCustomReturnType not implemented")) +} +func (srv *Server) CorrectableCombo(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method CorrectableCombo not implemented")) +} +func (srv *Server) CorrectableEmpty(ctx gorums.ServerCtx, request *Request) (response *empty.Empty, err error) { + panic(status.Errorf(codes.Unimplemented, "method CorrectableEmpty not implemented")) +} +func (srv *Server) CorrectableEmpty2(ctx gorums.ServerCtx, request *empty.Empty) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method CorrectableEmpty2 not implemented")) +} +func (srv *Server) CorrectableStream(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStream not implemented")) +} +func (srv *Server) CorrectableStreamPerNodeArg(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStreamPerNodeArg not implemented")) +} +func (srv *Server) CorrectableStreamCustomReturnType(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStreamCustomReturnType not implemented")) +} +func (srv *Server) CorrectableStreamCombo(ctx gorums.ServerCtx, request *Request, send func(response *Response) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStreamCombo not implemented")) +} +func (srv *Server) CorrectableStreamEmpty(ctx gorums.ServerCtx, request *Request, send func(response *empty.Empty) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStreamEmpty not implemented")) +} +func (srv *Server) CorrectableStreamEmpty2(ctx gorums.ServerCtx, request *empty.Empty, send func(response *Response) error) error { + panic(status.Errorf(codes.Unimplemented, "method CorrectableStreamEmpty2 not implemented")) +} +func (srv *Server) Unicast(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Unicast not implemented")) +} +func (srv *Server) Unicast2(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Unicast2 not implemented")) +} + +func RegisterZorumsServiceServer(srv *Server, impl ZorumsService) { srv.RegisterHandler("dev.ZorumsService.GRPCCall", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { req := in.Message.(*Request) defer ctx.Release() @@ -89,7 +215,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) }) srv.RegisterHandler("dev.ZorumsService.QuorumCallEmpty", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { - req := in.Message.(*emptypb.Empty) + req := in.Message.(*empty.Empty) defer ctx.Release() resp, err := impl.QuorumCallEmpty(ctx, req) gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) @@ -100,6 +226,15 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { resp, err := impl.QuorumCallEmpty2(ctx, req) gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) }) + srv.RegisterHandler("dev.ZorumsService.QuorumCallWithBroadcast", gorums.BroadcastHandler(impl.QuorumCallWithBroadcast, srv.Server)) + srv.RegisterHandler("dev.ZorumsService.MulticastWithBroadcast", gorums.BroadcastHandler(impl.MulticastWithBroadcast, srv.Server)) + srv.RegisterHandler("dev.ZorumsService.BroadcastInternal", gorums.BroadcastHandler(impl.BroadcastInternal, srv.Server)) + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandler1", gorums.BroadcastHandler(impl.BroadcastWithClientHandler1, srv.Server)) + srv.RegisterClientHandler("dev.ZorumsService.BroadcastWithClientHandler1") + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandler2", gorums.BroadcastHandler(impl.BroadcastWithClientHandler2, srv.Server)) + srv.RegisterClientHandler("dev.ZorumsService.BroadcastWithClientHandler2") + srv.RegisterHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption", gorums.BroadcastHandler(impl.BroadcastWithClientHandlerAndBroadcastOption, srv.Server)) + srv.RegisterClientHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption") srv.RegisterHandler("dev.ZorumsService.Multicast", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { req := in.Message.(*Request) defer ctx.Release() @@ -121,7 +256,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { impl.Multicast3(ctx, req) }) srv.RegisterHandler("dev.ZorumsService.Multicast4", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { - req := in.Message.(*emptypb.Empty) + req := in.Message.(*empty.Empty) defer ctx.Release() impl.Multicast4(ctx, req) }) @@ -162,7 +297,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) }) srv.RegisterHandler("dev.ZorumsService.QuorumCallAsyncEmpty2", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { - req := in.Message.(*emptypb.Empty) + req := in.Message.(*empty.Empty) defer ctx.Release() resp, err := impl.QuorumCallAsyncEmpty2(ctx, req) gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) @@ -198,7 +333,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) }) srv.RegisterHandler("dev.ZorumsService.CorrectableEmpty2", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { - req := in.Message.(*emptypb.Empty) + req := in.Message.(*empty.Empty) defer ctx.Release() resp, err := impl.CorrectableEmpty2(ctx, req) gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) @@ -254,7 +389,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { srv.RegisterHandler("dev.ZorumsService.CorrectableStreamEmpty", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { req := in.Message.(*Request) defer ctx.Release() - err := impl.CorrectableStreamEmpty(ctx, req, func(resp *emptypb.Empty) error { + err := impl.CorrectableStreamEmpty(ctx, req, func(resp *empty.Empty) error { // create a copy of the metadata, to avoid a data race between WrapMessage and SendMsg md := proto.Clone(in.Metadata) return gorums.SendMessage(ctx, finished, gorums.WrapMessage(md.(*ordering.Metadata), resp, nil)) @@ -264,7 +399,7 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { } }) srv.RegisterHandler("dev.ZorumsService.CorrectableStreamEmpty2", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { - req := in.Message.(*emptypb.Empty) + req := in.Message.(*empty.Empty) defer ctx.Release() err := impl.CorrectableStreamEmpty2(ctx, req, func(resp *Response) error { // create a copy of the metadata, to avoid a data race between WrapMessage and SendMsg @@ -285,4 +420,62 @@ func RegisterZorumsServiceServer(srv *gorums.Server, impl ZorumsService) { defer ctx.Release() impl.Unicast2(ctx, req) }) + srv.RegisterHandler(gorums.Cancellation, gorums.BroadcastHandler(gorums.CancelFunc, srv.Server)) +} + +func (srv *Server) BroadcastQuorumCallWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("dev.ZorumsService.QuorumCallWithBroadcast", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("dev.ZorumsService.QuorumCallWithBroadcast", req, options) + } +} + +func (srv *Server) BroadcastMulticastWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("dev.ZorumsService.MulticastWithBroadcast", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("dev.ZorumsService.MulticastWithBroadcast", req, options) + } +} + +func (srv *Server) BroadcastBroadcastInternal(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("dev.ZorumsService.BroadcastInternal", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("dev.ZorumsService.BroadcastInternal", req, options) + } +} + +func (srv *Server) BroadcastBroadcastWithClientHandlerAndBroadcastOption(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption", req, options) + } } + +const ( + ZorumsServiceQuorumCallWithBroadcast string = "dev.ZorumsService.QuorumCallWithBroadcast" + ZorumsServiceMulticastWithBroadcast string = "dev.ZorumsService.MulticastWithBroadcast" + ZorumsServiceBroadcastInternal string = "dev.ZorumsService.BroadcastInternal" + ZorumsServiceBroadcastWithClientHandler1 string = "dev.ZorumsService.BroadcastWithClientHandler1" + ZorumsServiceBroadcastWithClientHandler2 string = "dev.ZorumsService.BroadcastWithClientHandler2" + ZorumsServiceBroadcastWithClientHandlerAndBroadcastOption string = "dev.ZorumsService.BroadcastWithClientHandlerAndBroadcastOption" +) diff --git a/cmd/protoc-gen-gorums/dev/zorums_types_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_types_gorums.pb.go index 670dc138..1983654e 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_types_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_types_gorums.pb.go @@ -1,14 +1,14 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -20,7 +20,7 @@ const ( type internalEmpty struct { nid uint32 - reply *emptypb.Empty + reply *empty.Empty err error } @@ -37,12 +37,12 @@ type AsyncEmpty struct { // Get returns the reply and any error associated with the called method. // The method blocks until a reply or error is available. -func (f *AsyncEmpty) Get() (*emptypb.Empty, error) { +func (f *AsyncEmpty) Get() (*empty.Empty, error) { resp, err := f.Async.Get() if err != nil { return nil, err } - return resp.(*emptypb.Empty), err + return resp.(*empty.Empty), err } // AsyncMyResponse is a async object for processing replies. @@ -85,12 +85,12 @@ type CorrectableEmpty struct { // intermediate) reply or error is available. Level is set to LevelNotSet if no // reply has yet been received. The Done or Watch methods should be used to // ensure that a reply is available. -func (c *CorrectableEmpty) Get() (*emptypb.Empty, int, error) { +func (c *CorrectableEmpty) Get() (*empty.Empty, int, error) { resp, level, err := c.Correctable.Get() if err != nil { return nil, level, err } - return resp.(*emptypb.Empty), level, err + return resp.(*empty.Empty), level, err } // CorrectableMyResponse is a correctable object for processing replies. @@ -139,12 +139,12 @@ type CorrectableStreamEmpty struct { // intermediate) reply or error is available. Level is set to LevelNotSet if no // reply has yet been received. The Done or Watch methods should be used to // ensure that a reply is available. -func (c *CorrectableStreamEmpty) Get() (*emptypb.Empty, int, error) { +func (c *CorrectableStreamEmpty) Get() (*empty.Empty, int, error) { resp, level, err := c.Correctable.Get() if err != nil { return nil, level, err } - return resp.(*emptypb.Empty), level, err + return resp.(*empty.Empty), level, err } // CorrectableStreamMyResponse is a correctable object for processing replies. diff --git a/cmd/protoc-gen-gorums/dev/zorums_unicast_gorums.pb.go b/cmd/protoc-gen-gorums/dev/zorums_unicast_gorums.pb.go index 0d9b4b33..e5291e85 100644 --- a/cmd/protoc-gen-gorums/dev/zorums_unicast_gorums.pb.go +++ b/cmd/protoc-gen-gorums/dev/zorums_unicast_gorums.pb.go @@ -1,15 +1,15 @@ // Code generated by protoc-gen-gorums. DO NOT EDIT. // versions: // protoc-gen-gorums v0.7.0-devel -// protoc v4.25.3 +// protoc v3.12.4 // source: zorums.proto package dev import ( context "context" + empty "github.com/golang/protobuf/ptypes/empty" gorums "github.com/relab/gorums" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) const ( @@ -31,7 +31,7 @@ func (n *Node) Unicast(ctx context.Context, in *Request, opts ...gorums.CallOpti } // Reference imports to suppress errors if they are not otherwise used. -var _ emptypb.Empty +var _ empty.Empty // Unicast2 is a quorum call invoked on all nodes in configuration c, // with the same argument in, and returns a combined result. diff --git a/cmd/protoc-gen-gorums/gengorums/gorums.go b/cmd/protoc-gen-gorums/gengorums/gorums.go index 16ced0f1..51bd26cf 100644 --- a/cmd/protoc-gen-gorums/gengorums/gorums.go +++ b/cmd/protoc-gen-gorums/gengorums/gorums.go @@ -239,16 +239,17 @@ func callTypeName(ext *protoimpl.ExtensionInfo) string { // The entries in this map is used to generate dev/zorums_{type}.pb.go // files for the different keys. var gorumsCallTypesInfo = map[string]*callTypeInfo{ - "qspec": {template: qspecInterface}, - "types": {template: dataTypes}, - "server": {template: server}, + "qspec": {template: qspecInterface}, + "types": {template: dataTypes}, + "server": {template: server}, + "clientserver": {template: clientServer}, callTypeName(gorums.E_Rpc): { extInfo: gorums.E_Rpc, docName: "rpc", template: rpcCall, chkFn: func(m *protogen.Method) bool { - return !hasMethodOption(m, gorumsCallTypes...) + return !hasMethodOption(m, gorumsCallTypes...) && !hasMethodOption(m, gorums.E_Broadcast) }, }, callTypeName(gorums.E_Quorumcall): { @@ -310,6 +311,22 @@ var gorumsCallTypesInfo = map[string]*callTypeInfo{ return hasMethodOption(m, gorums.E_Unicast) }, }, + callTypeName(gorums.E_Broadcast): { + extInfo: gorums.E_Broadcast, + docName: "broadcast", + template: broadcastMethod, + chkFn: func(m *protogen.Method) bool { + return hasMethodOption(m, gorums.E_Broadcast) + }, + }, + callTypeName(gorums.E_Broadcastcall): { + extInfo: gorums.E_Broadcastcall, + docName: "broadcastcall", + template: broadcastCall, + chkFn: func(m *protogen.Method) bool { + return hasMethodOption(m, gorums.E_Broadcastcall) + }, + }, } // gorumsCallTypes should list all available call types supported by Gorums. @@ -320,6 +337,7 @@ var gorumsCallTypes = []*protoimpl.ExtensionInfo{ gorums.E_Correctable, gorums.E_Multicast, gorums.E_Unicast, + gorums.E_Broadcastcall, } // callTypesWithInternal should list all available call types that diff --git a/cmd/protoc-gen-gorums/gengorums/gorums_func_map.go b/cmd/protoc-gen-gorums/gengorums/gorums_func_map.go index b729a430..2da7954b 100644 --- a/cmd/protoc-gen-gorums/gengorums/gorums_func_map.go +++ b/cmd/protoc-gen-gorums/gengorums/gorums_func_map.go @@ -33,6 +33,8 @@ var importMap = map[string]protogen.GoImportPath{ "gorums": protogen.GoImportPath("github.com/relab/gorums"), "ordering": protogen.GoImportPath("github.com/relab/gorums/ordering"), "protoreflect": protogen.GoImportPath("google.golang.org/protobuf/reflect/protoreflect"), + "uuid": protogen.GoImportPath("github.com/google/uuid"), + "metadata": protogen.GoImportPath("google.golang.org/grpc/metadata"), } func addImport(path, ident string, g *protogen.GeneratedFile) string { @@ -106,6 +108,15 @@ var funcMap = template.FuncMap{ "isOneway": func(method *protogen.Method) bool { return hasMethodOption(method, gorums.E_Multicast, gorums.E_Unicast) }, + "isBroadcast": func(method *protogen.Method) bool { + return hasMethodOption(method, gorums.E_Broadcast, gorums.E_Broadcastcall) + }, + "isBroadcastOption": func(method *protogen.Method) bool { + return hasMethodOption(method, gorums.E_Broadcast) + }, + "isBroadcastCall": func(method *protogen.Method) bool { + return hasMethodOption(method, gorums.E_Broadcastcall) + }, "methods": func(services []*protogen.Service) (methods []*protogen.Method) { for _, s := range services { methods = append(methods, s.Methods...) diff --git a/cmd/protoc-gen-gorums/gengorums/template_broadcast.go b/cmd/protoc-gen-gorums/gengorums/template_broadcast.go new file mode 100644 index 00000000..42065a00 --- /dev/null +++ b/cmd/protoc-gen-gorums/gengorums/template_broadcast.go @@ -0,0 +1,23 @@ +package gengorums + +var broadcastVar = ` +{{$callData := use "gorums.CallData" .GenFile}} +` + +var broadcastSignature = `func (b *Broadcast) {{.Method.GoName}}(req *{{in .GenFile .Method}}, opts... gorums.BroadcastOption) {` + +var broadcastBody = ` + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.Broadcast{{.Method.GoName}} instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("{{.Method.Desc.FullName}}", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} +` + +var broadcastMethod = broadcastVar + + broadcastSignature + broadcastBody diff --git a/cmd/protoc-gen-gorums/gengorums/template_broadcastcall.go b/cmd/protoc-gen-gorums/gengorums/template_broadcastcall.go new file mode 100644 index 00000000..6bf6ec0a --- /dev/null +++ b/cmd/protoc-gen-gorums/gengorums/template_broadcastcall.go @@ -0,0 +1,70 @@ +package gengorums + +var clientServerVar = ` +{{$callData := use "gorums.CallData" .GenFile}} +{{$context := use "context.Context" .GenFile}} +{{$fmt := use "fmt.FMT" .GenFile}} +{{$time := use "time.TIME" .GenFile}} +{{$protoMessage := use "protoreflect.ProtoMessage" .GenFile}} +` + +var clientServerMethodImpl = ` +func (srv *clientServerImpl) client{{.Method.GoName}}(ctx context.Context, resp *{{out .GenFile .Method}}, broadcastID uint64) (*{{out .GenFile .Method}}, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +` + +var clientServerImplMethod = ` +func (c *Configuration) {{.Method.GoName}}(ctx context.Context, in *{{in .GenFile .Method}}, cancelOnTimeout... bool) (resp *{{out .GenFile .Method}}, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.{{.Method.GoName}}QF), "{{.Method.Desc.FullName}}") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*{{out .GenFile .Method}}) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} +` + +var broadcastCall = clientServerVar + clientServerMethodImpl + clientServerImplMethod diff --git a/cmd/protoc-gen-gorums/gengorums/template_clientserver.go b/cmd/protoc-gen-gorums/gengorums/template_clientserver.go new file mode 100644 index 00000000..db752563 --- /dev/null +++ b/cmd/protoc-gen-gorums/gengorums/template_clientserver.go @@ -0,0 +1,20 @@ +package gengorums + +var clientServerVariables = ` +{{$callData := use "gorums.CallData" .GenFile}} +` + +var clientServerHandlers = ` +{{$genFile := .GenFile}} +func registerClientServerHandlers(srv *clientServerImpl) { + {{range .Services -}} + {{- range .Methods}} + {{- if isBroadcastCall .}} + srv.RegisterHandler("{{.Desc.FullName}}", gorums.ClientHandler(srv.client{{.GoName}})) + {{- end}} + {{- end}} + {{- end}} +} +` + +var clientServer = clientServerVariables + clientServerHandlers diff --git a/cmd/protoc-gen-gorums/gengorums/template_qspec.go b/cmd/protoc-gen-gorums/gengorums/template_qspec.go index 71537829..e387247e 100644 --- a/cmd/protoc-gen-gorums/gengorums/template_qspec.go +++ b/cmd/protoc-gen-gorums/gengorums/template_qspec.go @@ -24,7 +24,11 @@ type QuorumSpec interface { // supplied to the {{$method}} method at call time, and may or may not // be used by the quorum function. If the in parameter is not needed // you should implement your quorum function with '_ *{{$in}}'. + {{- if isBroadcastCall .}} + {{.GoName}}QF(in *{{$in}}, replies []*{{$out}}) (*{{$out}}, bool) + {{- else}} {{$method}}QF(in *{{$in}}, replies map[uint32]*{{$out}}) (*{{$customOut}}{{withCorrectable . ", int"}}, bool) + {{- end}} {{end}} } {{end}} diff --git a/cmd/protoc-gen-gorums/gengorums/template_quorumcall.go b/cmd/protoc-gen-gorums/gengorums/template_quorumcall.go index 7fd40858..07e786b6 100644 --- a/cmd/protoc-gen-gorums/gengorums/template_quorumcall.go +++ b/cmd/protoc-gen-gorums/gengorums/template_quorumcall.go @@ -47,6 +47,10 @@ var qcVar = ` var quorumCallBody = ` cd := {{$callData}}{ Message: in, Method: "{{$fullName}}", + {{if isBroadcast .Method}} + BroadcastID: c.snowflake.NewBroadcastID(), + IsBroadcastClient: true, + {{end -}} } cd.QuorumFunction = func(req {{$protoMessage}}, replies map[uint32]{{$protoMessage}}) ({{$protoMessage}}, bool) { r := make(map[uint32]*{{$out}}, len(replies)) diff --git a/cmd/protoc-gen-gorums/gengorums/template_server.go b/cmd/protoc-gen-gorums/gengorums/template_server.go index b6bd0e66..497ad740 100644 --- a/cmd/protoc-gen-gorums/gengorums/template_server.go +++ b/cmd/protoc-gen-gorums/gengorums/template_server.go @@ -2,6 +2,8 @@ package gengorums var serverVariables = ` {{$context := use "gorums.ServerCtx" .GenFile}} +{{$codes := use "codes.Code" .GenFile}} +{{$status := use "status.Status" .GenFile}} ` var serverInterface = ` @@ -11,10 +13,12 @@ var serverInterface = ` // {{$service}} is the server-side API for the {{$service}} Service type {{$service}} interface { {{- range .Methods}} - {{- if isOneway .}} - {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}) + {{- if isBroadcast .}} + {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}, broadcast *Broadcast) {{- else if correctableStream .}} {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}, send func(response *{{out $genFile .}}) error) error + {{- else if isOneway .}} + {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}) {{- else}} {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}) (response *{{out $genFile .}}, err error) {{- end}} @@ -23,6 +27,33 @@ type {{$service}} interface { {{- end}} ` +var registerServerMethods = ` +{{$genFile := .GenFile}} + +{{range .Services -}} +{{$service := .GoName}} +{{- range .Methods}} +{{- if isOneway .}} +func (srv *Server) {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}) { + panic(status.Errorf(codes.Unimplemented, "method {{.GoName}} not implemented")) +} +{{- else if correctableStream .}} +func (srv *Server) {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}, send func(response *{{out $genFile .}}) error) error { + panic(status.Errorf(codes.Unimplemented, "method {{.GoName}} not implemented")) +} +{{- else if isBroadcast .}} +func (srv *Server) {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method {{.GoName}} not implemented")) +} +{{- else}} +func (srv *Server) {{.GoName}}(ctx {{$context}}, request *{{in $genFile .}}) (response *{{out $genFile .}}, err error) { + panic(status.Errorf(codes.Unimplemented, "method {{.GoName}} not implemented")) +} +{{- end}} +{{- end}} +{{- end}} +` + var registerInterface = ` {{$genFile := .GenFile}} {{$gorumsMessage := use "gorums.Message" .GenFile}} @@ -30,8 +61,14 @@ var registerInterface = ` {{$sendMessage := use "gorums.SendMessage" $genFile}} {{range .Services -}} {{$service := .GoName}} -func Register{{$service}}Server(srv *{{use "gorums.Server" $genFile}}, impl {{$service}}) { +func Register{{$service}}Server(srv *Server, impl {{$service}}) { {{- range .Methods}} + {{- if isBroadcast .}} + srv.RegisterHandler("{{.Desc.FullName}}", gorums.BroadcastHandler(impl.{{.GoName}}, srv.Server)) + {{- if isBroadcastCall .}} + srv.RegisterClientHandler("{{.Desc.FullName}}") + {{- end}} + {{- else }} srv.RegisterHandler("{{.Desc.FullName}}", func(ctx {{$context}}, in *{{$gorumsMessage}}, {{if isOneway .}} _ {{- else}} finished {{- end}} chan<- *{{$gorumsMessage}}) { req := in.Message.(*{{in $genFile .}}) defer ctx.Release() @@ -52,8 +89,54 @@ func Register{{$service}}Server(srv *{{use "gorums.Server" $genFile}}, impl {{$s {{- end}} }) {{- end}} + {{- end}} + srv.RegisterHandler(gorums.Cancellation, gorums.BroadcastHandler(gorums.CancelFunc, srv.Server)) } {{- end}} ` -var server = serverVariables + serverInterface + registerInterface +var registerServerBroadcast = ` +{{$genFile := .GenFile}} + +{{range .Services -}} +{{$service := .GoName}} +{{- range .Methods}} +{{- if isBroadcastOption .}} + +func (srv *Server) Broadcast{{.GoName}}(req *{{in $genFile .}}, opts... gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("{{.Desc.FullName}}", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("{{.Desc.FullName}}", req, options) + } +} + +{{- end}} +{{- end}} +{{- end}} +` + +var registerMethodConstants = ` +{{$genFile := .GenFile}} +{{$gorumsMessage := use "gorums.Message" .GenFile}} +const ( +{{range .Services -}} +{{$service := .GoName}} +{{- range .Methods}} +{{- if isBroadcast .}} + {{$service}}{{.GoName}} string = "{{.Desc.FullName}}" +{{- else }} + {{- if isBroadcastCall .}} + {{$service}}{{.GoName}} string = "{{.Desc.FullName}}" + {{- end}} +{{- end}} +{{- end}} +{{- end}} +) +` + +var server = serverVariables + serverInterface + registerServerMethods + registerInterface + registerServerBroadcast + registerMethodConstants diff --git a/cmd/protoc-gen-gorums/gengorums/template_static.go b/cmd/protoc-gen-gorums/gengorums/template_static.go index 0bbcd5ef..88ebc0e8 100644 --- a/cmd/protoc-gen-gorums/gengorums/template_static.go +++ b/cmd/protoc-gen-gorums/gengorums/template_static.go @@ -5,18 +5,20 @@ package gengorums // pkgIdentMap maps from package name to one of the package's identifiers. // These identifiers are used by the Gorums protoc plugin to generate import statements. -var pkgIdentMap = map[string]string{"fmt": "Errorf", "github.com/relab/gorums": "ConfigOption", "google.golang.org/grpc/encoding": "GetCodec"} +var pkgIdentMap = map[string]string{"fmt": "Errorf", "github.com/relab/gorums": "BroadcastMetadata", "google.golang.org/grpc": "Server", "google.golang.org/grpc/encoding": "GetCodec", "google.golang.org/protobuf/reflect/protoreflect": "ProtoMessage", "net": "Addr"} // reservedIdents holds the set of Gorums reserved identifiers. // These identifiers cannot be used to define message types in a proto file. -var reservedIdents = []string{"Configuration", "Manager", "Node", "QuorumSpec"} +var reservedIdents = []string{"Broadcast", "Configuration", "Manager", "Node", "QuorumSpec", "Server"} var staticCode = `// A Configuration represents a static set of nodes on which quorum remote // procedure calls may be invoked. type Configuration struct { gorums.RawConfiguration - qspec QuorumSpec - nodes []*Node + qspec QuorumSpec + srv *clientServerImpl + snowflake gorums.Snowflake + nodes []*Node } // ConfigurationFromRaw returns a new Configuration from the given raw configuration and QuorumSpec. @@ -72,6 +74,7 @@ func init() { // which quorum calls can be performed. type Manager struct { *gorums.RawManager + srv *clientServerImpl } // NewManager returns a new Manager for managing connection to nodes added @@ -83,6 +86,35 @@ func NewManager(opts ...gorums.ManagerOption) *Manager { } } +func (mgr *Manager) Close() { + if mgr.RawManager != nil { + mgr.RawManager.Close() + } + if mgr.srv != nil { + mgr.srv.stop() + } +} + +// AddClientServer starts a lightweight client-side server. This server only accepts responses +// to broadcast requests sent by the client. +// +// It is important to provide the listenAddr because this will be used to advertise the IP the +// servers should reply back to. +func (mgr *Manager) AddClientServer(lis net.Listener, clientAddr net.Addr, opts ...gorums.ServerOption) error { + options := []gorums.ServerOption{gorums.WithListenAddr(clientAddr)} + options = append(options, opts...) + srv := gorums.NewClientServer(lis, options...) + srvImpl := &clientServerImpl{ + ClientServer: srv, + } + registerClientServerHandlers(srvImpl) + go func() { + _ = srvImpl.Serve(lis) + }() + mgr.srv = srvImpl + return nil +} + // NewConfiguration returns a configuration based on the provided list of nodes (required) // and an optional quorum specification. The QuorumSpec is necessary for call types that // must process replies. For configurations only used for unicast or multicast call types, @@ -109,11 +141,16 @@ func (m *Manager) NewConfiguration(opts ...gorums.ConfigOption) (c *Configuratio return nil, fmt.Errorf("config: unknown option type: %v", v) } } - // return an error if the QuorumSpec interface is not empty and no implementation was provided. - var test interface{} = struct{}{} - if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { - return nil, fmt.Errorf("config: missing required QuorumSpec") + // register the client server if it exists. + // used to collect responses in BroadcastCalls + if m.srv != nil { + c.srv = m.srv } + c.snowflake = m.Snowflake() + //var test interface{} = struct{}{} + //if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { + // return nil, fmt.Errorf("config: missing required QuorumSpec") + //} // initialize the nodes slice c.nodes = make([]*Node, c.Size()) for i, n := range c.RawConfiguration { @@ -139,4 +176,115 @@ type Node struct { *gorums.RawNode } +type Server struct { + *gorums.Server + broadcast *Broadcast + View *Configuration +} + +func NewServer(opts ...gorums.ServerOption) *Server { + srv := &Server{ + Server: gorums.NewServer(opts...), + } + b := &Broadcast{ + orchestrator: gorums.NewBroadcastOrchestrator(srv.Server), + } + srv.broadcast = b + srv.RegisterBroadcaster(newBroadcaster) + return srv +} + +func newBroadcaster(m gorums.BroadcastMetadata, o *gorums.BroadcastOrchestrator, e gorums.EnqueueBroadcast) gorums.Broadcaster { + return &Broadcast{ + orchestrator: o, + metadata: m, + srvAddrs: make([]string, 0), + enqueueBroadcast: e, + } +} + +func (srv *Server) SetView(config *Configuration) { + srv.View = config + srv.RegisterConfig(config.RawConfiguration) +} + +type Broadcast struct { + orchestrator *gorums.BroadcastOrchestrator + metadata gorums.BroadcastMetadata + srvAddrs []string + enqueueBroadcast gorums.EnqueueBroadcast +} + +// Returns a readonly struct of the metadata used in the broadcast. +// +// Note: Some of the data are equal across the cluster, such as BroadcastID. +// Other fields are local, such as SenderAddr. +func (b *Broadcast) GetMetadata() gorums.BroadcastMetadata { + return b.metadata +} + +type clientServerImpl struct { + *gorums.ClientServer + grpcServer *grpc.Server +} + +func (c *clientServerImpl) stop() { + c.ClientServer.Stop() + if c.grpcServer != nil { + c.grpcServer.Stop() + } +} + +func (b *Broadcast) To(addrs ...string) *Broadcast { + if len(addrs) <= 0 { + return b + } + b.srvAddrs = append(b.srvAddrs, addrs...) + return b +} + +func (b *Broadcast) Forward(req protoreflect.ProtoMessage, addr string) error { + if addr == "" { + return fmt.Errorf("cannot forward to empty addr, got: %s", addr) + } + if !b.metadata.IsBroadcastClient { + return fmt.Errorf("can only forward client requests") + } + go b.orchestrator.ForwardHandler(req, b.metadata.OriginMethod, b.metadata.BroadcastID, addr, b.metadata.OriginAddr) + return nil +} + +// Done signals the end of a broadcast request. It is necessary to call +// either Done() or SendToClient() to properly terminate a broadcast request +// and free up resources. Otherwise, it could cause poor performance. +func (b *Broadcast) Done() { + b.orchestrator.DoneHandler(b.metadata.BroadcastID, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (b *Broadcast) SendToClient(resp protoreflect.ProtoMessage, err error) error { + return b.orchestrator.SendToClientHandler(b.metadata.BroadcastID, resp, err, b.enqueueBroadcast) +} + +// Cancel is a non-destructive method call that will transmit a cancellation +// to all servers in the view. It will not stop the execution but will cause +// the given ServerCtx to be cancelled, making it possible to listen for +// cancellations. +// +// Could be used together with either SendToClient() or Done(). +func (b *Broadcast) Cancel() error { + return b.orchestrator.CancelHandler(b.metadata.BroadcastID, b.srvAddrs, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (srv *Server) SendToClient(resp protoreflect.ProtoMessage, err error, broadcastID uint64) error { + return srv.SendToClientHandler(resp, err, broadcastID, nil) +} + ` diff --git a/config.go b/config.go index 274836be..66b3e051 100644 --- a/config.go +++ b/config.go @@ -1,6 +1,10 @@ package gorums -import "fmt" +import ( + "fmt" + + "github.com/relab/gorums/ordering" +) // RawConfiguration represents a static set of nodes on which quorum calls may be invoked. // @@ -58,3 +62,50 @@ func (c RawConfiguration) Equal(b RawConfiguration) bool { func (c RawConfiguration) getMsgID() uint64 { return c[0].mgr.getMsgID() } + +func (c RawConfiguration) sign(msg *Message, signOrigin ...bool) { + if c[0].mgr.opts.auth != nil { + if len(signOrigin) > 0 && signOrigin[0] { + originMsg, err := c[0].mgr.opts.auth.EncodeMsg(msg.Message) + if err != nil { + panic(err) + } + digest := c[0].mgr.opts.auth.Hash(originMsg) + originSignature, err := c[0].mgr.opts.auth.Sign(originMsg) + if err != nil { + panic(err) + } + pubKey, err := c[0].mgr.opts.auth.EncodePublic() + if err != nil { + panic(err) + } + msg.Metadata.BroadcastMsg.OriginDigest = digest + msg.Metadata.BroadcastMsg.OriginPubKey = pubKey + msg.Metadata.BroadcastMsg.OriginSignature = originSignature + } + encodedMsg, err := c.encodeMsg(msg) + if err != nil { + panic(err) + } + signature, err := c[0].mgr.opts.auth.Sign(encodedMsg) + if err != nil { + panic(err) + } + msg.Metadata.AuthMsg.Signature = signature + } +} + +func (c RawConfiguration) encodeMsg(msg *Message) ([]byte, error) { + // we do not want to include the signature field in the signature + auth := c[0].mgr.opts.auth + pubKey, err := auth.EncodePublic() + if err != nil { + panic(err) + } + msg.Metadata.AuthMsg = &ordering.AuthMsg{ + PublicKey: pubKey, + Signature: nil, + Sender: auth.Addr(), + } + return auth.EncodeMsg(*msg) +} diff --git a/config_opts.go b/config_opts.go index 8a44af80..8b9af89f 100644 --- a/config_opts.go +++ b/config_opts.go @@ -2,6 +2,9 @@ package gorums import ( "fmt" + "sync" + + "github.com/relab/gorums/logging" ) // ConfigOption is a marker interface for options to NewConfiguration. @@ -55,6 +58,7 @@ func (o nodeList) newConfig(mgr *RawManager) (nodes RawConfiguration, err error) if len(o.addrsList) == 0 { return nil, fmt.Errorf("config: missing required node addresses") } + var wg sync.WaitGroup nodes = make(RawConfiguration, 0, len(o.addrsList)) for _, naddr := range o.addrsList { node, err := NewRawNode(naddr) @@ -62,14 +66,19 @@ func (o nodeList) newConfig(mgr *RawManager) (nodes RawConfiguration, err error) return nil, err } if n, found := mgr.Node(node.ID()); !found { - if err = mgr.AddNode(node); err != nil { - return nil, err - } + wg.Add(1) + go func() { + if err = mgr.AddNode(node); err != nil { + mgr.log("manager: failed to add (retrying later)", err, logging.NodeID(node.ID()), logging.NodeAddr(node.Address())) + } + wg.Done() + }() } else { node = n } nodes = append(nodes, node) } + wg.Wait() // Sort nodes to ensure deterministic iteration. OrderedBy(ID).Sort(mgr.nodes) OrderedBy(ID).Sort(nodes) diff --git a/encoding.go b/encoding.go index a1802d21..300ce75a 100644 --- a/encoding.go +++ b/encoding.go @@ -112,6 +112,12 @@ func (c Codec) gorumsUnmarshal(b []byte, msg *Message) (err error) { // get method descriptor from registry desc, err := protoregistry.GlobalFiles.FindDescriptorByName(protoreflect.FullName(msg.Metadata.Method)) if err != nil { + // Cancellation is a special method that is not specified in the proto file. + // it is used by the broadcast server to cancel a broadcast request and is + // the msg.Message is empty and does not need marshalling. + if msg.Metadata.Method == Cancellation { + return nil + } return err } methodDesc := desc.(protoreflect.MethodDescriptor) diff --git a/go.mod b/go.mod index 6a136d2b..c7345393 100644 --- a/go.mod +++ b/go.mod @@ -3,6 +3,7 @@ module github.com/relab/gorums go 1.22.1 require ( + github.com/golang/protobuf v1.5.4 github.com/google/go-cmp v0.6.0 golang.org/x/sync v0.6.0 golang.org/x/tools v0.19.0 @@ -13,7 +14,6 @@ require ( ) require ( - github.com/golang/protobuf v1.5.4 // indirect golang.org/x/mod v0.16.0 // indirect golang.org/x/net v0.22.0 // indirect golang.org/x/sys v0.18.0 // indirect diff --git a/gorums.pb.go b/gorums.pb.go index c4d807d2..b2bda1e4 100644 --- a/gorums.pb.go +++ b/gorums.pb.go @@ -61,6 +61,14 @@ var file_gorums_proto_extTypes = []protoimpl.ExtensionInfo{ Tag: "varint,50005,opt,name=correctable", Filename: "gorums.proto", }, + { + ExtendedType: (*descriptorpb.MethodOptions)(nil), + ExtensionType: (*bool)(nil), + Field: 50006, + Name: "gorums.broadcastcall", + Tag: "varint,50006,opt,name=broadcastcall", + Filename: "gorums.proto", + }, { ExtendedType: (*descriptorpb.MethodOptions)(nil), ExtensionType: (*bool)(nil), @@ -85,6 +93,14 @@ var file_gorums_proto_extTypes = []protoimpl.ExtensionInfo{ Tag: "bytes,50030,opt,name=custom_return_type", Filename: "gorums.proto", }, + { + ExtendedType: (*descriptorpb.MethodOptions)(nil), + ExtensionType: (*bool)(nil), + Field: 50050, + Name: "gorums.broadcast", + Tag: "varint,50050,opt,name=broadcast", + Filename: "gorums.proto", + }, } // Extension fields to descriptorpb.MethodOptions. @@ -101,14 +117,18 @@ var ( E_Quorumcall = &file_gorums_proto_extTypes[3] // optional bool correctable = 50005; E_Correctable = &file_gorums_proto_extTypes[4] + // optional bool broadcastcall = 50006; + E_Broadcastcall = &file_gorums_proto_extTypes[5] // options for call types // // optional bool async = 50010; - E_Async = &file_gorums_proto_extTypes[5] + E_Async = &file_gorums_proto_extTypes[6] // optional bool per_node_arg = 50020; - E_PerNodeArg = &file_gorums_proto_extTypes[6] + E_PerNodeArg = &file_gorums_proto_extTypes[7] // optional string custom_return_type = 50030; - E_CustomReturnType = &file_gorums_proto_extTypes[7] + E_CustomReturnType = &file_gorums_proto_extTypes[8] + // optional bool broadcast = 50050; + E_Broadcast = &file_gorums_proto_extTypes[9] ) var File_gorums_proto protoreflect.FileDescriptor @@ -136,41 +156,52 @@ var file_gorums_proto_rawDesc = []byte{ 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xd5, 0x86, 0x03, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x36, - 0x0a, 0x05, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xda, 0x86, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x05, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x3a, 0x42, 0x0a, 0x0c, 0x70, 0x65, 0x72, 0x5f, 0x6e, 0x6f, - 0x64, 0x65, 0x5f, 0x61, 0x72, 0x67, 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe4, 0x86, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, - 0x70, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, 0x72, 0x67, 0x3a, 0x4e, 0x0a, 0x12, 0x63, 0x75, - 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x72, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, - 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0xee, 0x86, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, - 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x42, 0x19, 0x5a, 0x17, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x62, 0x2f, 0x67, - 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x08, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x72, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x46, + 0x0a, 0x0d, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x63, 0x61, 0x6c, 0x6c, 0x12, + 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0xd6, 0x86, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x63, 0x61, 0x6c, 0x6c, 0x3a, 0x36, 0x0a, 0x05, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x12, + 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0xda, 0x86, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x61, 0x73, 0x79, 0x6e, 0x63, 0x3a, 0x42, + 0x0a, 0x0c, 0x70, 0x65, 0x72, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x61, 0x72, 0x67, 0x12, 0x1e, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe4, + 0x86, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x41, + 0x72, 0x67, 0x3a, 0x4e, 0x0a, 0x12, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x72, 0x65, 0x74, + 0x75, 0x72, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xee, 0x86, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x52, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, + 0x70, 0x65, 0x3a, 0x3e, 0x0a, 0x09, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, + 0x1e, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x82, 0x87, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x42, 0x19, 0x5a, 0x17, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, + 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x62, 0x2f, 0x67, 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_gorums_proto_goTypes = []interface{}{ (*descriptorpb.MethodOptions)(nil), // 0: google.protobuf.MethodOptions } var file_gorums_proto_depIdxs = []int32{ - 0, // 0: gorums.rpc:extendee -> google.protobuf.MethodOptions - 0, // 1: gorums.unicast:extendee -> google.protobuf.MethodOptions - 0, // 2: gorums.multicast:extendee -> google.protobuf.MethodOptions - 0, // 3: gorums.quorumcall:extendee -> google.protobuf.MethodOptions - 0, // 4: gorums.correctable:extendee -> google.protobuf.MethodOptions - 0, // 5: gorums.async:extendee -> google.protobuf.MethodOptions - 0, // 6: gorums.per_node_arg:extendee -> google.protobuf.MethodOptions - 0, // 7: gorums.custom_return_type:extendee -> google.protobuf.MethodOptions - 8, // [8:8] is the sub-list for method output_type - 8, // [8:8] is the sub-list for method input_type - 8, // [8:8] is the sub-list for extension type_name - 0, // [0:8] is the sub-list for extension extendee - 0, // [0:0] is the sub-list for field type_name + 0, // 0: gorums.rpc:extendee -> google.protobuf.MethodOptions + 0, // 1: gorums.unicast:extendee -> google.protobuf.MethodOptions + 0, // 2: gorums.multicast:extendee -> google.protobuf.MethodOptions + 0, // 3: gorums.quorumcall:extendee -> google.protobuf.MethodOptions + 0, // 4: gorums.correctable:extendee -> google.protobuf.MethodOptions + 0, // 5: gorums.broadcastcall:extendee -> google.protobuf.MethodOptions + 0, // 6: gorums.async:extendee -> google.protobuf.MethodOptions + 0, // 7: gorums.per_node_arg:extendee -> google.protobuf.MethodOptions + 0, // 8: gorums.custom_return_type:extendee -> google.protobuf.MethodOptions + 0, // 9: gorums.broadcast:extendee -> google.protobuf.MethodOptions + 10, // [10:10] is the sub-list for method output_type + 10, // [10:10] is the sub-list for method input_type + 10, // [10:10] is the sub-list for extension type_name + 0, // [0:10] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name } func init() { file_gorums_proto_init() } @@ -185,7 +216,7 @@ func file_gorums_proto_init() { RawDescriptor: file_gorums_proto_rawDesc, NumEnums: 0, NumMessages: 0, - NumExtensions: 8, + NumExtensions: 10, NumServices: 0, }, GoTypes: file_gorums_proto_goTypes, diff --git a/gorums.proto b/gorums.proto index f55b5c14..a82498d3 100644 --- a/gorums.proto +++ b/gorums.proto @@ -11,8 +11,10 @@ extend google.protobuf.MethodOptions { bool multicast = 50003; bool quorumcall = 50004; bool correctable = 50005; + bool broadcastcall = 50006; // options for call types bool async = 50010; bool per_node_arg = 50020; string custom_return_type = 50030; + bool broadcast = 50050; } diff --git a/handler.go b/handler.go new file mode 100644 index 00000000..df38c2f7 --- /dev/null +++ b/handler.go @@ -0,0 +1,235 @@ +package gorums + +import ( + "context" + "fmt" + + "github.com/relab/gorums/broadcast" + "github.com/relab/gorums/ordering" + "google.golang.org/protobuf/reflect/protoreflect" +) + +func DefaultHandler[T protoreflect.ProtoMessage, V protoreflect.ProtoMessage](impl defaultImplementationFunc[T, V]) func(ctx ServerCtx, in *Message, finished chan<- *Message) { + return func(ctx ServerCtx, in *Message, finished chan<- *Message) { + req := in.Message.(T) + defer ctx.Release() + resp, err := impl(ctx, req) + _ = SendMessage(ctx, finished, WrapMessage(in.Metadata, protoreflect.ProtoMessage(resp), err)) + } +} + +func ClientHandler[T protoreflect.ProtoMessage, V protoreflect.ProtoMessage](impl clientImplementationFunc[T, V]) func(ctx ServerCtx, in *Message, finished chan<- *Message) { + return func(ctx ServerCtx, in *Message, _ chan<- *Message) { + defer ctx.Release() + req := in.Message.(T) + //err := status.FromProto(in.Metadata.GetStatus()).Err() + _, _ = impl(ctx, req, in.Metadata.BroadcastMsg.BroadcastID) + } +} + +func BroadcastHandler[T protoreflect.ProtoMessage, V Broadcaster](impl implementationFunc[T, V], srv *Server) func(ctx ServerCtx, in *Message, finished chan<- *Message) { + return func(ctx ServerCtx, in *Message, finished chan<- *Message) { + // release immediately to process next message + ctx.Release() + + // guard: + // - A broadcastID should be non-empty: + // - Maybe the request should be unique? Remove duplicates of the same broadcast? <- Most likely no (up to the implementer) + if err := srv.broadcastSrv.validateMessage(in); err != nil { + if srv.broadcastSrv.logger != nil { + srv.broadcastSrv.logger.Debug("broadcast request not valid", "metadata", in.Metadata, "err", err) + } + return + } + + // interface conversion can fail if proto message of the wrong type is given. + // this happens when Cancellations arrive because the proto message is nil but + // we still want to process the message. + req, ok := in.Message.(T) + if !ok && in.Metadata.Method != Cancellation { + return + } + if in.Metadata.BroadcastMsg.IsBroadcastClient { + // keep track of the method called by the user + in.Metadata.BroadcastMsg.OriginMethod = in.Metadata.Method + } + // due to ordering we wrap the actual implementation function to be able to + // run it at a later time. + run := func(reqCtx context.Context, enqueueBroadcast func(*broadcast.Msg) error) { + // we need to pass in the reqCtx and broadcastChan because we can only retrieve + // them after we have gotten a response from the shard. The reqCtx + // is used for cancellations and broadcastChan for broadcasts. + broadcastMetadata := newBroadcastMetadata(in.Metadata) + broadcaster := srv.broadcastSrv.createBroadcaster(broadcastMetadata, srv.broadcastSrv.orchestrator, enqueueBroadcast).(V) + ctx.Context = reqCtx + impl(ctx, req, broadcaster) + } + + msg := broadcast.Content{} + createRequest(&msg, ctx, in, finished, run) + + // we are not interested in the server context as this is tied to the previous hop. + // instead we want to check whether the client has cancelled the broadcast request + // and if so, we return a cancelled context. This enables the implementer to listen + // for cancels and do proper actions. + reqCtx, enqueueBroadcast, err := srv.broadcastSrv.manager.Process(&msg) + if err != nil { + return + } + + run(reqCtx, enqueueBroadcast) + } +} + +func createRequest(msg *broadcast.Content, ctx ServerCtx, in *Message, finished chan<- *Message, run func(context.Context, func(*broadcast.Msg) error)) { + msg.BroadcastID = in.Metadata.BroadcastMsg.BroadcastID + msg.IsBroadcastClient = in.Metadata.BroadcastMsg.IsBroadcastClient + msg.OriginAddr = in.Metadata.BroadcastMsg.OriginAddr + msg.OriginMethod = in.Metadata.BroadcastMsg.OriginMethod + msg.SenderAddr = in.Metadata.BroadcastMsg.SenderAddr + if msg.SenderAddr == "" && msg.IsBroadcastClient { + msg.SenderAddr = "client" + } + if in.Metadata.BroadcastMsg.OriginDigest != nil { + msg.OriginDigest = in.Metadata.BroadcastMsg.OriginDigest + } + if in.Metadata.BroadcastMsg.OriginSignature != nil { + msg.OriginSignature = in.Metadata.BroadcastMsg.OriginSignature + } + if in.Metadata.BroadcastMsg.OriginPubKey != "" { + msg.OriginPubKey = in.Metadata.BroadcastMsg.OriginPubKey + } + msg.CurrentMethod = in.Metadata.Method + msg.Ctx = ctx.Context + msg.Run = run + if msg.OriginAddr == "" && msg.IsBroadcastClient { + msg.SendFn = createSendFn(in.Metadata.MessageID, in.Metadata.Method, finished, ctx) + } + if in.Metadata.Method == Cancellation { + msg.IsCancellation = true + } +} + +func createSendFn(msgID uint64, method string, finished chan<- *Message, ctx ServerCtx) func(resp protoreflect.ProtoMessage, err error) error { + return func(resp protoreflect.ProtoMessage, err error) error { + md := &ordering.Metadata{ + MessageID: msgID, + Method: method, + } + msg := WrapMessage(md, resp, err) + return SendMessage(ctx, finished, msg) + } +} + +func (srv *broadcastServer) validateMessage(in *Message) error { + if in == nil { + return fmt.Errorf("message cannot be empty. got: %v", in) + } + if in.Metadata == nil { + return fmt.Errorf("metadata cannot be empty. got: %v", in.Metadata) + } + if in.Metadata.BroadcastMsg == nil { + return fmt.Errorf("broadcastMsg cannot be empty. got: %v", in.Metadata.BroadcastMsg) + } + if in.Metadata.BroadcastMsg.BroadcastID <= 0 { + return fmt.Errorf("broadcastID cannot be empty. got: %v", in.Metadata.BroadcastMsg.BroadcastID) + } + return nil +} + +func (srv *Server) RegisterBroadcaster(broadcaster func(m BroadcastMetadata, o *BroadcastOrchestrator, e EnqueueBroadcast) Broadcaster) { + srv.broadcastSrv.createBroadcaster = broadcaster + srv.broadcastSrv.orchestrator = NewBroadcastOrchestrator(srv) +} + +func (srv *broadcastServer) broadcastHandler(method string, req protoreflect.ProtoMessage, broadcastID uint64, enqueueBroadcast EnqueueBroadcast, opts ...broadcast.BroadcastOptions) error { + return srv.manager.Broadcast(broadcastID, req, method, enqueueBroadcast, opts...) +} + +func (srv *broadcastServer) sendToClientHandler(broadcastID uint64, resp protoreflect.ProtoMessage, err error, enqueueBroadcast EnqueueBroadcast) error { + return srv.manager.SendToClient(broadcastID, resp, err, enqueueBroadcast) +} + +func (srv *broadcastServer) forwardHandler(req protoreflect.ProtoMessage, method string, broadcastID uint64, forwardAddr, originAddr string) { + cd := BroadcastCallData{ + Message: req, + Method: method, + BroadcastID: broadcastID, + IsBroadcastClient: true, + OriginAddr: originAddr, + ServerAddresses: []string{forwardAddr}, + } + srv.viewMutex.RLock() + srv.view.BroadcastCall(context.Background(), cd) + srv.viewMutex.RUnlock() +} + +func (srv *broadcastServer) cancelHandler(broadcastID uint64, srvAddrs []string, enqueueBroadcast EnqueueBroadcast) error { + return srv.manager.Cancel(broadcastID, srvAddrs, enqueueBroadcast) +} + +func (srv *broadcastServer) doneHandler(broadcastID uint64, enqueueBroadcast EnqueueBroadcast) { + srv.manager.Done(broadcastID, enqueueBroadcast) +} + +func (srv *broadcastServer) canceler(broadcastID uint64, srvAddrs []string) { + cd := BroadcastCallData{ + Message: nil, + Method: Cancellation, + BroadcastID: broadcastID, + ServerAddresses: srvAddrs, + } + srv.viewMutex.RLock() + srv.view.BroadcastCall(context.Background(), cd) + srv.viewMutex.RUnlock() +} + +func (srv *broadcastServer) serverBroadcastHandler(method string, req protoreflect.ProtoMessage, opts ...broadcast.BroadcastOptions) { + cd := BroadcastCallData{ + Message: req, + Method: method, + BroadcastID: srv.manager.NewBroadcastID(), + OriginAddr: broadcast.ServerOriginAddr, + IsBroadcastClient: false, + } + srv.viewMutex.RLock() + srv.view.BroadcastCall(context.Background(), cd) + srv.viewMutex.RUnlock() +} + +func (srv *Server) SendToClientHandler(resp protoreflect.ProtoMessage, err error, broadcastID uint64, enqueueBroadcast EnqueueBroadcast) error { + return srv.broadcastSrv.sendToClientHandler(broadcastID, resp, err, enqueueBroadcast) +} + +func (srv *broadcastServer) registerBroadcastFunc(method string) { + srv.manager.AddHandler(method, broadcast.ServerHandler(func(ctx context.Context, in protoreflect.ProtoMessage, broadcastID uint64, originAddr, originMethod string, options broadcast.BroadcastOptions, id uint32, addr string, originDigest, originSignature []byte, originPubKey string) { + cd := BroadcastCallData{ + Message: in, + Method: method, + BroadcastID: broadcastID, + IsBroadcastClient: false, + SenderAddr: addr, + OriginAddr: originAddr, + OriginMethod: originMethod, + ServerAddresses: options.ServerAddresses, + SkipSelf: options.SkipSelf, + OriginDigest: originDigest, + OriginSignature: originSignature, + OriginPubKey: originPubKey, + } + srv.viewMutex.RLock() + srv.view.BroadcastCall(ctx, cd) + srv.viewMutex.RUnlock() + })) +} + +func (srv *broadcastServer) registerSendToClientHandler(method string) { + srv.manager.AddHandler(method, nil) +} + +func (srv *Server) RegisterConfig(config RawConfiguration) { + srv.broadcastSrv.viewMutex.Lock() + srv.broadcastSrv.manager.ResetState() + srv.broadcastSrv.view = config + srv.broadcastSrv.viewMutex.Unlock() +} diff --git a/logging/logging.go b/logging/logging.go new file mode 100644 index 00000000..a87c7b22 --- /dev/null +++ b/logging/logging.go @@ -0,0 +1,109 @@ +package logging + +import ( + "log/slog" + "time" +) + +// the log entry used in slog with correct types and json mapping +type LogEntry struct { + Time time.Time `json:"time"` + Level string `json:"level"` + Msg string `json:"msg"` + MsgType string `json:"msgType"` + BroadcastID uint64 `json:"BroadcastID"` + Err error `json:"err"` + Method string `json:"method"` + From string `json:"from"` + Cancelled bool `json:"cancelled"` + MachineID uint64 `json:"MachineID"` + MsgID uint64 `json:"msgID"` + NodeID uint64 `json:"nodeID"` + NodeAddr string `json:"nodeAddr"` + Type string `json:"type"` + Reconnect bool `json:"reconnect"` + RetryNum float64 `json:"retryNum"` + MaxRetries int `json:"maxRetries"` + NumFailed int `json:"numFailed"` + Stopping bool `json:"stopping"` + IsBroadcastCall bool `json:"isBroadcastCall"` + Started time.Time `json:"started"` + Ended time.Time `json:"ended"` +} + +// funcs: used to get type safety on fields when logging +func MsgType(msgType string) slog.Attr { + return slog.String("msgType", msgType) +} + +func BroadcastID(broadcastID uint64) slog.Attr { + return slog.Uint64("BroadcastID", broadcastID) +} + +func Err(err error) slog.Attr { + return slog.Any("err", err) +} + +func Method(m string) slog.Attr { + return slog.String("method", m) +} + +func From(from string) slog.Attr { + return slog.String("from", from) +} + +func Cancelled(cancelled bool) slog.Attr { + return slog.Bool("cancelled", cancelled) +} + +func MachineID(machineID uint64) slog.Attr { + return slog.Uint64("MachineID", machineID) +} + +func MsgID(msgID uint64) slog.Attr { + return slog.Uint64("msgID", msgID) +} + +func NodeID(nodeID uint32) slog.Attr { + return slog.Uint64("nodeID", uint64(nodeID)) +} + +func NodeAddr(nodeAddr string) slog.Attr { + return slog.String("nodeAddr", nodeAddr) +} + +func Type(t string) slog.Attr { + return slog.String("type", t) +} + +func Reconnect(reconnect bool) slog.Attr { + return slog.Bool("reconnect", reconnect) +} + +func RetryNum(num float64) slog.Attr { + return slog.Float64("retryNum", num) +} + +func MaxRetries(maxRetries int) slog.Attr { + return slog.Int("maxRetries", maxRetries) +} + +func NumFailed(num int) slog.Attr { + return slog.Int("numFailed", num) +} + +func Stopping(stopping bool) slog.Attr { + return slog.Bool("stopping", stopping) +} + +func IsBroadcastCall(isBroadcastCall bool) slog.Attr { + return slog.Bool("isBroadcastCall", isBroadcastCall) +} + +func Started(started time.Time) slog.Attr { + return slog.Time("started", started) +} + +func Ended(ended time.Time) slog.Attr { + return slog.Time("ended", ended) +} diff --git a/mgr.go b/mgr.go index ffd15ebb..d4185f5a 100644 --- a/mgr.go +++ b/mgr.go @@ -1,11 +1,14 @@ package gorums import ( + "context" "fmt" - "log" + "log/slog" "sync" "sync/atomic" + "github.com/relab/gorums/broadcast" + "github.com/relab/gorums/logging" "google.golang.org/grpc" "google.golang.org/grpc/backoff" ) @@ -20,9 +23,10 @@ type RawManager struct { nodes []*RawNode lookup map[uint32]*RawNode closeOnce sync.Once - logger *log.Logger + logger *slog.Logger opts managerOptions nextMsgID uint64 + snowflake Snowflake } // NewRawManager returns a new RawManager for managing connection to nodes added @@ -37,8 +41,11 @@ func NewRawManager(opts ...ManagerOption) *RawManager { for _, opt := range opts { opt(&m.opts) } + snowflake := broadcast.NewSnowflake(m.opts.machineID) if m.opts.logger != nil { - m.logger = m.opts.logger + // a random machineID will be generated if m.opts.machineID is invalid + mID := snowflake.MachineID + m.logger = m.opts.logger.With(slog.Uint64("MachineID", mID)) } m.opts.grpcDialOpts = append(m.opts.grpcDialOpts, grpc.WithDefaultCallOptions( grpc.CallContentSubtype(ContentSubtype), @@ -48,17 +55,16 @@ func NewRawManager(opts ...ManagerOption) *RawManager { grpc.ConnectParams{Backoff: m.opts.backoff}, )) } - if m.logger != nil { - m.logger.Printf("ready") - } + m.log("manager: ready", nil) + m.snowflake = snowflake return m } func (m *RawManager) closeNodeConns() { for _, node := range m.nodes { err := node.close() - if err != nil && m.logger != nil { - m.logger.Printf("error closing: %v", err) + if err != nil { + m.log("manager: error closing node", err, logging.NodeID(node.ID())) } } } @@ -66,9 +72,7 @@ func (m *RawManager) closeNodeConns() { // Close closes all node connections and any client streams. func (m *RawManager) Close() { m.closeOnce.Do(func() { - if m.logger != nil { - m.logger.Printf("closing") - } + m.log("manager: closing", nil) m.closeNodeConns() }) } @@ -115,15 +119,10 @@ func (m *RawManager) AddNode(node *RawNode) error { // Node IDs must be unique return fmt.Errorf("config: node %d (%s) already exists", node.ID(), node.Address()) } - if m.logger != nil { - m.logger.Printf("Connecting to %s with id %d\n", node, node.id) - } + m.log("manager: connecting to node", nil, logging.NodeID(node.ID()), logging.NodeAddr(node.Address())) if err := node.connect(m); err != nil { - if m.logger != nil { - m.logger.Printf("Failed to connect to %s: %v (retrying)", node, err) - } + m.log("manager: failed to connect to node (retrying later)", err, logging.NodeID(node.ID()), logging.NodeAddr(node.Address())) } - m.mu.Lock() defer m.mu.Unlock() m.lookup[node.id] = node @@ -131,7 +130,22 @@ func (m *RawManager) AddNode(node *RawNode) error { return nil } +func (m *RawManager) Snowflake() Snowflake { + return m.snowflake +} + // getMsgID returns a unique message ID. func (m *RawManager) getMsgID() uint64 { return atomic.AddUint64(&m.nextMsgID, 1) } + +func (m *RawManager) log(msg string, err error, args ...slog.Attr) { + if m.logger != nil { + args = append(args, logging.Err(err), logging.Type("manager")) + level := slog.LevelInfo + if err != nil { + level = slog.LevelError + } + m.logger.LogAttrs(context.Background(), level, msg, args...) + } +} diff --git a/mgr_test.go b/mgr_test.go index 0640128d..d86d0cb2 100644 --- a/mgr_test.go +++ b/mgr_test.go @@ -2,7 +2,7 @@ package gorums_test import ( "bytes" - "log" + "log/slog" "testing" "time" @@ -19,13 +19,13 @@ var ( func TestManagerLogging(t *testing.T) { var ( - buf bytes.Buffer - logger = log.New(&buf, "logger: ", log.Lshortfile) + buf bytes.Buffer ) + buf.WriteString("\n") _ = gorums.NewRawManager( gorums.WithNoConnect(), - gorums.WithLogger(logger), + gorums.WithLogger(slog.Default()), ) t.Log(buf.String()) } diff --git a/multicast.go b/multicast.go index 1b265c8c..98b771d1 100644 --- a/multicast.go +++ b/multicast.go @@ -12,7 +12,9 @@ import ( // before the message has been sent. func (c RawConfiguration) Multicast(ctx context.Context, d QuorumCallData, opts ...CallOption) { o := getCallOptions(E_Multicast, opts) - md := &ordering.Metadata{MessageID: c.getMsgID(), Method: d.Method} + md := &ordering.Metadata{MessageID: c.getMsgID(), Method: d.Method, BroadcastMsg: &ordering.BroadcastMsg{ + IsBroadcastClient: d.IsBroadcastClient, BroadcastID: d.BroadcastID, OriginAddr: d.OriginAddr, + }} sentMsgs := 0 var replyChan chan response diff --git a/node.go b/node.go index 0fb65827..2a23c58e 100644 --- a/node.go +++ b/node.go @@ -7,6 +7,7 @@ import ( "net" "sort" "strconv" + "sync" "time" "google.golang.org/grpc" @@ -22,11 +23,12 @@ const nilAngleString = "" // You should use the generated `Node` struct instead. type RawNode struct { // Only assigned at creation. - id uint32 - addr string - conn *grpc.ClientConn - cancel func() - mgr *RawManager + id uint32 + addr string + conn *grpc.ClientConn + cancel func() + mgr *RawManager + dialMut sync.Mutex // the default channel channel *channel @@ -73,6 +75,8 @@ func (n *RawNode) connect(mgr *RawManager) error { // dial the node and close the current connection. func (n *RawNode) dial() error { + n.dialMut.Lock() + defer n.dialMut.Unlock() if n.conn != nil { // close the current connection before dialing again. n.conn.Close() @@ -92,10 +96,16 @@ func (n *RawNode) dial() error { // fresh contexts. Reusing contexts could result in reusing // a cancelled context. func (n *RawNode) newContext() context.Context { + n.dialMut.Lock() + defer n.dialMut.Unlock() md := n.mgr.opts.metadata.Copy() if n.mgr.opts.perNodeMD != nil { md = metadata.Join(md, n.mgr.opts.perNodeMD(n.id)) } + if n.cancel != nil { + // make sure to close any old ctx + n.cancel() + } var ctx context.Context ctx, n.cancel = context.WithCancel(context.Background()) return metadata.NewOutgoingContext(ctx, md) @@ -103,6 +113,8 @@ func (n *RawNode) newContext() context.Context { // close this node. func (n *RawNode) close() error { + n.dialMut.Lock() + defer n.dialMut.Unlock() // important to cancel first to stop goroutines n.cancel() if n.conn == nil { diff --git a/opts.go b/opts.go index 2def8017..bdfae2a9 100644 --- a/opts.go +++ b/opts.go @@ -1,9 +1,11 @@ package gorums import ( - "log" + "log/slog" "time" + "github.com/relab/gorums/authentication" + "github.com/relab/gorums/broadcast" "google.golang.org/grpc" "google.golang.org/grpc/backoff" "google.golang.org/grpc/metadata" @@ -12,19 +14,28 @@ import ( type managerOptions struct { grpcDialOpts []grpc.DialOption nodeDialTimeout time.Duration - logger *log.Logger + logger *slog.Logger noConnect bool backoff backoff.Config sendBuffer uint metadata metadata.MD perNodeMD func(uint32) metadata.MD + machineID uint64 // used for generating SnowflakeIDs + maxSendRetries int // number of times we try to resend a failed msg + maxConnRetries int // number of times we try to reconnect (in the background) to a node + auth *authentication.EllipticCurve // used when authenticating msgs } func newManagerOptions() managerOptions { return managerOptions{ backoff: backoff.DefaultConfig, - sendBuffer: 0, + sendBuffer: 100, nodeDialTimeout: 50 * time.Millisecond, + // Provide an illegal machineID to avoid unintentional collisions. + // 0 is a valid MachineID and should not be used as default. + machineID: uint64(broadcast.MaxMachineID) + 1, + maxSendRetries: 0, + maxConnRetries: -1, // no limit } } @@ -47,9 +58,10 @@ func WithGrpcDialOptions(opts ...grpc.DialOption) ManagerOption { } } -// WithLogger returns a ManagerOption which sets an optional error logger for -// the Manager. -func WithLogger(logger *log.Logger) ManagerOption { +// WithLogger returns a ManagerOption which sets an optional structured logger for +// the Manager. This will log events regarding creation of nodes and transmission +// of messages. +func WithLogger(logger *slog.Logger) ManagerOption { return func(o *managerOptions) { o.logger = logger } @@ -95,3 +107,37 @@ func WithPerNodeMetadata(f func(uint32) metadata.MD) ManagerOption { o.perNodeMD = f } } + +// WithAuthentication returns a ManagerOptions that enables digital signatures for msgs. +func WithAuthentication(auth *authentication.EllipticCurve) ManagerOption { + return func(o *managerOptions) { + o.auth = auth + } +} + +// WithMachineID returns a ManagerOption that allows you to set a unique ID for the client. +// This ID will be embedded in broadcast request sent from the client, making the requests +// trackable by the whole cluster. A random ID will be generated if not set. This can cause +// collisions if there are many clients. MinID = 0 and MaxID = 4095. +func WithMachineID(id uint64) ManagerOption { + return func(o *managerOptions) { + o.machineID = id + } +} + +// WithSendRetries returns a ManagerOption that allows you to specify how many times the node +// will try to send a message. The message will be dropped if it fails to send the message +// more than the specified number of times. +func WithSendRetries(maxRetries int) ManagerOption { + return func(o *managerOptions) { + o.maxSendRetries = maxRetries + } +} + +// WithConnRetries returns a ManagerOption that allows you to specify how many times the node +// will try to reconnect to a node. Default: no limit but it will follow a backoff strategy. +func WithConnRetries(maxRetries int) ManagerOption { + return func(o *managerOptions) { + o.maxConnRetries = maxRetries + } +} diff --git a/ordering/ordering.pb.go b/ordering/ordering.pb.go index 990c7bd7..0e3c6ae6 100644 --- a/ordering/ordering.pb.go +++ b/ordering/ordering.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.33.0 -// protoc v4.25.3 +// protoc-gen-go v1.32.0 +// protoc v3.12.4 // source: ordering/ordering.proto package ordering @@ -28,9 +28,11 @@ type Metadata struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - MessageID uint64 `protobuf:"varint,1,opt,name=MessageID,proto3" json:"MessageID,omitempty"` - Method string `protobuf:"bytes,2,opt,name=Method,proto3" json:"Method,omitempty"` - Status *status.Status `protobuf:"bytes,3,opt,name=Status,proto3" json:"Status,omitempty"` + MessageID uint64 `protobuf:"varint,1,opt,name=MessageID,proto3" json:"MessageID,omitempty"` + Method string `protobuf:"bytes,2,opt,name=Method,proto3" json:"Method,omitempty"` // uint32: 4 bytes + Status *status.Status `protobuf:"bytes,3,opt,name=Status,proto3" json:"Status,omitempty"` + BroadcastMsg *BroadcastMsg `protobuf:"bytes,4,opt,name=BroadcastMsg,proto3" json:"BroadcastMsg,omitempty"` + AuthMsg *AuthMsg `protobuf:"bytes,5,opt,name=AuthMsg,proto3" json:"AuthMsg,omitempty"` } func (x *Metadata) Reset() { @@ -86,20 +88,232 @@ func (x *Metadata) GetStatus() *status.Status { return nil } +func (x *Metadata) GetBroadcastMsg() *BroadcastMsg { + if x != nil { + return x.BroadcastMsg + } + return nil +} + +func (x *Metadata) GetAuthMsg() *AuthMsg { + if x != nil { + return x.AuthMsg + } + return nil +} + +type BroadcastMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IsBroadcastClient bool `protobuf:"varint,1,opt,name=IsBroadcastClient,proto3" json:"IsBroadcastClient,omitempty"` + BroadcastID uint64 `protobuf:"varint,2,opt,name=BroadcastID,proto3" json:"BroadcastID,omitempty"` + SenderAddr string `protobuf:"bytes,3,opt,name=SenderAddr,proto3" json:"SenderAddr,omitempty"` // bytes -> ipv4: 32 bit/ 4 bytes, ipv6: 128 bit / 16 bytes -> + Port: 16 bit / 2 bytes + OriginAddr string `protobuf:"bytes,4,opt,name=OriginAddr,proto3" json:"OriginAddr,omitempty"` // bytes -> ipv4: 32 bit/ 4 bytes, ipv6: 128 bit / 16 bytes -> + Port: 16 bit / 2 bytes + OriginMethod string `protobuf:"bytes,5,opt,name=OriginMethod,proto3" json:"OriginMethod,omitempty"` // uint32: 4 bytes + OriginDigest []byte `protobuf:"bytes,6,opt,name=OriginDigest,proto3" json:"OriginDigest,omitempty"` + OriginSignature []byte `protobuf:"bytes,7,opt,name=OriginSignature,proto3" json:"OriginSignature,omitempty"` + OriginPubKey string `protobuf:"bytes,8,opt,name=OriginPubKey,proto3" json:"OriginPubKey,omitempty"` +} + +func (x *BroadcastMsg) Reset() { + *x = BroadcastMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_ordering_ordering_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *BroadcastMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BroadcastMsg) ProtoMessage() {} + +func (x *BroadcastMsg) ProtoReflect() protoreflect.Message { + mi := &file_ordering_ordering_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BroadcastMsg.ProtoReflect.Descriptor instead. +func (*BroadcastMsg) Descriptor() ([]byte, []int) { + return file_ordering_ordering_proto_rawDescGZIP(), []int{1} +} + +func (x *BroadcastMsg) GetIsBroadcastClient() bool { + if x != nil { + return x.IsBroadcastClient + } + return false +} + +func (x *BroadcastMsg) GetBroadcastID() uint64 { + if x != nil { + return x.BroadcastID + } + return 0 +} + +func (x *BroadcastMsg) GetSenderAddr() string { + if x != nil { + return x.SenderAddr + } + return "" +} + +func (x *BroadcastMsg) GetOriginAddr() string { + if x != nil { + return x.OriginAddr + } + return "" +} + +func (x *BroadcastMsg) GetOriginMethod() string { + if x != nil { + return x.OriginMethod + } + return "" +} + +func (x *BroadcastMsg) GetOriginDigest() []byte { + if x != nil { + return x.OriginDigest + } + return nil +} + +func (x *BroadcastMsg) GetOriginSignature() []byte { + if x != nil { + return x.OriginSignature + } + return nil +} + +func (x *BroadcastMsg) GetOriginPubKey() string { + if x != nil { + return x.OriginPubKey + } + return "" +} + +type AuthMsg struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PublicKey string `protobuf:"bytes,1,opt,name=PublicKey,proto3" json:"PublicKey,omitempty"` + Signature []byte `protobuf:"bytes,2,opt,name=Signature,proto3" json:"Signature,omitempty"` + Sender string `protobuf:"bytes,3,opt,name=Sender,proto3" json:"Sender,omitempty"` +} + +func (x *AuthMsg) Reset() { + *x = AuthMsg{} + if protoimpl.UnsafeEnabled { + mi := &file_ordering_ordering_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *AuthMsg) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AuthMsg) ProtoMessage() {} + +func (x *AuthMsg) ProtoReflect() protoreflect.Message { + mi := &file_ordering_ordering_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AuthMsg.ProtoReflect.Descriptor instead. +func (*AuthMsg) Descriptor() ([]byte, []int) { + return file_ordering_ordering_proto_rawDescGZIP(), []int{2} +} + +func (x *AuthMsg) GetPublicKey() string { + if x != nil { + return x.PublicKey + } + return "" +} + +func (x *AuthMsg) GetSignature() []byte { + if x != nil { + return x.Signature + } + return nil +} + +func (x *AuthMsg) GetSender() string { + if x != nil { + return x.Sender + } + return "" +} + var File_ordering_ordering_proto protoreflect.FileDescriptor var file_ordering_ordering_proto_rawDesc = []byte{ 0x0a, 0x17, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2f, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x08, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x1a, 0x17, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x6c, 0x0a, 0x08, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x2a, - 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x32, 0x42, 0x0a, 0x06, 0x47, 0x6f, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd5, 0x01, 0x0a, + 0x08, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, + 0x2a, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3a, 0x0a, 0x0c, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x52, 0x0c, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x12, 0x2b, 0x0a, 0x07, 0x41, 0x75, 0x74, 0x68, 0x4d, + 0x73, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x6f, 0x72, 0x64, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x75, 0x74, 0x68, 0x4d, 0x73, 0x67, 0x52, 0x07, 0x41, 0x75, 0x74, + 0x68, 0x4d, 0x73, 0x67, 0x22, 0xb4, 0x02, 0x0a, 0x0c, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x4d, 0x73, 0x67, 0x12, 0x2c, 0x0a, 0x11, 0x49, 0x73, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x11, 0x49, 0x73, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x43, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x41, + 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x53, 0x65, 0x6e, 0x64, 0x65, + 0x72, 0x41, 0x64, 0x64, 0x72, 0x12, 0x1e, 0x0a, 0x0a, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x41, + 0x64, 0x64, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x4f, 0x72, 0x69, 0x67, 0x69, + 0x6e, 0x41, 0x64, 0x64, 0x72, 0x12, 0x22, 0x0a, 0x0c, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x4d, + 0x65, 0x74, 0x68, 0x6f, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x4f, 0x72, 0x69, + 0x67, 0x69, 0x6e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x22, 0x0a, 0x0c, 0x4f, 0x72, 0x69, + 0x67, 0x69, 0x6e, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x0c, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x28, 0x0a, + 0x0f, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x53, 0x69, + 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x4f, 0x72, 0x69, 0x67, 0x69, + 0x6e, 0x50, 0x75, 0x62, 0x4b, 0x65, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x4f, + 0x72, 0x69, 0x67, 0x69, 0x6e, 0x50, 0x75, 0x62, 0x4b, 0x65, 0x79, 0x22, 0x5d, 0x0a, 0x07, 0x41, + 0x75, 0x74, 0x68, 0x4d, 0x73, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, + 0x4b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x63, 0x4b, 0x65, 0x79, 0x12, 0x1c, 0x0a, 0x09, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x53, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x32, 0x42, 0x0a, 0x06, 0x47, 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x12, 0x38, 0x0a, 0x0a, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x12, 0x2e, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x12, 0x2e, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, @@ -121,20 +335,24 @@ func file_ordering_ordering_proto_rawDescGZIP() []byte { return file_ordering_ordering_proto_rawDescData } -var file_ordering_ordering_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_ordering_ordering_proto_msgTypes = make([]protoimpl.MessageInfo, 3) var file_ordering_ordering_proto_goTypes = []interface{}{ (*Metadata)(nil), // 0: ordering.Metadata - (*status.Status)(nil), // 1: google.rpc.Status + (*BroadcastMsg)(nil), // 1: ordering.BroadcastMsg + (*AuthMsg)(nil), // 2: ordering.AuthMsg + (*status.Status)(nil), // 3: google.rpc.Status } var file_ordering_ordering_proto_depIdxs = []int32{ - 1, // 0: ordering.Metadata.Status:type_name -> google.rpc.Status - 0, // 1: ordering.Gorums.NodeStream:input_type -> ordering.Metadata - 0, // 2: ordering.Gorums.NodeStream:output_type -> ordering.Metadata - 2, // [2:3] is the sub-list for method output_type - 1, // [1:2] is the sub-list for method input_type - 1, // [1:1] is the sub-list for extension type_name - 1, // [1:1] is the sub-list for extension extendee - 0, // [0:1] is the sub-list for field type_name + 3, // 0: ordering.Metadata.Status:type_name -> google.rpc.Status + 1, // 1: ordering.Metadata.BroadcastMsg:type_name -> ordering.BroadcastMsg + 2, // 2: ordering.Metadata.AuthMsg:type_name -> ordering.AuthMsg + 0, // 3: ordering.Gorums.NodeStream:input_type -> ordering.Metadata + 0, // 4: ordering.Gorums.NodeStream:output_type -> ordering.Metadata + 4, // [4:5] is the sub-list for method output_type + 3, // [3:4] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name } func init() { file_ordering_ordering_proto_init() } @@ -155,6 +373,30 @@ func file_ordering_ordering_proto_init() { return nil } } + file_ordering_ordering_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_ordering_ordering_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*AuthMsg); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -162,7 +404,7 @@ func file_ordering_ordering_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_ordering_ordering_proto_rawDesc, NumEnums: 0, - NumMessages: 1, + NumMessages: 3, NumExtensions: 0, NumServices: 1, }, diff --git a/ordering/ordering.proto b/ordering/ordering.proto index 380596d0..695e219d 100644 --- a/ordering/ordering.proto +++ b/ordering/ordering.proto @@ -16,6 +16,25 @@ service Gorums { // and contains information necessary for Gorums to handle the messages. message Metadata { uint64 MessageID = 1; - string Method = 2; + string Method = 2; // uint32: 4 bytes google.rpc.Status Status = 3; + BroadcastMsg BroadcastMsg = 4; + AuthMsg AuthMsg = 5; } + +message BroadcastMsg { + bool IsBroadcastClient = 1; + uint64 BroadcastID = 2; + string SenderAddr = 3; // bytes -> ipv4: 32 bit/ 4 bytes, ipv6: 128 bit / 16 bytes -> + Port: 16 bit / 2 bytes + string OriginAddr = 4; // bytes -> ipv4: 32 bit/ 4 bytes, ipv6: 128 bit / 16 bytes -> + Port: 16 bit / 2 bytes + string OriginMethod = 5; // uint32: 4 bytes + bytes OriginDigest = 6; + bytes OriginSignature = 7; + string OriginPubKey = 8; +} + +message AuthMsg { + string PublicKey = 1; + bytes Signature = 2; + string Sender = 3; +} \ No newline at end of file diff --git a/ordering/ordering_grpc.pb.go b/ordering/ordering_grpc.pb.go index 2a1bdb4f..2c005c60 100644 --- a/ordering/ordering_grpc.pb.go +++ b/ordering/ordering_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.3.0 -// - protoc v4.25.3 +// - protoc-gen-go-grpc v1.2.0 +// - protoc v3.12.4 // source: ordering/ordering.proto package ordering @@ -18,10 +18,6 @@ import ( // Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 -const ( - Gorums_NodeStream_FullMethodName = "/ordering.Gorums/NodeStream" -) - // GorumsClient is the client API for Gorums service. // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. @@ -41,7 +37,7 @@ func NewGorumsClient(cc grpc.ClientConnInterface) GorumsClient { } func (c *gorumsClient) NodeStream(ctx context.Context, opts ...grpc.CallOption) (Gorums_NodeStreamClient, error) { - stream, err := c.cc.NewStream(ctx, &Gorums_ServiceDesc.Streams[0], Gorums_NodeStream_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Gorums_ServiceDesc.Streams[0], "/ordering.Gorums/NodeStream", opts...) if err != nil { return nil, err } diff --git a/quorumcall.go b/quorumcall.go index 3e3f97e8..2feb0614 100644 --- a/quorumcall.go +++ b/quorumcall.go @@ -13,10 +13,13 @@ import ( // // This struct should be used by generated code only. type QuorumCallData struct { - Message protoreflect.ProtoMessage - Method string - PerNodeArgFn func(protoreflect.ProtoMessage, uint32) protoreflect.ProtoMessage - QuorumFunction func(protoreflect.ProtoMessage, map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) + Message protoreflect.ProtoMessage + Method string + PerNodeArgFn func(protoreflect.ProtoMessage, uint32) protoreflect.ProtoMessage + QuorumFunction func(protoreflect.ProtoMessage, map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) + IsBroadcastClient bool + OriginAddr string + BroadcastID uint64 // a unique identifier for the current message } // QuorumCall performs a quorum call on the configuration. @@ -24,7 +27,9 @@ type QuorumCallData struct { // This method should be used by generated code only. func (c RawConfiguration) QuorumCall(ctx context.Context, d QuorumCallData) (resp protoreflect.ProtoMessage, err error) { expectedReplies := len(c) - md := &ordering.Metadata{MessageID: c.getMsgID(), Method: d.Method} + md := &ordering.Metadata{MessageID: c.getMsgID(), Method: d.Method, BroadcastMsg: &ordering.BroadcastMsg{ + IsBroadcastClient: d.IsBroadcastClient, BroadcastID: d.BroadcastID, OriginAddr: d.OriginAddr, + }} replyChan := make(chan response, expectedReplies) for _, n := range c { diff --git a/rpc.go b/rpc.go index ff2ad7a0..14713dcf 100644 --- a/rpc.go +++ b/rpc.go @@ -11,15 +11,26 @@ import ( // // This struct should be used by generated code only. type CallData struct { - Message protoreflect.ProtoMessage - Method string + Message protoreflect.ProtoMessage + Method string + BroadcastID uint64 + OriginDigest []byte + OriginSignature []byte + OriginPubKey string } // RPCCall executes a remote procedure call on the node. // // This method should be used by generated code only. func (n *RawNode) RPCCall(ctx context.Context, d CallData) (protoreflect.ProtoMessage, error) { - md := &ordering.Metadata{MessageID: n.mgr.getMsgID(), Method: d.Method} + md := &ordering.Metadata{MessageID: n.mgr.getMsgID(), Method: d.Method, BroadcastMsg: &ordering.BroadcastMsg{ + BroadcastID: d.BroadcastID, + OriginDigest: d.OriginDigest, + OriginPubKey: d.OriginPubKey, + OriginSignature: d.OriginSignature, + }} + + //md := &ordering.Metadata{MessageID: n.mgr.getMsgID(), Method: d.Method} replyChan := make(chan response, 1) n.channel.enqueue(request{ctx: ctx, msg: &Message{Metadata: md, Message: d.Message}}, replyChan, false) diff --git a/server.go b/server.go index cce8efc7..1bffc986 100644 --- a/server.go +++ b/server.go @@ -2,9 +2,15 @@ package gorums import ( "context" + "crypto/elliptic" + "fmt" + "log/slog" "net" "sync" + "time" + "github.com/relab/gorums/authentication" + "github.com/relab/gorums/broadcast" "github.com/relab/gorums/ordering" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -32,6 +38,59 @@ func newOrderingServer(opts *serverOptions) *orderingServer { return s } +func (s *orderingServer) encodeMsg(req *Message) ([]byte, error) { + // we must not consider the signature field when validating. + // also the msgType must be set to requestType. + signature := make([]byte, len(req.Metadata.AuthMsg.Signature)) + copy(signature, req.Metadata.AuthMsg.Signature) + reqType := req.msgType + req.Metadata.AuthMsg.Signature = nil + req.msgType = 0 + encodedMsg, err := s.opts.auth.EncodeMsg(*req) + req.Metadata.AuthMsg.Signature = make([]byte, len(signature)) + copy(req.Metadata.AuthMsg.Signature, signature) + req.msgType = reqType + return encodedMsg, err +} + +func (s *orderingServer) verify(req *Message) error { + if s.opts.auth == nil { + return nil + } + if req.Metadata.AuthMsg == nil { + return fmt.Errorf("missing authMsg") + } + if req.Metadata.AuthMsg.Signature == nil { + return fmt.Errorf("missing signature") + } + if req.Metadata.AuthMsg.PublicKey == "" { + return fmt.Errorf("missing publicKey") + } + auth := s.opts.auth + authMsg := req.Metadata.AuthMsg + if s.opts.allowList != nil { + pemEncodedPub, ok := s.opts.allowList[authMsg.Sender] + if !ok { + return fmt.Errorf("not allowed") + } + if pemEncodedPub != authMsg.PublicKey { + return fmt.Errorf("publicKey did not match") + } + } + encodedMsg, err := s.encodeMsg(req) + if err != nil { + return err + } + valid, err := auth.VerifySignature(authMsg.PublicKey, encodedMsg, authMsg.Signature) + if err != nil { + return err + } + if !valid { + return fmt.Errorf("invalid signature") + } + return nil +} + // SendMessage attempts to send a message on a channel. // // This function should be used by generated code only. @@ -91,6 +150,10 @@ func (s *orderingServer) NodeStream(srv ordering.Gorums_NodeStreamServer) error if err != nil { return err } + err = s.verify(req) + if err != nil { + continue + } if handler, ok := s.handlers[req.Metadata.Method]; ok { // We start the handler in a new goroutine in order to allow multiple handlers to run concurrently. // However, to preserve request ordering, the handler must unlock the shared mutex when it has either @@ -103,9 +166,24 @@ func (s *orderingServer) NodeStream(srv ordering.Gorums_NodeStreamServer) error } type serverOptions struct { - buffer uint - grpcOpts []grpc.ServerOption - connectCallback func(context.Context) + buffer uint + grpcOpts []grpc.ServerOption + connectCallback func(context.Context) + logger *slog.Logger + executionOrder map[string]int + machineID uint64 + clientDialTimeout time.Duration + reqTTL time.Duration // the lifetime of a broadcast request + shardBuffer int // size of a newly initialized shard + sendBuffer int // buffer on the send and broadcast channels for a broadcast processor + // this is the address other nodes should connect to. Sometimes, e.g. when + // running in a docker container it is useful to listen to the loopback + // address and use forwarding from the host. If not this option is not given, + // the listen address used on the gRPC listener will be used instead. + listenAddr string + allowList map[string]string + auth *authentication.EllipticCurve + grpcDialOpts []grpc.DialOption } // ServerOption is used to change settings for the GorumsServer @@ -136,23 +214,162 @@ func WithConnectCallback(callback func(context.Context)) ServerOption { } } +// WithOrder returns a ServerOption which defines the order of execution +// of gRPC methods. +// +// E.g. in PBFT we can specify the order: PrePrepare, Prepare, and Commit. +// Gorums will then make sure to only execute messages in this order. +// The rules are defined as such: +// 1. Messages to the first gRPC method (PrePrepare) will always be executed. +// 2. Messages to gRPC methods not defined in the order will always be executed. +// 3. Messages to gRPC methods appearing later in the order will be cached and executed later. +// 4. Messages to gRPC methods appearing earlier than the current method will be executed immediately. +// E.g. if current method is Commit, then messages to PrePrepare and Prepare will be accepted. +// 5. The server itself needs to call the next method in the order to progress to the next gRPC method. +// E.g. by calling broadcast.Prepare(). +// 6. If the BroadcastOption ProgressTo() is used, then it will progress to the given gRPC method. +func WithOrder(executionOrder ...string) ServerOption { + return func(o *serverOptions) { + o.executionOrder = make(map[string]int) + for i, method := range executionOrder { + o.executionOrder[method] = i + } + } +} + +// WithClientDialTimeout returns a ServerOption which sets the dial timeout +// used to send replies back to the client in a BroadcastCall. +func WithClientDialTimeout(dialTimeout time.Duration) ServerOption { + return func(o *serverOptions) { + o.clientDialTimeout = dialTimeout + } +} + +// WithServerGrpcDialOptions returns a ServerOption which sets any gRPC dial options +// the Broadcast Router should use when connecting to each client. +func WithServerGrpcDialOptions(opts ...grpc.DialOption) ServerOption { + return func(o *serverOptions) { + o.grpcDialOpts = make([]grpc.DialOption, 0, len(opts)) + o.grpcDialOpts = append(o.grpcDialOpts, opts...) + } +} + +// WithShardBuffer returns a ServerOption which sets the buffer size +// of the shards. A higher shard size uses more memory but saves time +// when the number of broadcast requests is large. +func WithShardBuffer(shardBuffer int) ServerOption { + return func(o *serverOptions) { + o.shardBuffer = shardBuffer + } +} + +// WithSendBuffer returns a ServerOption which sets the buffer size +// of the sendChannel and broadcastChannel for a broadcast processor. +// A higher sendBuffer uses more memory but saves time when the number +// broadcast messages is large. +func WithSendBuffer(sendBuffer int) ServerOption { + return func(o *serverOptions) { + o.sendBuffer = sendBuffer + } +} + +// WithBroadcastReqTTL returns a ServerOption which sets the lifetime +// used for the broadcast processors. +func WithBroadcastReqTTL(reqTTL time.Duration) ServerOption { + return func(o *serverOptions) { + o.reqTTL = reqTTL + } +} + +// WithSLogger returns a ServerOption which sets an optional structured logger for +// the Server. This will log internal events regarding broadcast requests. The +// ManagerOption WithLogger() should be used when creating the manager in order +// to log events related to transmission of messages. +func WithSLogger(logger *slog.Logger) ServerOption { + return func(o *serverOptions) { + o.logger = logger + } +} + +// WithSrvID sets the MachineID of the broadcast server. This ID is used to +// generate BroadcastIDs. This method should be used if a replica needs to +// initiate a broadcast request. +// +// An example use case is in Paxos: +// The designated leader sends a prepare and receives some promises it has +// never seen before. It thus needs to send accept messages correspondingly. +// These accept messages are not part of any broadcast request and the server +// is thus responsible for the origin of these requests. +func WithSrvID(machineID uint64) ServerOption { + return func(o *serverOptions) { + o.machineID = machineID + } +} + +// WithAllowList accepts (address, publicKey) pairs which is used to validate +// messages. Only nodes added to the allow list are allowed to send msgs to +// the server. +func WithAllowList(curve elliptic.Curve, allowed ...string) ServerOption { + return func(o *serverOptions) { + o.allowList = make(map[string]string) + if len(allowed)%2 != 0 { + panic("must provide (address, publicKey) pairs to WithAllowList()") + } + for i := range allowed { + if i%2 != 0 { + continue + } + o.allowList[allowed[i]] = allowed[i+1] + } + o.auth = authentication.New(curve) + } +} + +// EnforceAuthentication accepts an elliptic curve which is used to validate +// messages. Only msgs with a pubKey and signature will be processed by the +// server. +func EnforceAuthentication(curve elliptic.Curve) ServerOption { + return func(o *serverOptions) { + o.auth = authentication.New(curve) + } +} + +// WithListenAddr sets the IP address of the broadcast server which will be used in messages +// sent by the server. The network of the address has to be a TCP network name. Hence, +// net.ResolveTCPAddr() can be used to obtain the net.Addr type of an address. +func WithListenAddr(listenAddr net.Addr) ServerOption { + return func(o *serverOptions) { + o.listenAddr = listenAddr.String() + } +} + // Server serves all ordering based RPCs using registered handlers. type Server struct { - srv *orderingServer - grpcServer *grpc.Server + srv *orderingServer + grpcServer *grpc.Server + broadcastSrv *broadcastServer } // NewServer returns a new instance of GorumsServer. // This function is intended for internal Gorums use. // You should call `NewServer` in the generated code instead. func NewServer(opts ...ServerOption) *Server { - var serverOpts serverOptions + serverOpts := serverOptions{ + clientDialTimeout: 10 * time.Second, + shardBuffer: 200, + sendBuffer: 30, + reqTTL: 5 * time.Minute, + // Provide an illegal machineID to avoid unintentional collisions. + // 0 is a valid MachineID and should not be used as default. + machineID: uint64(broadcast.MaxMachineID) + 1, + } for _, opt := range opts { opt(&serverOpts) } s := &Server{ - srv: newOrderingServer(&serverOpts), - grpcServer: grpc.NewServer(serverOpts.grpcOpts...), + srv: newOrderingServer(&serverOpts), + grpcServer: grpc.NewServer(serverOpts.grpcOpts...), + broadcastSrv: newBroadcastServer(&serverOpts), } ordering.RegisterGorumsServer(s.grpcServer, s.srv) return s @@ -162,9 +379,14 @@ func NewServer(opts ...ServerOption) *Server { // // This function should only be used by generated code. func (s *Server) RegisterHandler(method string, handler requestHandler) { + s.broadcastSrv.registerBroadcastFunc(method) s.srv.handlers[method] = handler } +func (s *Server) RegisterClientHandler(method string) { + s.broadcastSrv.registerSendToClientHandler(method) +} + // Serve starts serving on the listener. func (s *Server) Serve(listener net.Listener) error { return s.grpcServer.Serve(listener) @@ -172,11 +394,17 @@ func (s *Server) Serve(listener net.Listener) error { // GracefulStop waits for all RPCs to finish before stopping. func (s *Server) GracefulStop() { + if s.broadcastSrv != nil { + s.broadcastSrv.stop() + } s.grpcServer.GracefulStop() } // Stop stops the server immediately. func (s *Server) Stop() { + if s.broadcastSrv != nil { + s.broadcastSrv.stop() + } s.grpcServer.Stop() } diff --git a/tests/Makefile b/tests/Makefile index 5b8f51ca..d19b885a 100644 --- a/tests/Makefile +++ b/tests/Makefile @@ -1,7 +1,7 @@ # Tests that should be run each time -RUNTESTS := qf ordering metadata tls unresponsive dummy oneway config correctable +RUNTESTS := qf ordering metadata tls unresponsive dummy oneway config correctable broadcast -.PHONY: all qf ordering metadata tls unresponsive dummy oneway config correctable +.PHONY: all qf ordering metadata tls unresponsive dummy oneway config correctable broadcast all: $(RUNTESTS) @@ -23,6 +23,8 @@ config: config/config.pb.go config/config_gorums.pb.go correctable: correctable/correctable.pb.go correctable/correctable_gorums.pb.go +broadcast: broadcast/broadcast.pb.go broadcast/broadcast_gorums.pb.go + %.pb.go : %.proto @protoc -I=..:. --go_out=paths=source_relative:. $< diff --git a/tests/broadcast/broadcast.pb.go b/tests/broadcast/broadcast.pb.go new file mode 100644 index 00000000..ccf472f8 --- /dev/null +++ b/tests/broadcast/broadcast.pb.go @@ -0,0 +1,398 @@ +// Test to benchmark quorum functions with and without the request parameter. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.32.0 +// protoc v3.12.4 +// source: broadcast/broadcast.proto + +package broadcast + +import ( + _ "github.com/relab/gorums" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Value int64 `protobuf:"varint,1,opt,name=Value,proto3" json:"Value,omitempty"` + From string `protobuf:"bytes,2,opt,name=From,proto3" json:"From,omitempty"` +} + +func (x *Request) Reset() { + *x = Request{} + if protoimpl.UnsafeEnabled { + mi := &file_broadcast_broadcast_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Request) ProtoMessage() {} + +func (x *Request) ProtoReflect() protoreflect.Message { + mi := &file_broadcast_broadcast_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Request.ProtoReflect.Descriptor instead. +func (*Request) Descriptor() ([]byte, []int) { + return file_broadcast_broadcast_proto_rawDescGZIP(), []int{0} +} + +func (x *Request) GetValue() int64 { + if x != nil { + return x.Value + } + return 0 +} + +func (x *Request) GetFrom() string { + if x != nil { + return x.From + } + return "" +} + +type Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Result int64 `protobuf:"varint,1,opt,name=Result,proto3" json:"Result,omitempty"` + Replies int64 `protobuf:"varint,2,opt,name=Replies,proto3" json:"Replies,omitempty"` + From string `protobuf:"bytes,3,opt,name=From,proto3" json:"From,omitempty"` +} + +func (x *Response) Reset() { + *x = Response{} + if protoimpl.UnsafeEnabled { + mi := &file_broadcast_broadcast_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Response) ProtoMessage() {} + +func (x *Response) ProtoReflect() protoreflect.Message { + mi := &file_broadcast_broadcast_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Response.ProtoReflect.Descriptor instead. +func (*Response) Descriptor() ([]byte, []int) { + return file_broadcast_broadcast_proto_rawDescGZIP(), []int{1} +} + +func (x *Response) GetResult() int64 { + if x != nil { + return x.Result + } + return 0 +} + +func (x *Response) GetReplies() int64 { + if x != nil { + return x.Replies + } + return 0 +} + +func (x *Response) GetFrom() string { + if x != nil { + return x.From + } + return "" +} + +type Empty struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *Empty) Reset() { + *x = Empty{} + if protoimpl.UnsafeEnabled { + mi := &file_broadcast_broadcast_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Empty) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Empty) ProtoMessage() {} + +func (x *Empty) ProtoReflect() protoreflect.Message { + mi := &file_broadcast_broadcast_proto_msgTypes[2] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Empty.ProtoReflect.Descriptor instead. +func (*Empty) Descriptor() ([]byte, []int) { + return file_broadcast_broadcast_proto_rawDescGZIP(), []int{2} +} + +var File_broadcast_broadcast_proto protoreflect.FileDescriptor + +var file_broadcast_broadcast_proto_rawDesc = []byte{ + 0x0a, 0x19, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2f, 0x62, 0x72, 0x6f, 0x61, + 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, 0x62, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x1a, 0x0c, 0x67, 0x6f, 0x72, 0x75, 0x6d, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x33, 0x0a, 0x07, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x46, 0x72, 0x6f, 0x6d, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x46, 0x72, 0x6f, 0x6d, 0x22, 0x50, 0x0a, 0x08, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x18, 0x0a, + 0x07, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x65, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x46, 0x72, 0x6f, 0x6d, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x46, 0x72, 0x6f, 0x6d, 0x22, 0x07, 0x0a, 0x05, 0x45, + 0x6d, 0x70, 0x74, 0x79, 0x32, 0x82, 0x09, 0x0a, 0x10, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x3b, 0x0a, 0x0a, 0x51, 0x75, 0x6f, + 0x72, 0x75, 0x6d, 0x43, 0x61, 0x6c, 0x6c, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x12, 0x4c, 0x0a, 0x17, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, + 0x43, 0x61, 0x6c, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x08, 0xa0, 0xb5, 0x18, 0x01, + 0x90, 0xb8, 0x18, 0x01, 0x12, 0x48, 0x0a, 0x17, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x43, 0x61, + 0x6c, 0x6c, 0x57, 0x69, 0x74, 0x68, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x12, + 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xa0, 0xb5, 0x18, 0x01, 0x12, 0x37, + 0x0a, 0x09, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x63, 0x61, 0x73, 0x74, 0x12, 0x12, 0x2e, 0x62, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x43, 0x0a, 0x15, 0x4d, 0x75, 0x6c, 0x74, 0x69, + 0x63, 0x61, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x74, 0x65, + 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x98, 0xb5, 0x18, 0x01, 0x12, 0x3e, 0x0a, 0x0d, + 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x43, 0x61, 0x6c, 0x6c, 0x12, 0x12, 0x2e, + 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, 0x01, 0x12, 0x43, 0x0a, 0x15, + 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6d, 0x65, + 0x64, 0x69, 0x61, 0x74, 0x65, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, + 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, + 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, 0xb8, 0x18, + 0x01, 0x12, 0x37, 0x0a, 0x09, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x12, + 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, + 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x45, 0x0a, 0x14, 0x42, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x43, 0x61, 0x6c, 0x6c, 0x46, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, + 0x01, 0x12, 0x40, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x43, 0x61, + 0x6c, 0x6c, 0x54, 0x6f, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, + 0xb5, 0x18, 0x01, 0x12, 0x41, 0x0a, 0x13, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x54, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, + 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x22, 0x04, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x37, 0x0a, 0x06, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, 0x01, 0x12, + 0x40, 0x0a, 0x0f, 0x4c, 0x6f, 0x6e, 0x67, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x54, 0x61, + 0x73, 0x6b, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, + 0x01, 0x12, 0x37, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x56, 0x61, 0x6c, 0x12, 0x12, 0x2e, 0x62, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, 0x18, 0x01, 0x12, 0x36, 0x0a, 0x05, 0x4f, 0x72, + 0x64, 0x65, 0x72, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x13, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x04, 0xb0, 0xb5, + 0x18, 0x01, 0x12, 0x38, 0x0a, 0x0a, 0x50, 0x72, 0x65, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, + 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, 0xb8, 0x18, 0x01, 0x12, 0x35, 0x0a, 0x07, + 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x12, 0x12, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, + 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, + 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, + 0xb8, 0x18, 0x01, 0x12, 0x34, 0x0a, 0x06, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x12, 0x12, 0x2e, + 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x10, 0x2e, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x22, 0x04, 0x90, 0xb8, 0x18, 0x01, 0x42, 0x29, 0x5a, 0x27, 0x67, 0x69, 0x74, + 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x62, 0x2f, 0x67, 0x6f, + 0x72, 0x75, 0x6d, 0x73, 0x2f, 0x74, 0x65, 0x73, 0x74, 0x73, 0x2f, 0x62, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_broadcast_broadcast_proto_rawDescOnce sync.Once + file_broadcast_broadcast_proto_rawDescData = file_broadcast_broadcast_proto_rawDesc +) + +func file_broadcast_broadcast_proto_rawDescGZIP() []byte { + file_broadcast_broadcast_proto_rawDescOnce.Do(func() { + file_broadcast_broadcast_proto_rawDescData = protoimpl.X.CompressGZIP(file_broadcast_broadcast_proto_rawDescData) + }) + return file_broadcast_broadcast_proto_rawDescData +} + +var file_broadcast_broadcast_proto_msgTypes = make([]protoimpl.MessageInfo, 3) +var file_broadcast_broadcast_proto_goTypes = []interface{}{ + (*Request)(nil), // 0: broadcast.Request + (*Response)(nil), // 1: broadcast.Response + (*Empty)(nil), // 2: broadcast.Empty +} +var file_broadcast_broadcast_proto_depIdxs = []int32{ + 0, // 0: broadcast.BroadcastService.QuorumCall:input_type -> broadcast.Request + 0, // 1: broadcast.BroadcastService.QuorumCallWithBroadcast:input_type -> broadcast.Request + 0, // 2: broadcast.BroadcastService.QuorumCallWithMulticast:input_type -> broadcast.Request + 0, // 3: broadcast.BroadcastService.Multicast:input_type -> broadcast.Request + 0, // 4: broadcast.BroadcastService.MulticastIntermediate:input_type -> broadcast.Request + 0, // 5: broadcast.BroadcastService.BroadcastCall:input_type -> broadcast.Request + 0, // 6: broadcast.BroadcastService.BroadcastIntermediate:input_type -> broadcast.Request + 0, // 7: broadcast.BroadcastService.Broadcast:input_type -> broadcast.Request + 0, // 8: broadcast.BroadcastService.BroadcastCallForward:input_type -> broadcast.Request + 0, // 9: broadcast.BroadcastService.BroadcastCallTo:input_type -> broadcast.Request + 0, // 10: broadcast.BroadcastService.BroadcastToResponse:input_type -> broadcast.Request + 0, // 11: broadcast.BroadcastService.Search:input_type -> broadcast.Request + 0, // 12: broadcast.BroadcastService.LongRunningTask:input_type -> broadcast.Request + 0, // 13: broadcast.BroadcastService.GetVal:input_type -> broadcast.Request + 0, // 14: broadcast.BroadcastService.Order:input_type -> broadcast.Request + 0, // 15: broadcast.BroadcastService.PrePrepare:input_type -> broadcast.Request + 0, // 16: broadcast.BroadcastService.Prepare:input_type -> broadcast.Request + 0, // 17: broadcast.BroadcastService.Commit:input_type -> broadcast.Request + 1, // 18: broadcast.BroadcastService.QuorumCall:output_type -> broadcast.Response + 1, // 19: broadcast.BroadcastService.QuorumCallWithBroadcast:output_type -> broadcast.Response + 1, // 20: broadcast.BroadcastService.QuorumCallWithMulticast:output_type -> broadcast.Response + 2, // 21: broadcast.BroadcastService.Multicast:output_type -> broadcast.Empty + 2, // 22: broadcast.BroadcastService.MulticastIntermediate:output_type -> broadcast.Empty + 1, // 23: broadcast.BroadcastService.BroadcastCall:output_type -> broadcast.Response + 2, // 24: broadcast.BroadcastService.BroadcastIntermediate:output_type -> broadcast.Empty + 2, // 25: broadcast.BroadcastService.Broadcast:output_type -> broadcast.Empty + 1, // 26: broadcast.BroadcastService.BroadcastCallForward:output_type -> broadcast.Response + 1, // 27: broadcast.BroadcastService.BroadcastCallTo:output_type -> broadcast.Response + 2, // 28: broadcast.BroadcastService.BroadcastToResponse:output_type -> broadcast.Empty + 1, // 29: broadcast.BroadcastService.Search:output_type -> broadcast.Response + 1, // 30: broadcast.BroadcastService.LongRunningTask:output_type -> broadcast.Response + 1, // 31: broadcast.BroadcastService.GetVal:output_type -> broadcast.Response + 1, // 32: broadcast.BroadcastService.Order:output_type -> broadcast.Response + 2, // 33: broadcast.BroadcastService.PrePrepare:output_type -> broadcast.Empty + 2, // 34: broadcast.BroadcastService.Prepare:output_type -> broadcast.Empty + 2, // 35: broadcast.BroadcastService.Commit:output_type -> broadcast.Empty + 18, // [18:36] is the sub-list for method output_type + 0, // [0:18] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_broadcast_broadcast_proto_init() } +func file_broadcast_broadcast_proto_init() { + if File_broadcast_broadcast_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_broadcast_broadcast_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_broadcast_broadcast_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_broadcast_broadcast_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Empty); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_broadcast_broadcast_proto_rawDesc, + NumEnums: 0, + NumMessages: 3, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_broadcast_broadcast_proto_goTypes, + DependencyIndexes: file_broadcast_broadcast_proto_depIdxs, + MessageInfos: file_broadcast_broadcast_proto_msgTypes, + }.Build() + File_broadcast_broadcast_proto = out.File + file_broadcast_broadcast_proto_rawDesc = nil + file_broadcast_broadcast_proto_goTypes = nil + file_broadcast_broadcast_proto_depIdxs = nil +} diff --git a/tests/broadcast/broadcast.proto b/tests/broadcast/broadcast.proto new file mode 100644 index 00000000..7bb26c39 --- /dev/null +++ b/tests/broadcast/broadcast.proto @@ -0,0 +1,77 @@ +// Test to benchmark quorum functions with and without the request parameter. +syntax = "proto3"; +package broadcast; +option go_package = "github.com/relab/gorums/tests/broadcast"; + +import "gorums.proto"; + +service BroadcastService { + rpc QuorumCall(Request) returns (Response) { + option (gorums.quorumcall) = true; + } + rpc QuorumCallWithBroadcast(Request) returns (Response) { + option (gorums.quorumcall) = true; + option (gorums.broadcast) = true; + } + rpc QuorumCallWithMulticast(Request) returns (Response) { + option (gorums.quorumcall) = true; + } + rpc Multicast(Request) returns (Empty) { + option (gorums.multicast) = true; + } + rpc MulticastIntermediate(Request) returns (Empty) { + option (gorums.multicast) = true; + } + rpc BroadcastCall(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc BroadcastIntermediate(Request) returns (Empty) { + option (gorums.broadcast) = true; + } + rpc Broadcast(Request) returns (Empty) { + option (gorums.broadcast) = true; + } + rpc BroadcastCallForward(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc BroadcastCallTo(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc BroadcastToResponse(Request) returns (Empty) { + option (gorums.broadcast) = true; + } + rpc Search(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc LongRunningTask(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc GetVal(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc Order(Request) returns (Response) { + option (gorums.broadcastcall) = true; + } + rpc PrePrepare(Request) returns (Empty) { + option (gorums.broadcast) = true; + } + rpc Prepare(Request) returns (Empty) { + option (gorums.broadcast) = true; + } + rpc Commit(Request) returns (Empty) { + option (gorums.broadcast) = true; + } +} + +message Request { + int64 Value = 1; + string From = 2; +} + +message Response { + int64 Result = 1; + int64 Replies = 2; + string From = 3; +} + +message Empty {} diff --git a/tests/broadcast/broadcast_gorums.pb.go b/tests/broadcast/broadcast_gorums.pb.go new file mode 100644 index 00000000..f7210889 --- /dev/null +++ b/tests/broadcast/broadcast_gorums.pb.go @@ -0,0 +1,1173 @@ +// Code generated by protoc-gen-gorums. DO NOT EDIT. +// versions: +// protoc-gen-gorums v0.7.0-devel +// protoc v3.12.4 +// source: broadcast/broadcast.proto + +package broadcast + +import ( + context "context" + fmt "fmt" + gorums "github.com/relab/gorums" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + encoding "google.golang.org/grpc/encoding" + status "google.golang.org/grpc/status" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + net "net" + time "time" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = gorums.EnforceVersion(7 - gorums.MinVersion) + // Verify that the gorums runtime is sufficiently up-to-date. + _ = gorums.EnforceVersion(gorums.MaxVersion - 7) +) + +// A Configuration represents a static set of nodes on which quorum remote +// procedure calls may be invoked. +type Configuration struct { + gorums.RawConfiguration + qspec QuorumSpec + srv *clientServerImpl + snowflake gorums.Snowflake + nodes []*Node +} + +// ConfigurationFromRaw returns a new Configuration from the given raw configuration and QuorumSpec. +// +// This function may for example be used to "clone" a configuration but install a different QuorumSpec: +// +// cfg1, err := mgr.NewConfiguration(qspec1, opts...) +// cfg2 := ConfigurationFromRaw(cfg1.RawConfig, qspec2) +func ConfigurationFromRaw(rawCfg gorums.RawConfiguration, qspec QuorumSpec) (*Configuration, error) { + // return an error if the QuorumSpec interface is not empty and no implementation was provided. + var test interface{} = struct{}{} + if _, empty := test.(QuorumSpec); !empty && qspec == nil { + return nil, fmt.Errorf("config: missing required QuorumSpec") + } + newCfg := &Configuration{ + RawConfiguration: rawCfg, + qspec: qspec, + } + // initialize the nodes slice + newCfg.nodes = make([]*Node, newCfg.Size()) + for i, n := range rawCfg { + newCfg.nodes[i] = &Node{n} + } + return newCfg, nil +} + +// Nodes returns a slice of each available node. IDs are returned in the same +// order as they were provided in the creation of the Manager. +// +// NOTE: mutating the returned slice is not supported. +func (c *Configuration) Nodes() []*Node { + return c.nodes +} + +// And returns a NodeListOption that can be used to create a new configuration combining c and d. +func (c Configuration) And(d *Configuration) gorums.NodeListOption { + return c.RawConfiguration.And(d.RawConfiguration) +} + +// Except returns a NodeListOption that can be used to create a new configuration +// from c without the nodes in rm. +func (c Configuration) Except(rm *Configuration) gorums.NodeListOption { + return c.RawConfiguration.Except(rm.RawConfiguration) +} + +func init() { + if encoding.GetCodec(gorums.ContentSubtype) == nil { + encoding.RegisterCodec(gorums.NewCodec()) + } +} + +// Manager maintains a connection pool of nodes on +// which quorum calls can be performed. +type Manager struct { + *gorums.RawManager + srv *clientServerImpl +} + +// NewManager returns a new Manager for managing connection to nodes added +// to the manager. This function accepts manager options used to configure +// various aspects of the manager. +func NewManager(opts ...gorums.ManagerOption) *Manager { + return &Manager{ + RawManager: gorums.NewRawManager(opts...), + } +} + +func (mgr *Manager) Close() { + if mgr.RawManager != nil { + mgr.RawManager.Close() + } + if mgr.srv != nil { + mgr.srv.stop() + } +} + +// AddClientServer starts a lightweight client-side server. This server only accepts responses +// to broadcast requests sent by the client. +// +// It is important to provide the listenAddr because this will be used to advertise the IP the +// servers should reply back to. +func (mgr *Manager) AddClientServer(lis net.Listener, clientAddr net.Addr, opts ...gorums.ServerOption) error { + options := []gorums.ServerOption{gorums.WithListenAddr(clientAddr)} + options = append(options, opts...) + srv := gorums.NewClientServer(lis, options...) + srvImpl := &clientServerImpl{ + ClientServer: srv, + } + registerClientServerHandlers(srvImpl) + go func() { + _ = srvImpl.Serve(lis) + }() + mgr.srv = srvImpl + return nil +} + +// NewConfiguration returns a configuration based on the provided list of nodes (required) +// and an optional quorum specification. The QuorumSpec is necessary for call types that +// must process replies. For configurations only used for unicast or multicast call types, +// a QuorumSpec is not needed. The QuorumSpec interface is also a ConfigOption. +// Nodes can be supplied using WithNodeMap or WithNodeList, or WithNodeIDs. +// A new configuration can also be created from an existing configuration, +// using the And, WithNewNodes, Except, and WithoutNodes methods. +func (m *Manager) NewConfiguration(opts ...gorums.ConfigOption) (c *Configuration, err error) { + if len(opts) < 1 || len(opts) > 2 { + return nil, fmt.Errorf("config: wrong number of options: %d", len(opts)) + } + c = &Configuration{} + for _, opt := range opts { + switch v := opt.(type) { + case gorums.NodeListOption: + c.RawConfiguration, err = gorums.NewRawConfiguration(m.RawManager, v) + if err != nil { + return nil, err + } + case QuorumSpec: + // Must be last since v may match QuorumSpec if it is interface{} + c.qspec = v + default: + return nil, fmt.Errorf("config: unknown option type: %v", v) + } + } + // register the client server if it exists. + // used to collect responses in BroadcastCalls + if m.srv != nil { + c.srv = m.srv + } + c.snowflake = m.Snowflake() + //var test interface{} = struct{}{} + //if _, empty := test.(QuorumSpec); !empty && c.qspec == nil { + // return nil, fmt.Errorf("config: missing required QuorumSpec") + //} + // initialize the nodes slice + c.nodes = make([]*Node, c.Size()) + for i, n := range c.RawConfiguration { + c.nodes[i] = &Node{n} + } + return c, nil +} + +// Nodes returns a slice of available nodes on this manager. +// IDs are returned in the order they were added at creation of the manager. +func (m *Manager) Nodes() []*Node { + gorumsNodes := m.RawManager.Nodes() + nodes := make([]*Node, len(gorumsNodes)) + for i, n := range gorumsNodes { + nodes[i] = &Node{n} + } + return nodes +} + +// Node encapsulates the state of a node on which a remote procedure call +// can be performed. +type Node struct { + *gorums.RawNode +} + +type Server struct { + *gorums.Server + broadcast *Broadcast + View *Configuration +} + +func NewServer(opts ...gorums.ServerOption) *Server { + srv := &Server{ + Server: gorums.NewServer(opts...), + } + b := &Broadcast{ + orchestrator: gorums.NewBroadcastOrchestrator(srv.Server), + } + srv.broadcast = b + srv.RegisterBroadcaster(newBroadcaster) + return srv +} + +func newBroadcaster(m gorums.BroadcastMetadata, o *gorums.BroadcastOrchestrator, e gorums.EnqueueBroadcast) gorums.Broadcaster { + return &Broadcast{ + orchestrator: o, + metadata: m, + srvAddrs: make([]string, 0), + enqueueBroadcast: e, + } +} + +func (srv *Server) SetView(config *Configuration) { + srv.View = config + srv.RegisterConfig(config.RawConfiguration) +} + +type Broadcast struct { + orchestrator *gorums.BroadcastOrchestrator + metadata gorums.BroadcastMetadata + srvAddrs []string + enqueueBroadcast gorums.EnqueueBroadcast +} + +// Returns a readonly struct of the metadata used in the broadcast. +// +// Note: Some of the data are equal across the cluster, such as BroadcastID. +// Other fields are local, such as SenderAddr. +func (b *Broadcast) GetMetadata() gorums.BroadcastMetadata { + return b.metadata +} + +type clientServerImpl struct { + *gorums.ClientServer + grpcServer *grpc.Server +} + +func (c *clientServerImpl) stop() { + c.ClientServer.Stop() + if c.grpcServer != nil { + c.grpcServer.Stop() + } +} + +func (b *Broadcast) To(addrs ...string) *Broadcast { + if len(addrs) <= 0 { + return b + } + b.srvAddrs = append(b.srvAddrs, addrs...) + return b +} + +func (b *Broadcast) Forward(req protoreflect.ProtoMessage, addr string) error { + if addr == "" { + return fmt.Errorf("cannot forward to empty addr, got: %s", addr) + } + if !b.metadata.IsBroadcastClient { + return fmt.Errorf("can only forward client requests") + } + go b.orchestrator.ForwardHandler(req, b.metadata.OriginMethod, b.metadata.BroadcastID, addr, b.metadata.OriginAddr) + return nil +} + +// Done signals the end of a broadcast request. It is necessary to call +// either Done() or SendToClient() to properly terminate a broadcast request +// and free up resources. Otherwise, it could cause poor performance. +func (b *Broadcast) Done() { + b.orchestrator.DoneHandler(b.metadata.BroadcastID, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (b *Broadcast) SendToClient(resp protoreflect.ProtoMessage, err error) error { + return b.orchestrator.SendToClientHandler(b.metadata.BroadcastID, resp, err, b.enqueueBroadcast) +} + +// Cancel is a non-destructive method call that will transmit a cancellation +// to all servers in the view. It will not stop the execution but will cause +// the given ServerCtx to be cancelled, making it possible to listen for +// cancellations. +// +// Could be used together with either SendToClient() or Done(). +func (b *Broadcast) Cancel() error { + return b.orchestrator.CancelHandler(b.metadata.BroadcastID, b.srvAddrs, b.enqueueBroadcast) +} + +// SendToClient sends a message back to the calling client. It also terminates +// the broadcast request, meaning subsequent messages related to the broadcast +// request will be dropped. Either SendToClient() or Done() should be used at +// the end of a broadcast request in order to free up resources. +func (srv *Server) SendToClient(resp protoreflect.ProtoMessage, err error, broadcastID uint64) error { + return srv.SendToClientHandler(resp, err, broadcastID, nil) +} + +func (b *Broadcast) QuorumCallWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastQuorumCallWithBroadcast instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.QuorumCallWithBroadcast", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) BroadcastIntermediate(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastBroadcastIntermediate instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.BroadcastIntermediate", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) Broadcast(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastBroadcast instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.Broadcast", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) BroadcastToResponse(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastBroadcastToResponse instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.BroadcastToResponse", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) PrePrepare(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastPrePrepare instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.PrePrepare", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) Prepare(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastPrepare instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.Prepare", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (b *Broadcast) Commit(req *Request, opts ...gorums.BroadcastOption) { + if b.metadata.BroadcastID == 0 { + panic("broadcastID cannot be empty. Use srv.BroadcastCommit instead") + } + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + options.ServerAddresses = append(options.ServerAddresses, b.srvAddrs...) + b.orchestrator.BroadcastHandler("broadcast.BroadcastService.Commit", req, b.metadata.BroadcastID, b.enqueueBroadcast, options) +} + +func (srv *clientServerImpl) clientBroadcastCall(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastCall(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastCallQF), "broadcast.BroadcastService.BroadcastCall") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientBroadcastCallForward(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastCallForward(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastCallForwardQF), "broadcast.BroadcastService.BroadcastCallForward") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientBroadcastCallTo(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) BroadcastCallTo(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.BroadcastCallToQF), "broadcast.BroadcastService.BroadcastCallTo") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientSearch(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) Search(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.SearchQF), "broadcast.BroadcastService.Search") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientLongRunningTask(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) LongRunningTask(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.LongRunningTaskQF), "broadcast.BroadcastService.LongRunningTask") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientGetVal(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) GetVal(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.GetValQF), "broadcast.BroadcastService.GetVal") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func (srv *clientServerImpl) clientOrder(ctx context.Context, resp *Response, broadcastID uint64) (*Response, error) { + err := srv.AddResponse(ctx, resp, broadcastID) + return resp, err +} + +func (c *Configuration) Order(ctx context.Context, in *Request, cancelOnTimeout ...bool) (resp *Response, err error) { + if c.srv == nil { + return nil, fmt.Errorf("config: a client server is not defined. Use mgr.AddClientServer() to define a client server") + } + if c.qspec == nil { + return nil, fmt.Errorf("a qspec is not defined") + } + var ( + timeout time.Duration + ok bool + response protoreflect.ProtoMessage + ) + // use the same timeout as defined in the given context. + // this is used for cancellation. + deadline, ok := ctx.Deadline() + if ok { + timeout = deadline.Sub(time.Now()) + } else { + timeout = 5 * time.Second + } + broadcastID := c.snowflake.NewBroadcastID() + doneChan, cd := c.srv.AddRequest(broadcastID, ctx, in, gorums.ConvertToType(c.qspec.OrderQF), "broadcast.BroadcastService.Order") + c.RawConfiguration.BroadcastCall(ctx, cd, gorums.WithNoSendWaiting(), gorums.WithOriginAuthentication()) + select { + case response, ok = <-doneChan: + case <-ctx.Done(): + if len(cancelOnTimeout) > 0 && cancelOnTimeout[0] { + go func() { + bd := gorums.BroadcastCallData{ + Method: gorums.Cancellation, + BroadcastID: broadcastID, + } + cancelCtx, cancelCancel := context.WithTimeout(context.Background(), timeout) + defer cancelCancel() + c.RawConfiguration.BroadcastCall(cancelCtx, bd) + }() + } + return nil, fmt.Errorf("context cancelled") + } + if !ok { + return nil, fmt.Errorf("done channel was closed before returning a value") + } + resp, ok = response.(*Response) + if !ok { + return nil, fmt.Errorf("wrong proto format") + } + return resp, nil +} + +func registerClientServerHandlers(srv *clientServerImpl) { + + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCall", gorums.ClientHandler(srv.clientBroadcastCall)) + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCallForward", gorums.ClientHandler(srv.clientBroadcastCallForward)) + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCallTo", gorums.ClientHandler(srv.clientBroadcastCallTo)) + srv.RegisterHandler("broadcast.BroadcastService.Search", gorums.ClientHandler(srv.clientSearch)) + srv.RegisterHandler("broadcast.BroadcastService.LongRunningTask", gorums.ClientHandler(srv.clientLongRunningTask)) + srv.RegisterHandler("broadcast.BroadcastService.GetVal", gorums.ClientHandler(srv.clientGetVal)) + srv.RegisterHandler("broadcast.BroadcastService.Order", gorums.ClientHandler(srv.clientOrder)) +} + +// Multicast is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) Multicast(ctx context.Context, in *Request, opts ...gorums.CallOption) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "broadcast.BroadcastService.Multicast", + } + + c.RawConfiguration.Multicast(ctx, cd, opts...) +} + +// MulticastIntermediate is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) MulticastIntermediate(ctx context.Context, in *Request, opts ...gorums.CallOption) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "broadcast.BroadcastService.MulticastIntermediate", + } + + c.RawConfiguration.Multicast(ctx, cd, opts...) +} + +// QuorumSpec is the interface of quorum functions for BroadcastService. +type QuorumSpec interface { + gorums.ConfigOption + + // QuorumCallQF is the quorum function for the QuorumCall + // quorum call method. The in parameter is the request object + // supplied to the QuorumCall method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + QuorumCallQF(in *Request, replies map[uint32]*Response) (*Response, bool) + + // QuorumCallWithBroadcastQF is the quorum function for the QuorumCallWithBroadcast + // quorum call method. The in parameter is the request object + // supplied to the QuorumCallWithBroadcast method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + QuorumCallWithBroadcastQF(in *Request, replies map[uint32]*Response) (*Response, bool) + + // QuorumCallWithMulticastQF is the quorum function for the QuorumCallWithMulticast + // quorum call method. The in parameter is the request object + // supplied to the QuorumCallWithMulticast method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + QuorumCallWithMulticastQF(in *Request, replies map[uint32]*Response) (*Response, bool) + + // BroadcastCallQF is the quorum function for the BroadcastCall + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastCall method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastCallQF(in *Request, replies []*Response) (*Response, bool) + + // BroadcastCallForwardQF is the quorum function for the BroadcastCallForward + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastCallForward method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastCallForwardQF(in *Request, replies []*Response) (*Response, bool) + + // BroadcastCallToQF is the quorum function for the BroadcastCallTo + // broadcastcall call method. The in parameter is the request object + // supplied to the BroadcastCallTo method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + BroadcastCallToQF(in *Request, replies []*Response) (*Response, bool) + + // SearchQF is the quorum function for the Search + // broadcastcall call method. The in parameter is the request object + // supplied to the Search method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + SearchQF(in *Request, replies []*Response) (*Response, bool) + + // LongRunningTaskQF is the quorum function for the LongRunningTask + // broadcastcall call method. The in parameter is the request object + // supplied to the LongRunningTask method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + LongRunningTaskQF(in *Request, replies []*Response) (*Response, bool) + + // GetValQF is the quorum function for the GetVal + // broadcastcall call method. The in parameter is the request object + // supplied to the GetVal method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + GetValQF(in *Request, replies []*Response) (*Response, bool) + + // OrderQF is the quorum function for the Order + // broadcastcall call method. The in parameter is the request object + // supplied to the Order method at call time, and may or may not + // be used by the quorum function. If the in parameter is not needed + // you should implement your quorum function with '_ *Request'. + OrderQF(in *Request, replies []*Response) (*Response, bool) +} + +// QuorumCall is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) QuorumCall(ctx context.Context, in *Request) (resp *Response, err error) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "broadcast.BroadcastService.QuorumCall", + } + cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + r := make(map[uint32]*Response, len(replies)) + for k, v := range replies { + r[k] = v.(*Response) + } + return c.qspec.QuorumCallQF(req.(*Request), r) + } + + res, err := c.RawConfiguration.QuorumCall(ctx, cd) + if err != nil { + return nil, err + } + return res.(*Response), err +} + +// QuorumCallWithBroadcast is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) QuorumCallWithBroadcast(ctx context.Context, in *Request) (resp *Response, err error) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "broadcast.BroadcastService.QuorumCallWithBroadcast", + + BroadcastID: c.snowflake.NewBroadcastID(), + IsBroadcastClient: true, + } + cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + r := make(map[uint32]*Response, len(replies)) + for k, v := range replies { + r[k] = v.(*Response) + } + return c.qspec.QuorumCallWithBroadcastQF(req.(*Request), r) + } + + res, err := c.RawConfiguration.QuorumCall(ctx, cd) + if err != nil { + return nil, err + } + return res.(*Response), err +} + +// QuorumCallWithMulticast is a quorum call invoked on all nodes in configuration c, +// with the same argument in, and returns a combined result. +func (c *Configuration) QuorumCallWithMulticast(ctx context.Context, in *Request) (resp *Response, err error) { + cd := gorums.QuorumCallData{ + Message: in, + Method: "broadcast.BroadcastService.QuorumCallWithMulticast", + } + cd.QuorumFunction = func(req protoreflect.ProtoMessage, replies map[uint32]protoreflect.ProtoMessage) (protoreflect.ProtoMessage, bool) { + r := make(map[uint32]*Response, len(replies)) + for k, v := range replies { + r[k] = v.(*Response) + } + return c.qspec.QuorumCallWithMulticastQF(req.(*Request), r) + } + + res, err := c.RawConfiguration.QuorumCall(ctx, cd) + if err != nil { + return nil, err + } + return res.(*Response), err +} + +// BroadcastService is the server-side API for the BroadcastService Service +type BroadcastService interface { + QuorumCall(ctx gorums.ServerCtx, request *Request) (response *Response, err error) + QuorumCallWithBroadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + QuorumCallWithMulticast(ctx gorums.ServerCtx, request *Request) (response *Response, err error) + Multicast(ctx gorums.ServerCtx, request *Request) + MulticastIntermediate(ctx gorums.ServerCtx, request *Request) + BroadcastCall(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastIntermediate(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + Broadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastCallForward(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastCallTo(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + BroadcastToResponse(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + Search(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + LongRunningTask(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + GetVal(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + Order(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + PrePrepare(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + Prepare(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) + Commit(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) +} + +func (srv *Server) QuorumCall(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCall not implemented")) +} +func (srv *Server) QuorumCallWithBroadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallWithBroadcast not implemented")) +} +func (srv *Server) QuorumCallWithMulticast(ctx gorums.ServerCtx, request *Request) (response *Response, err error) { + panic(status.Errorf(codes.Unimplemented, "method QuorumCallWithMulticast not implemented")) +} +func (srv *Server) Multicast(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method Multicast not implemented")) +} +func (srv *Server) MulticastIntermediate(ctx gorums.ServerCtx, request *Request) { + panic(status.Errorf(codes.Unimplemented, "method MulticastIntermediate not implemented")) +} +func (srv *Server) BroadcastCall(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastCall not implemented")) +} +func (srv *Server) BroadcastIntermediate(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastIntermediate not implemented")) +} +func (srv *Server) Broadcast(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method Broadcast not implemented")) +} +func (srv *Server) BroadcastCallForward(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastCallForward not implemented")) +} +func (srv *Server) BroadcastCallTo(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastCallTo not implemented")) +} +func (srv *Server) BroadcastToResponse(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method BroadcastToResponse not implemented")) +} +func (srv *Server) Search(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method Search not implemented")) +} +func (srv *Server) LongRunningTask(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method LongRunningTask not implemented")) +} +func (srv *Server) GetVal(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method GetVal not implemented")) +} +func (srv *Server) Order(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method Order not implemented")) +} +func (srv *Server) PrePrepare(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method PrePrepare not implemented")) +} +func (srv *Server) Prepare(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method Prepare not implemented")) +} +func (srv *Server) Commit(ctx gorums.ServerCtx, request *Request, broadcast *Broadcast) { + panic(status.Errorf(codes.Unimplemented, "method Commit not implemented")) +} + +func RegisterBroadcastServiceServer(srv *Server, impl BroadcastService) { + srv.RegisterHandler("broadcast.BroadcastService.QuorumCall", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { + req := in.Message.(*Request) + defer ctx.Release() + resp, err := impl.QuorumCall(ctx, req) + gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) + }) + srv.RegisterHandler("broadcast.BroadcastService.QuorumCallWithBroadcast", gorums.BroadcastHandler(impl.QuorumCallWithBroadcast, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.QuorumCallWithMulticast", func(ctx gorums.ServerCtx, in *gorums.Message, finished chan<- *gorums.Message) { + req := in.Message.(*Request) + defer ctx.Release() + resp, err := impl.QuorumCallWithMulticast(ctx, req) + gorums.SendMessage(ctx, finished, gorums.WrapMessage(in.Metadata, resp, err)) + }) + srv.RegisterHandler("broadcast.BroadcastService.Multicast", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*Request) + defer ctx.Release() + impl.Multicast(ctx, req) + }) + srv.RegisterHandler("broadcast.BroadcastService.MulticastIntermediate", func(ctx gorums.ServerCtx, in *gorums.Message, _ chan<- *gorums.Message) { + req := in.Message.(*Request) + defer ctx.Release() + impl.MulticastIntermediate(ctx, req) + }) + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCall", gorums.BroadcastHandler(impl.BroadcastCall, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.BroadcastCall") + srv.RegisterHandler("broadcast.BroadcastService.BroadcastIntermediate", gorums.BroadcastHandler(impl.BroadcastIntermediate, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.Broadcast", gorums.BroadcastHandler(impl.Broadcast, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCallForward", gorums.BroadcastHandler(impl.BroadcastCallForward, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.BroadcastCallForward") + srv.RegisterHandler("broadcast.BroadcastService.BroadcastCallTo", gorums.BroadcastHandler(impl.BroadcastCallTo, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.BroadcastCallTo") + srv.RegisterHandler("broadcast.BroadcastService.BroadcastToResponse", gorums.BroadcastHandler(impl.BroadcastToResponse, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.Search", gorums.BroadcastHandler(impl.Search, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.Search") + srv.RegisterHandler("broadcast.BroadcastService.LongRunningTask", gorums.BroadcastHandler(impl.LongRunningTask, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.LongRunningTask") + srv.RegisterHandler("broadcast.BroadcastService.GetVal", gorums.BroadcastHandler(impl.GetVal, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.GetVal") + srv.RegisterHandler("broadcast.BroadcastService.Order", gorums.BroadcastHandler(impl.Order, srv.Server)) + srv.RegisterClientHandler("broadcast.BroadcastService.Order") + srv.RegisterHandler("broadcast.BroadcastService.PrePrepare", gorums.BroadcastHandler(impl.PrePrepare, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.Prepare", gorums.BroadcastHandler(impl.Prepare, srv.Server)) + srv.RegisterHandler("broadcast.BroadcastService.Commit", gorums.BroadcastHandler(impl.Commit, srv.Server)) + srv.RegisterHandler(gorums.Cancellation, gorums.BroadcastHandler(gorums.CancelFunc, srv.Server)) +} + +func (srv *Server) BroadcastQuorumCallWithBroadcast(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.QuorumCallWithBroadcast", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.QuorumCallWithBroadcast", req, options) + } +} + +func (srv *Server) BroadcastBroadcastIntermediate(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.BroadcastIntermediate", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.BroadcastIntermediate", req, options) + } +} + +func (srv *Server) BroadcastBroadcast(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.Broadcast", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.Broadcast", req, options) + } +} + +func (srv *Server) BroadcastBroadcastToResponse(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.BroadcastToResponse", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.BroadcastToResponse", req, options) + } +} + +func (srv *Server) BroadcastPrePrepare(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.PrePrepare", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.PrePrepare", req, options) + } +} + +func (srv *Server) BroadcastPrepare(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.Prepare", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.Prepare", req, options) + } +} + +func (srv *Server) BroadcastCommit(req *Request, opts ...gorums.BroadcastOption) { + options := gorums.NewBroadcastOptions() + for _, opt := range opts { + opt(&options) + } + if options.RelatedToReq > 0 { + srv.broadcast.orchestrator.BroadcastHandler("broadcast.BroadcastService.Commit", req, options.RelatedToReq, nil, options) + } else { + srv.broadcast.orchestrator.ServerBroadcastHandler("broadcast.BroadcastService.Commit", req, options) + } +} + +const ( + BroadcastServiceQuorumCallWithBroadcast string = "broadcast.BroadcastService.QuorumCallWithBroadcast" + BroadcastServiceBroadcastCall string = "broadcast.BroadcastService.BroadcastCall" + BroadcastServiceBroadcastIntermediate string = "broadcast.BroadcastService.BroadcastIntermediate" + BroadcastServiceBroadcast string = "broadcast.BroadcastService.Broadcast" + BroadcastServiceBroadcastCallForward string = "broadcast.BroadcastService.BroadcastCallForward" + BroadcastServiceBroadcastCallTo string = "broadcast.BroadcastService.BroadcastCallTo" + BroadcastServiceBroadcastToResponse string = "broadcast.BroadcastService.BroadcastToResponse" + BroadcastServiceSearch string = "broadcast.BroadcastService.Search" + BroadcastServiceLongRunningTask string = "broadcast.BroadcastService.LongRunningTask" + BroadcastServiceGetVal string = "broadcast.BroadcastService.GetVal" + BroadcastServiceOrder string = "broadcast.BroadcastService.Order" + BroadcastServicePrePrepare string = "broadcast.BroadcastService.PrePrepare" + BroadcastServicePrepare string = "broadcast.BroadcastService.Prepare" + BroadcastServiceCommit string = "broadcast.BroadcastService.Commit" +) + +type internalResponse struct { + nid uint32 + reply *Response + err error +} diff --git a/tests/broadcast/broadcast_test.go b/tests/broadcast/broadcast_test.go new file mode 100644 index 00000000..867c7943 --- /dev/null +++ b/tests/broadcast/broadcast_test.go @@ -0,0 +1,1467 @@ +package broadcast + +import ( + "context" + fmt "fmt" + net "net" + _ "net/http/pprof" + "os" + "runtime" + "runtime/pprof" + "sync" + "testing" + "time" +) + +func createAuthServers(numSrvs int) ([]*testServer, []string, func(), error) { + skip := 0 + srvs := make([]*testServer, 0, numSrvs) + srvAddrs := make([]string, numSrvs) + for i := 0; i < numSrvs; i++ { + srvAddrs[i] = fmt.Sprintf("127.0.0.1:500%v", i) + } + for _, addr := range srvAddrs { + if skip > 0 { + skip-- + continue + } + srv := newAuthenticatedServer(addr, srvAddrs) + lis, err := net.Listen("tcp4", srv.addr) + if err != nil { + return nil, nil, nil, err + } + srv.lis = lis + go srv.start(lis) + srvs = append(srvs, srv) + } + return srvs, srvAddrs, func() { + // stop the servers + for _, srv := range srvs { + srv.Stop() + } + }, nil +} + +func createSrvs(numSrvs int, down ...int) ([]*testServer, []string, func(), error) { + ordering := false + skip := 0 + if len(down) > 0 { + skip = down[0] + if skip > numSrvs { + skip = 0 + ordering = true + } + } + srvs := make([]*testServer, 0, numSrvs) + srvAddrs := make([]string, numSrvs) + for i := 0; i < numSrvs; i++ { + srvAddrs[i] = fmt.Sprintf("127.0.0.1:500%v", i) + } + for i, addr := range srvAddrs { + if skip > 0 { + skip-- + continue + } + var srv *testServer + if ordering { + srv = newtestServer(addr, srvAddrs, i, true) + } else { + srv = newtestServer(addr, srvAddrs, i) + } + lis, err := net.Listen("tcp4", srv.addr) + if err != nil { + return nil, nil, nil, err + } + srv.lis = lis + go srv.start(lis) + srvs = append(srvs, srv) + } + return srvs, srvAddrs, func() { + // stop the servers + for _, srv := range srvs { + srv.Stop() + } + }, nil +} + +func TestSimpleBroadcastCall(t *testing.T) { + numSrvs := 3 + numReqs := 10 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.BroadcastCall(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Errorf("resp is wrong, want: %v, got: %v", val, resp.GetResult()) + } + cancel() + } +} + +func TestSimpleBroadcastTo(t *testing.T) { + numSrvs := 3 + numReqs := 10 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + // only want a response from the leader, hence qsize = 1 + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080", 1) + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.BroadcastCallTo(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetFrom() != leader { + t.Errorf("resp is wrong, want: %v, got: %v", leader, resp.GetFrom()) + } + cancel() + } +} + +func TestSimpleBroadcastCancel(t *testing.T) { + numSrvs := 3 + numReqs := 10 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 10) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.Search(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != 1 { + t.Errorf("resp is wrong, want: %v, got: %v", 1, resp.GetResult()) + } + cancel() + } +} + +func TestBroadcastCancel(t *testing.T) { + numSrvs := 3 + numReqs := 3 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + _, _ = config.LongRunningTask(ctx, &Request{Value: val}, true) + cancel() + // wait until cancel has reaced the servers before asking for the result + time.Sleep(1 * time.Second) + ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.GetVal(ctx, &Request{Value: val}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != 1 { + t.Errorf("resp is wrong, want: %v, got: %v", 1, resp.GetResult()) + } + } +} + +func TestBroadcastCancelOneSrvDown(t *testing.T) { + numSrvs := 3 + numReqs := 10 + srvs, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + // stop one of the servers + srvs[numSrvs-1].Stop() + if err != nil { + t.Error(err) + } + defer srvCleanup() + + // only want response from the online servers + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080", numSrvs-1) + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + _, _ = config.LongRunningTask(ctx, &Request{Value: val}, true) + cancel() + ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.GetVal(ctx, &Request{Value: val}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != 1 { + t.Errorf("resp is wrong, want: %v, got: %v", 1, resp.GetResult()) + } + } +} + +func TestBroadcastCancelOneSrvFails(t *testing.T) { + numSrvs := 3 + srvs, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + // only want response from the online servers + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080", numSrvs-1) + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + _, _ = config.LongRunningTask(ctx, &Request{Value: val}, true) + cancel() + + // stop one of the servers + srvs[numSrvs-1].Stop() + + ctx, cancel = context.WithTimeout(context.Background(), 5*time.Second) + resp, err := config.GetVal(ctx, &Request{Value: val}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != 1 { + t.Errorf("resp is wrong, want: %v, got: %v", 1, resp.GetResult()) + } +} + +func TestBroadcastCancelOneClientFails(t *testing.T) { + numSrvs := 3 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + // only want response from the online servers + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + + val := int64(100) + go func() { + _, _ = config.LongRunningTask(context.Background(), &Request{Value: val}, true) + }() + + // make sure the request is sent and stop the client + time.Sleep(100 * time.Millisecond) + clientCleanup() + + // only want response from the online servers + config2, clientCleanup2, err2 := newClient(srvAddrs, "127.0.0.1:8081") + defer clientCleanup2() + if err2 != nil { + t.Error(err2) + } + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + resp, err := config2.GetVal(ctx, &Request{Value: val}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != 1 { + t.Errorf("resp is wrong, want: %v, got: %v", 1, resp.GetResult()) + } +} + +func TestBroadcastCallOrderingSendToOneSrv(t *testing.T) { + numSrvs := 3 + numReqs := 3 + useOrdering := 2 * numSrvs + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs, useOrdering) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[1:2], "127.0.0.1:8080", len(srvAddrs)) + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.Order(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != 0 { + t.Errorf("resp is wrong, want: %v, got: %v", 0, resp.GetResult()) + } + cancel() + } +} + +func TestBroadcastCallOrderingSendToAllSrvs(t *testing.T) { + numSrvs := 3 + numReqs := 3 + useOrdering := 2 * numSrvs + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs, useOrdering) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.Order(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != 0 { + t.Errorf("resp is wrong, want: %v, got: %v", 0, resp.GetResult()) + } + cancel() + } +} + +func TestBroadcastCallOrderingDoesNotInterfereWithMethodsNotSpecifiedInOrder(t *testing.T) { + numSrvs := 3 + numReqs := 10 + useOrdering := 2 * numSrvs + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs, useOrdering) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + // BroadcastCall is not specified in the order + resp, err := config.BroadcastCall(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Errorf("resp is wrong, want: %v, got: %v", val, resp.GetResult()) + } + cancel() + } +} + +func TestBroadcastCallRace(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(1) + resp, err := config.BroadcastCall(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + for i := 0; i <= 100; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } +} + +func TestBroadcastCallClientKnowsOnlyOneServer(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[0:1], "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(1) + resp, err := config.BroadcastCall(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + for i := 0; i <= 100; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } +} + +func TestBroadcastCallOneServerIsDown(t *testing.T) { + numSrvs := 3 + skip := 1 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs, skip) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + start := skip + if start < 0 { + start = 0 + } + end := numSrvs - 1 + if end > len(srvAddrs) { + end = len(srvAddrs) + } + config, clientCleanup, err := newClient(srvAddrs[start:end], "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(1) + resp, err := config.BroadcastCall(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + for i := 0; i <= 100; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } +} + +func TestBroadcastCallForward(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[1:2], "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 0; i < 10; i++ { + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + defer cancel() + resp, err := config.BroadcastCallForward(ctx, &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } +} + +func TestBroadcastCallForwardMultiple(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[1:], "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 0; i < 10; i++ { + resp, err := config.BroadcastCallForward(context.Background(), &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } +} + +func TestBroadcastCallRaceTwoClients(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + client1, clientCleanup1, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup1() + + client2, clientCleanup2, err := newClient(srvAddrs, "127.0.0.1:8081") + if err != nil { + t.Error(err) + } + defer clientCleanup2() + + val := int64(1) + resp, err := client1.BroadcastCall(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + resp, err = client2.BroadcastCall(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + + timeout := 3 * time.Second + var wg sync.WaitGroup + for i := 0; i <= 100; i++ { + go func(j int) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + resp, err := client1.BroadcastCall(ctx, &Request{Value: int64(j)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(j) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), j) + } + wg.Done() + }(i) + go func(j int) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + resp, err := client2.BroadcastCall(ctx, &Request{Value: int64(j)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(j) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), j) + } + wg.Done() + }(i) + wg.Add(2) + } + wg.Wait() +} + +func TestBroadcastCallAsyncReqs(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + numClients := 10 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs, fmt.Sprintf("127.0.0.1:808%v", c), 3) + if err != nil { + t.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := client.BroadcastCall(ctx, &Request{Value: int64(init)}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(init) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + var wg sync.WaitGroup + for i := 0; i < 1; i++ { + for _, client := range clients { + go func(j int, c *Configuration) { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := c.BroadcastCall(ctx, &Request{Value: int64(j)}) + cancel() + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(j) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), j) + } + wg.Done() + }(i, client) + wg.Add(1) + } + } + wg.Wait() +} + +func TestQCBroadcastOptionRace(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(1) + resp, err := config.QuorumCallWithBroadcast(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatalf("resp is wrong, got: %v, want: %v", resp.GetResult(), val) + } + for i := 0; i < 100; i++ { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + resp, err := config.QuorumCallWithBroadcast(ctx, &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + cancel() + } +} + +func TestQCMulticastRace(t *testing.T) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + val := int64(1) + resp, err := config.QuorumCallWithMulticast(context.Background(), &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Fatal("resp is wrong") + } + for i := 0; i < 100; i++ { + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + resp, err := config.QuorumCallWithMulticast(ctx, &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + cancel() + } +} + +func BenchmarkQuorumCall(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "") + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.QuorumCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileQF") + memProfile, _ := os.Create("memprofileQF") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("QC_AllSuccessful_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := config.QuorumCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkQCMulticast(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "") + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.QuorumCallWithMulticast(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileQFwithB") + memProfile, _ := os.Create("memprofileQFwithB") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("QCM_AllSuccessful_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + resp, err := config.QuorumCallWithMulticast(ctx, &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + cancel() + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkQCBroadcastOption(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "") + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.QuorumCallWithBroadcast(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileQFwithB") + memProfile, _ := os.Create("memprofileQFwithB") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("QCB_AllSuccessful_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + resp, err := config.QuorumCallWithBroadcast(ctx, &Request{Value: int64(i)}) + if err != nil { + b.Error(err, i) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + cancel() + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkQCBroadcastOptionManyClients(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + numClients := 10 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs, fmt.Sprintf("127.0.0.1:808%v", c), 3) + if err != nil { + b.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + resp, err := client.QuorumCallWithBroadcast(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + cpuProfile, _ := os.Create("cpuprofileQCB") + memProfile, _ := os.Create("memprofileQCB") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("QCB_ManyClients_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + for _, client := range clients { + wg.Add(1) + go func(i int, c *Configuration) { + defer wg.Done() + resp, err := c.QuorumCallWithBroadcast(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + }(i, client) + } + wg.Wait() + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkBroadcastCallAllServers(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileBC") + memProfile, _ := os.Create("memprofileBC") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("BC_AllSuccessful_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkBroadcastCallToOneServer(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[0:1], "127.0.0.1:8080", 3) + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileBC") + memProfile, _ := os.Create("memprofileBC") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("BC_OneSrv_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkBroadcastCallOneFailedServer(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3, 1) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs, "127.0.0.1:8080", 2) + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileBC") + memProfile, _ := os.Create("memprofileBC") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("BC_OneSrvDown_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkBroadcastCallOneDownSrvToOneSrv(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3, 1) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newClient(srvAddrs[1:2], "127.0.0.1:8080", 2) + if err != nil { + b.Error(err) + } + defer clientCleanup() + + init := 1 + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + + cpuProfile, _ := os.Create("cpuprofileBC") + memProfile, _ := os.Create("memprofileBC") + _ = pprof.StartCPUProfile(cpuProfile) + + b.Run(fmt.Sprintf("BC_OneDownToOne_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := config.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + pprof.StopCPUProfile() + _ = pprof.WriteHeapProfile(memProfile) +} + +func BenchmarkBroadcastCallManyClients(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + numClients := 10 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs[0:1], fmt.Sprintf("127.0.0.1:%v", 8080+c), 3) + if err != nil { + b.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + resp, err := client.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + b.Run(fmt.Sprintf("BC_OneClientOneReq_%d", 0), func(b *testing.B) { + for i := 0; i < b.N; i++ { + resp, err := clients[0].BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + b.Run(fmt.Sprintf("BC_OneClientAsync_%d", 1), func(b *testing.B) { + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + for j := range clients { + go func(i, j int, c *Configuration) { + val := i*100 + j + resp, err := c.BroadcastCall(context.Background(), &Request{Value: int64(val)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(val) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), val) + } + wg.Done() + }(i, j, clients[0]) + wg.Add(1) + } + wg.Wait() + } + }) + b.Run(fmt.Sprintf("BC_OneClientSync_%d", 2), func(b *testing.B) { + for i := 0; i < b.N; i++ { + for j := range clients { + val := i*100 + j + resp, err := clients[0].BroadcastCall(context.Background(), &Request{Value: int64(val)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(val) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), val) + } + } + } + }) + b.Run(fmt.Sprintf("BC_ManyClientsAsync_%d", 3), func(b *testing.B) { + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + for _, client := range clients { + go func(i int, c *Configuration) { + resp, err := c.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + wg.Done() + }(i, client) + wg.Add(1) + } + wg.Wait() + } + }) + b.Run(fmt.Sprintf("BC_ManyClientsSync_%d", 4), func(b *testing.B) { + for i := 0; i < b.N; i++ { + for _, client := range clients { + resp, err := client.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + } + }) +} + +func BenchmarkBroadcastCallTenClientsCPU(b *testing.B) { + _, srvAddrs, srvCleanup, err := createSrvs(3) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + numClients := 10 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs[0:1], fmt.Sprintf("127.0.0.1:%v", 8080+c), 3) + if err != nil { + b.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + resp, err := client.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + cpuProfile, _ := os.Create("cpuprofileBC") + err = pprof.StartCPUProfile(cpuProfile) + if err != nil { + b.Error(err) + } + + b.ResetTimer() + b.Run(fmt.Sprintf("BC_%d", 3), func(b *testing.B) { + for i := 0; i < b.N; i++ { + var wg sync.WaitGroup + for _, client := range clients { + go func(i int, c *Configuration) { + resp, err := c.BroadcastCall(context.Background(), &Request{Value: int64(i)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + wg.Done() + }(i, client) + wg.Add(1) + } + wg.Wait() + } + }) + b.StopTimer() + pprof.StopCPUProfile() + cpuProfile.Close() +} + +func BenchmarkBroadcastCallTenClientsMEM(b *testing.B) { + numSrvs := 3 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + numClients := 1 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs[0:1], fmt.Sprintf("127.0.0.1:%v", 8080+c), numSrvs) + if err != nil { + b.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + resp, err := client.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + memProfile, _ := os.Create("memprofileBC") + runtime.GC() + + b.ResetTimer() + for _, client := range clients { + b.RunParallel(func(pb *testing.PB) { + for i := 0; pb.Next(); i++ { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := client.BroadcastCall(ctx, &Request{Value: int64(i)}) + cancel() + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + } + b.StopTimer() + _ = pprof.WriteHeapProfile(memProfile) + memProfile.Close() +} + +func BenchmarkBroadcastCallTenClientsTRACE(b *testing.B) { + numSrvs := 1 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + b.Error(err) + } + defer srvCleanup() + + numClients := 1 + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs[0:1], fmt.Sprintf("127.0.0.1:%v", 8080+c), numSrvs) + if err != nil { + b.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + for _, client := range clients { + init := 1 + resp, err := client.BroadcastCall(context.Background(), &Request{Value: int64(init)}) + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(init) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + } + + stop, err := StartTrace("traceprofileBC") + if err != nil { + b.Error(err) + } + defer func() { + _ = stop() + }() + + b.ResetTimer() + for _, client := range clients { + b.RunParallel(func(pb *testing.PB) { + for i := 0; pb.Next(); i++ { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := client.BroadcastCall(ctx, &Request{Value: int64(i)}) + cancel() + if err != nil { + b.Error(err) + } + if resp.GetResult() != int64(i) { + b.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + } + }) + } + b.StopTimer() +} + +func TestBroadcastCallManyRequestsAsync(t *testing.T) { + numSrvs := 3 + numClients := 20 + numReqs := 10 + _, srvAddrs, srvCleanup, err := createSrvs(numSrvs) + if err != nil { + t.Error(err) + } + + clients := make([]*Configuration, numClients) + for c := 0; c < numClients; c++ { + config, clientCleanup, err := newClient(srvAddrs[0:1], fmt.Sprintf("127.0.0.1:%v", 8080+c), numSrvs) + if err != nil { + t.Error(err) + } + defer clientCleanup() + clients[c] = config + } + + var wg1 sync.WaitGroup + for _, client := range clients { + init := 1 + wg1.Add(1) + go func(client *Configuration) { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + resp, err := client.BroadcastCall(ctx, &Request{Value: int64(init)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(init) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), init) + } + wg1.Done() + }(client) + } + wg1.Wait() + time.Sleep(500 * time.Millisecond) + + var wg sync.WaitGroup + for r := 0; r < numReqs; r++ { + for i, client := range clients { + wg.Add(1) + go func(i int, client *Configuration) { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + resp, err := client.BroadcastCall(ctx, &Request{Value: int64(i)}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != int64(i) { + t.Errorf("result is wrong. got: %v, want: %v", resp.GetResult(), i) + } + wg.Done() + }(i, client) + } + } + wg.Wait() + srvCleanup() +} + +func TestAuthenticationBroadcastCall(t *testing.T) { + numSrvs := 3 + numReqs := 10 + _, srvAddrs, srvCleanup, err := createAuthServers(numSrvs) + if err != nil { + t.Error(err) + } + defer srvCleanup() + + config, clientCleanup, err := newAuthClient(srvAddrs, "127.0.0.1:8080") + if err != nil { + t.Error(err) + } + defer clientCleanup() + + for i := 1; i <= numReqs; i++ { + val := int64(i * 100) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + resp, err := config.BroadcastCall(ctx, &Request{Value: val}) + if err != nil { + t.Error(err) + } + if resp.GetResult() != val { + t.Errorf("resp is wrong, want: %v, got: %v", val, resp.GetResult()) + } + cancel() + } +} diff --git a/tests/broadcast/client.go b/tests/broadcast/client.go new file mode 100644 index 00000000..f447f6c1 --- /dev/null +++ b/tests/broadcast/client.go @@ -0,0 +1,203 @@ +package broadcast + +import ( + "crypto/elliptic" + "log/slog" + net "net" + + gorums "github.com/relab/gorums" + grpc "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +type testQSpec struct { + quorumSize int + broadcastSize int +} + +func newQSpec(qSize, broadcastSize int) QuorumSpec { + return &testQSpec{ + quorumSize: qSize, + broadcastSize: broadcastSize, + } +} + +func (qs *testQSpec) QuorumCallQF(in *Request, replies map[uint32]*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) QuorumCallWithBroadcastQF(in *Request, replies map[uint32]*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) QuorumCallWithMulticastQF(in *Request, replies map[uint32]*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) BroadcastCallQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) BroadcastCallForwardQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) BroadcastCallToQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + return resp, true + } + } + return nil, false +} + +func (qs *testQSpec) SearchQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) < qs.quorumSize { + return nil, false + } + numCorrect := 0 + for _, resp := range replies { + if resp.Result == 1 { + numCorrect++ + } + } + if numCorrect == 1 { + return &Response{Result: 1}, true + } + slog.Info("got wrong res", "replies", replies) + return &Response{Result: 0}, true +} + +func (qs *testQSpec) LongRunningTaskQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + return nil, true + } + return nil, false +} + +func (qs *testQSpec) GetValQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, reply := range replies { + // all responses should be cancelled + if reply.GetResult() != 1 { + return &Response{Result: 0}, true + } + } + return &Response{Result: 1}, true + } + return nil, false +} + +func (qs *testQSpec) OrderQF(in *Request, replies []*Response) (*Response, bool) { + if len(replies) >= qs.quorumSize { + for _, resp := range replies { + if resp.GetResult() != 0 { + return resp, true + } + } + return &Response{ + Result: 0, + }, true + } + return nil, false +} + +func newClient(srvAddrs []string, listenAddr string, qsize ...int) (*Configuration, func(), error) { + quorumSize := len(srvAddrs) + if len(qsize) > 0 { + quorumSize = qsize[0] + } + mgr := NewManager( + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + if listenAddr != "" { + lis, err := net.Listen("tcp", "127.0.0.1:") + if err != nil { + return nil, nil, err + } + err = mgr.AddClientServer(lis, lis.Addr()) + if err != nil { + return nil, nil, err + } + } + config, err := mgr.NewConfiguration( + gorums.WithNodeList(srvAddrs), + newQSpec(quorumSize, quorumSize), + ) + if err != nil { + return nil, nil, err + } + return config, func() { + mgr.Close() + }, nil +} + +func newAuthClient(srvAddrs []string, listenAddr string, qsize ...int) (*Configuration, func(), error) { + quorumSize := len(srvAddrs) + if len(qsize) > 0 { + quorumSize = qsize[0] + } + var ( + lis net.Listener + err error + ) + if listenAddr != "" { + lis, err = net.Listen("tcp", "127.0.0.1:") + if err != nil { + return nil, nil, err + } + } + auth := gorums.NewAuth(elliptic.P256()) + _ = auth.GenerateKeys() + privKey, pubKey := auth.Keys() + auth.RegisterKeys(lis.Addr(), privKey, pubKey) + mgr := NewManager( + gorums.WithAuthentication(auth), + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + if listenAddr != "" { + err = mgr.AddClientServer(lis, lis.Addr()) + if err != nil { + return nil, nil, err + } + } + config, err := mgr.NewConfiguration( + gorums.WithNodeList(srvAddrs), + newQSpec(quorumSize, quorumSize), + ) + if err != nil { + return nil, nil, err + } + return config, func() { + mgr.Close() + }, nil +} diff --git a/tests/broadcast/profiling.go b/tests/broadcast/profiling.go new file mode 100644 index 00000000..222faaca --- /dev/null +++ b/tests/broadcast/profiling.go @@ -0,0 +1,24 @@ +package broadcast + +import ( + "os" + "runtime/trace" +) + +func StartTrace(tracePath string) (stop func() error, err error) { + traceFile, err := os.Create(tracePath) + if err != nil { + return nil, err + } + if err := trace.Start(traceFile); err != nil { + return nil, err + } + return func() error { + trace.Stop() + err = traceFile.Close() + if err != nil { + return err + } + return nil + }, nil +} diff --git a/tests/broadcast/server.go b/tests/broadcast/server.go new file mode 100644 index 00000000..83b6b409 --- /dev/null +++ b/tests/broadcast/server.go @@ -0,0 +1,309 @@ +package broadcast + +import ( + "context" + "crypto/elliptic" + "errors" + net "net" + "sync" + "time" + + gorums "github.com/relab/gorums" + grpc "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +var leader = "127.0.0.1:5000" + +type response struct { + respChan chan int64 + messageID int64 +} + +type testServer struct { + *Server + mut sync.Mutex + leader string + addr string + peers []string + lis net.Listener + mgr *Manager + respChan map[int64]response + processingTime time.Duration + val int64 + order []string +} + +func newtestServer(addr string, srvAddresses []string, _ int, withOrder ...bool) *testServer { + address, err := net.ResolveTCPAddr("tcp", addr) + if err != nil { + panic(err) + } + var osrv *Server + if len(withOrder) > 0 { + osrv = NewServer(gorums.WithListenAddr(address), gorums.WithOrder(BroadcastServicePrePrepare, BroadcastServicePrepare, BroadcastServiceCommit)) + } else { + osrv = NewServer(gorums.WithListenAddr(address)) + } + srv := testServer{ + Server: osrv, + respChan: make(map[int64]response), + leader: leader, + order: make([]string, 0), + } + RegisterBroadcastServiceServer(srv.Server, &srv) + srv.peers = srvAddresses + srv.addr = addr + if addr != leader { + srv.processingTime = 1 * time.Second + } + srv.mgr = NewManager( + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + view, err := srv.mgr.NewConfiguration(gorums.WithNodeList(srv.peers)) + if err != nil { + panic(err) + } + srv.SetView(view) + return &srv +} + +func newAuthenticatedServer(addr string, srvAddresses []string) *testServer { + address, err := net.ResolveTCPAddr("tcp", addr) + if err != nil { + panic(err) + } + asrv := NewServer(gorums.WithListenAddr(address), gorums.EnforceAuthentication(elliptic.P256())) + srv := testServer{ + Server: asrv, + respChan: make(map[int64]response), + leader: leader, + } + RegisterBroadcastServiceServer(srv.Server, &srv) + srv.peers = srvAddresses + srv.addr = addr + if addr != leader { + srv.processingTime = 100 * time.Millisecond + } + auth := gorums.NewAuth(elliptic.P256()) + _ = auth.GenerateKeys() + privKey, pubKey := auth.Keys() + auth.RegisterKeys(address, privKey, pubKey) + srv.mgr = NewManager( + gorums.WithAuthentication(auth), + gorums.WithGrpcDialOptions( + grpc.WithTransportCredentials(insecure.NewCredentials()), + ), + ) + view, err := srv.mgr.NewConfiguration(gorums.WithNodeList(srv.peers)) + if err != nil { + panic(err) + } + srv.SetView(view) + return &srv +} + +func (srv *testServer) start(lis net.Listener) { + _ = srv.Serve(lis) +} + +func (srv *testServer) Stop() { + srv.Server.Stop() + srv.mgr.Close() +} + +func (srv *testServer) QuorumCall(ctx gorums.ServerCtx, req *Request) (resp *Response, err error) { + srv.mut.Lock() + defer srv.mut.Unlock() + return &Response{Result: req.Value}, nil +} + +func (srv *testServer) QuorumCallWithBroadcast(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + srv.mut.Lock() + defer srv.mut.Unlock() + broadcast.BroadcastIntermediate(req) +} + +func (srv *testServer) QuorumCallWithMulticast(ctx gorums.ServerCtx, req *Request) (resp *Response, err error) { + done := make(chan int64) + srv.mut.Lock() + srv.respChan[req.Value] = response{ + messageID: req.Value, + respChan: done, + } + srv.mut.Unlock() + srv.View.MulticastIntermediate(context.Background(), req, gorums.WithNoSendWaiting()) + ctx.Release() + res := <-done + return &Response{Result: res}, nil +} + +func (srv *testServer) MulticastIntermediate(ctx gorums.ServerCtx, req *Request) { + ctx.Release() + srv.View.Multicast(context.Background(), req, gorums.WithNoSendWaiting()) +} + +func (srv *testServer) Multicast(ctx gorums.ServerCtx, req *Request) { + ctx.Release() + srv.mut.Lock() + defer srv.mut.Unlock() + if response, ok := srv.respChan[req.Value]; ok { + response.respChan <- req.Value + close(response.respChan) + delete(srv.respChan, req.Value) + } +} + +func (srv *testServer) BroadcastCall(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + broadcast.BroadcastIntermediate(req) +} + +func (srv *testServer) BroadcastIntermediate(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + broadcast.Broadcast(req) +} + +func (srv *testServer) Broadcast(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + _ = broadcast.SendToClient(&Response{ + Result: req.Value, + }, nil) +} + +func (srv *testServer) BroadcastCallForward(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + if srv.addr != srv.leader { + _ = broadcast.Forward(req, srv.leader) + return + } + broadcast.Broadcast(req) +} + +func (srv *testServer) BroadcastCallTo(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + broadcast.To(srv.leader).BroadcastToResponse(req) // only broadcast to the leader +} + +func (srv *testServer) BroadcastToResponse(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + }, nil) +} + +func (srv *testServer) Search(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + // make sure the client req reaches all servers first. + // this is because the cancellation only give weak + // guarantees. Meaning, cancellations not yet related + // to a broadcast request (e.g. because the client req has + // not yet arrived) will be dropped. + time.Sleep(1 * time.Millisecond) + select { + case <-ctx.Done(): + _ = broadcast.Cancel() + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 0, + }, nil) + case <-time.After(srv.processingTime): + _ = broadcast.Cancel() + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 1, + }, nil) + } +} + +func (srv *testServer) LongRunningTask(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + srv.mut.Lock() + defer srv.mut.Unlock() + select { + case <-ctx.Done(): + srv.val = 1 + case <-time.After(5 * time.Second): + srv.val = 0 + } + broadcast.Done() +} + +func (srv *testServer) GetVal(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + srv.mut.Lock() + defer srv.mut.Unlock() + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: srv.val, + }, nil) +} + +func (srv *testServer) Order(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + broadcast.PrePrepare(&Request{}) +} + +func (srv *testServer) PrePrepare(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + // this will cause the leader to be late to call broadcast.Prepare(). + // Hence, it will receive Prepare and Commit from the other servers + // before calling Prepare. The order of received msgs will thus be + // wrong and the msgs need to be stored temporarily. + if srv.addr == srv.leader { + time.Sleep(200 * time.Millisecond) + } + srv.mut.Lock() + added := false + for _, m := range srv.order { + if m == "PrePrepare" { + added = true + break + } + } + if !added { + srv.order = append(srv.order, "PrePrepare") + } + srv.mut.Unlock() + broadcast.Prepare(&Request{}) +} + +func (srv *testServer) Prepare(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + srv.mut.Lock() + if len(srv.order) <= 0 { + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 1, + }, errors.New("did not receive PrePrepare before Prepare")) + srv.mut.Unlock() + return + } + added := false + for _, m := range srv.order { + if m == "Prepare" { + added = true + break + } + } + if !added { + srv.order = append(srv.order, "Prepare") + } + srv.mut.Unlock() + broadcast.Commit(&Request{}) +} + +func (srv *testServer) Commit(ctx gorums.ServerCtx, req *Request, broadcast *Broadcast) { + srv.mut.Lock() + if len(srv.order) <= 0 { + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 2, + }, errors.New("did not receive PrePrepare and Prepare before Commit")) + srv.mut.Unlock() + return + } + if len(srv.order) <= 1 { + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 3, + }, errors.New("did not receive Prepare before Commit")) + srv.mut.Unlock() + return + } + srv.mut.Unlock() + _ = broadcast.SendToClient(&Response{ + From: srv.addr, + Result: 0, + }, nil) +} diff --git a/unicast.go b/unicast.go index 21185343..f1edcbe9 100644 --- a/unicast.go +++ b/unicast.go @@ -13,7 +13,7 @@ import ( func (n *RawNode) Unicast(ctx context.Context, d CallData, opts ...CallOption) { o := getCallOptions(E_Unicast, opts) - md := &ordering.Metadata{MessageID: n.mgr.getMsgID(), Method: d.Method} + md := &ordering.Metadata{MessageID: n.mgr.getMsgID(), Method: d.Method, BroadcastMsg: &ordering.BroadcastMsg{BroadcastID: d.BroadcastID}} req := request{ctx: ctx, msg: &Message{Metadata: md, Message: d.Message}, opts: o} if o.noSendWaiting {