diff --git a/cmd/horcrux/cmd/leader_election.go b/cmd/horcrux/cmd/leader_election.go index 8ec1d92e..c5745345 100644 --- a/cmd/horcrux/cmd/leader_election.go +++ b/cmd/horcrux/cmd/leader_election.go @@ -70,16 +70,16 @@ horcrux elect 2 # elect specific leader`, ctx, cancelFunc := context.WithTimeout(context.Background(), 30*time.Second) defer cancelFunc() - grpcClient := proto.NewICosignerGRPCServerClient(conn) + grpcClient := proto.NewIRaftGRPCClient(conn) _, err = grpcClient.TransferLeadership( ctx, - &proto.CosignerGRPCTransferLeadershipRequest{LeaderID: leaderID}, + &proto.RaftGRPCTransferLeadershipRequest{LeaderID: leaderID}, ) if err != nil { return err } - res, err := grpcClient.GetLeader(ctx, &proto.CosignerGRPCGetLeaderRequest{}) + res, err := grpcClient.GetLeader(ctx, &proto.RaftGRPCGetLeaderRequest{}) if err != nil { return err } @@ -167,9 +167,9 @@ func getLeaderCmd() *cobra.Command { ctx, cancelFunc := context.WithTimeout(context.Background(), 30*time.Second) defer cancelFunc() - grpcClient := proto.NewICosignerGRPCServerClient(conn) + grpcClient := proto.NewIRaftGRPCClient(conn) - res, err := grpcClient.GetLeader(ctx, &proto.CosignerGRPCGetLeaderRequest{}) + res, err := grpcClient.GetLeader(ctx, &proto.RaftGRPCGetLeaderRequest{}) if err != nil { return err } diff --git a/cmd/horcrux/cmd/threshold.go b/cmd/horcrux/cmd/threshold.go index bc973fbe..a3129551 100644 --- a/cmd/horcrux/cmd/threshold.go +++ b/cmd/horcrux/cmd/threshold.go @@ -25,7 +25,8 @@ func NewThresholdValidator( thresholdCfg := config.Config.ThresholdModeConfig // NOTE: Shouldnt this be a list of concrete type instead of interface type? - remoteCosigners := make([]node.ICosigner, 0, len(thresholdCfg.Cosigners)-1) + remoteCosigners := make([]pcosigner.IRemoteCosigner, 0, len(thresholdCfg.Cosigners)-1) + remoteIcosigners := make([]pcosigner.ICosigner, 0, len(thresholdCfg.Cosigners)-1) var p2pListen string @@ -42,10 +43,14 @@ func NewThresholdValidator( for _, c := range thresholdCfg.Cosigners { if c.ShardID != security.GetID() { + temp := pcosigner.NewRemoteCosigner(c.ShardID, c.P2PAddr) remoteCosigners = append( remoteCosigners, - pcosigner.NewRemoteCosigner(c.ShardID, c.P2PAddr), + temp, ) + remoteIcosigners = append( + remoteIcosigners, + temp) } else { p2pListen = c.P2PAddr } @@ -74,9 +79,8 @@ func NewThresholdValidator( // RAFT node ID is the cosigner ID nodeID := fmt.Sprint(security.GetID()) - // Start RAFT store listener raftStore := node.NewRaftStore(nodeID, - raftDir, p2pListen, raftTimeout, logger) + raftDir, p2pListen, raftTimeout, logger, localCosigner, remoteIcosigners) if err := raftStore.Start(); err != nil { return nil, nil, fmt.Errorf("error starting raft store: %w", err) } diff --git a/pkg/multiresolver/multi_test.go b/pkg/multiresolver/multi_test.go index 888237fe..10b131bd 100644 --- a/pkg/multiresolver/multi_test.go +++ b/pkg/multiresolver/multi_test.go @@ -9,7 +9,9 @@ import ( "testing" "time" + "github.com/cometbft/cometbft/libs/log" "github.com/strangelove-ventures/horcrux/pkg/node" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner" grpcretry "github.com/grpc-ecosystem/go-grpc-middleware/retry" "github.com/strangelove-ventures/horcrux/pkg/multiresolver" @@ -26,27 +28,32 @@ func createListener(nodeID string, homedir string) (string, func(), error) { } port := strconv.Itoa(sock.Addr().(*net.TCPAddr).Port) + localcosign := pcosigner.NewLocalCosigner( + log.NewNopLogger(), nil, nil, "") + var remoteCosigners []pcosigner.IRemoteCosigner + var timeDuration time.Duration + + shadowRemoteCosign := pcosigner.ToIcosigner(remoteCosigners) s := node.NewRaftStore( nodeID, homedir, "127.0.0.1:"+port, 500*time.Millisecond, - nil) + nil, localcosign, shadowRemoteCosign) // Need to set pointers to avoid nil pointers. - var cosigners []node.ICosigner - var timeDuration time.Duration - thresholdvalidator := node.NewThresholdValidator(nil, nil, 0, timeDuration, 0, nil, cosigners, nil) + + thresholdvalidator := node.NewThresholdValidator(log.NewNopLogger(), nil, 0, timeDuration, 0, localcosign, remoteCosigners, s) s.SetThresholdValidator(thresholdvalidator) - transportManager, err := s.Open() + transportManager, err := s.Open(shadowRemoteCosign) if err != nil { return "", nil, err } grpcServer := grpc.NewServer() - proto.RegisterICosignerGRPCServerServer(grpcServer, node.NewGRPCServer(nil, nil, s)) + proto.RegisterICosignerGRPCServer(grpcServer, node.NewGRPCServer(nil, s)) transportManager.Register(grpcServer) go func() { @@ -97,8 +104,8 @@ func TestMultiResolver(t *testing.T) { ctx, cancelFunc := context.WithTimeout(context.Background(), 30*time.Second) defer cancelFunc() - grpcClient := proto.NewICosignerGRPCServerClient(connDNS) - _, err = grpcClient.GetLeader(ctx, &proto.CosignerGRPCGetLeaderRequest{}) + grpcClient := proto.NewIRaftGRPCClient(connDNS) + _, err = grpcClient.GetLeader(ctx, &proto.RaftGRPCGetLeaderRequest{}) require.NoError(t, err) connIP, err := grpc.Dial(targetIP, @@ -110,7 +117,7 @@ func TestMultiResolver(t *testing.T) { require.NoError(t, err) defer connIP.Close() - grpcClient = proto.NewICosignerGRPCServerClient(connIP) - _, err = grpcClient.GetLeader(ctx, &proto.CosignerGRPCGetLeaderRequest{}) + grpcClient = proto.NewIRaftGRPCClient(connIP) + _, err = grpcClient.GetLeader(ctx, &proto.RaftGRPCGetLeaderRequest{}) require.NoError(t, err) } diff --git a/pkg/node/grpc_server.go b/pkg/node/grpc_server.go index 13db2a01..3046eeed 100644 --- a/pkg/node/grpc_server.go +++ b/pkg/node/grpc_server.go @@ -1,3 +1,4 @@ +// This is responsible for the Cosigners Connections. package node import ( @@ -5,6 +6,7 @@ import ( "fmt" "time" + "github.com/cometbft/cometbft/libs/log" "github.com/strangelove-ventures/horcrux/pkg/pcosigner" "github.com/strangelove-ventures/horcrux/pkg/types" @@ -13,35 +15,54 @@ import ( ) // Enures that GRPCServer implements the proto.CosignerGRPCServer interface. -var _ proto.ICosignerGRPCServerServer = &GRPCServer{} +var _ proto.ICosignerGRPCServer = &GRPCServer{} +var _ proto.IRaftGRPCServer = &GRPCServer{} -type GRPCServer struct { - cosigner *pcosigner.LocalCosigner // The "node's" LocalCosigner - thresholdValidator *ThresholdValidator // The "node's" ThresholdValidator - raftStore *RaftStore // The "node's" RaftStore +// TODO Implement as + +type CosignGRPCServer struct { + cosigner pcosigner.ILocalCosigner + + logger log.Logger + proto.UnimplementedICosignerGRPCServer // Promoted Fields is embedded to have forward compatiblitity - proto.UnimplementedICosignerGRPCServerServer +} + +type RaftGRPCServer struct { + // logger log.Logger + peers []pcosigner.ICosigner + // The "node's" ThresholdValidator + // thresholdValidator *ThresholdValidator + + // The "node's" RaftStore + raftStore *RaftStore + + // Promoted Fields is embedded to have forward compatiblitity + proto.UnimplementedIRaftGRPCServer +} +type GRPCServer struct { + *CosignGRPCServer + // The "node's" LocalCosigner + *RaftGRPCServer } // NewGRPCServer returns a new GRPCServer. func NewGRPCServer( - cosigner *pcosigner.LocalCosigner, - thresholdValidator *ThresholdValidator, + cosigner pcosigner.ILocalCosigner, + // thresholdValidator *ThresholdValidator, raftStore *RaftStore, ) *GRPCServer { return &GRPCServer{ - // TODO: This is a hack to get around the fact that the cosigner is not a? - cosigner: cosigner, - thresholdValidator: thresholdValidator, - raftStore: raftStore, + CosignGRPCServer: &CosignGRPCServer{cosigner: cosigner}, // The nodes local cosigner + RaftGRPCServer: &RaftGRPCServer{raftStore: raftStore}, // The nodes raftStore } } // SignBlock "pseudo-implements" the ICosignerGRPCServer interface in pkg/proto/cosigner_grpc_server_grpc.pb.go -func (rpc *GRPCServer) SignBlock( +func (rpc *RaftGRPCServer) SignBlock( _ context.Context, - req *proto.CosignerGRPCSignBlockRequest, -) (*proto.CosignerGRPCSignBlockResponse, error) { + req *proto.RaftGRPCSignBlockRequest, +) (*proto.RaftGRPCSignBlockResponse, error) { block := &Block{ Height: req.Block.GetHeight(), Round: req.Block.GetRound(), @@ -50,16 +71,53 @@ func (rpc *GRPCServer) SignBlock( Timestamp: time.Unix(0, req.Block.GetTimestamp()), } // this - res, _, err := rpc.thresholdValidator.SignBlock(req.ChainID, block) + res, _, err := rpc.raftStore.thresholdValidator.SignBlock(req.ChainID, block) if err != nil { return nil, err } - return &proto.CosignerGRPCSignBlockResponse{ + return &proto.RaftGRPCSignBlockResponse{ Signature: res, }, nil } -func (rpc *GRPCServer) SetNoncesAndSign( +// TransferLeadership pseudo-implements the ICosignerGRPCServer interface in pkg/proto/cosigner_grpc_server_grpc.pb.go +func (rpc *RaftGRPCServer) TransferLeadership( + _ context.Context, + req *proto.RaftGRPCTransferLeadershipRequest, +) (*proto.RaftGRPCTransferLeadershipResponse, error) { + if rpc.raftStore.raft.State() != raft.Leader { + return &proto.RaftGRPCTransferLeadershipResponse{}, nil + } + leaderID := req.GetLeaderID() + if leaderID != "" { + // TODO: Not an elegant fix. Se other notes. + for _, c := range rpc.peers { + shardID := fmt.Sprint(c.GetID()) + if shardID == leaderID { + raftAddress := p2pURLToRaftAddress(c.GetAddress()) + fmt.Printf("Transferring leadership to ID: %s - Address: %s\n", shardID, raftAddress) + rpc.raftStore.raft.LeadershipTransferToServer(raft.ServerID(shardID), raft.ServerAddress(raftAddress)) + return &proto.RaftGRPCTransferLeadershipResponse{LeaderID: shardID, LeaderAddress: raftAddress}, nil + } + } + } + fmt.Printf("Transferring leadership to next candidate\n") + rpc.raftStore.raft.LeadershipTransfer() + return &proto.RaftGRPCTransferLeadershipResponse{}, nil +} + +// GetLeader pseudo-implements the ICosignerGRPCServer interface in pkg/proto/cosigner_grpc_server_grpc.pb.go +// GetLeader gets the current raft cluster leader and send it as respons. +func (rpc *RaftGRPCServer) GetLeader( + context.Context, + *proto.RaftGRPCGetLeaderRequest, +) (*proto.RaftGRPCGetLeaderResponse, error) { + leader := rpc.raftStore.GetLeader() + return &proto.RaftGRPCGetLeaderResponse{Leader: string(leader)}, nil +} + +// SetNoncesAndSign implements the ICosignerGRPCServer interface. +func (rpc *CosignGRPCServer) SetNoncesAndSign( _ context.Context, req *proto.CosignerGRPCSetNoncesAndSignRequest, ) (*proto.CosignerGRPCSetNoncesAndSignResponse, error) { @@ -71,7 +129,7 @@ func (rpc *GRPCServer) SetNoncesAndSign( SignBytes: req.GetSignBytes(), }) if err != nil { - rpc.raftStore.logger.Error( + rpc.logger.Error( "Failed to sign with shard", "chain_id", req.ChainID, "height", req.Hrst.Height, @@ -81,7 +139,7 @@ func (rpc *GRPCServer) SetNoncesAndSign( ) return nil, err } - rpc.raftStore.logger.Info( + rpc.logger.Info( "Signed with shard", "chain_id", req.ChainID, "height", req.Hrst.Height, @@ -96,7 +154,7 @@ func (rpc *GRPCServer) SetNoncesAndSign( } // GetNonces implements the ICosignerGRPCServer interface. -func (rpc *GRPCServer) GetNonces( +func (rpc *CosignGRPCServer) GetNonces( _ context.Context, req *proto.CosignerGRPCGetNoncesRequest, ) (*proto.CosignerGRPCGetNoncesResponse, error) { @@ -111,38 +169,3 @@ func (rpc *GRPCServer) GetNonces( Nonces: pcosigner.CosignerNonces(res.Nonces).ToProto(), }, nil } - -// TransferLeadership pseudo-implements the ICosignerGRPCServer interface in pkg/proto/cosigner_grpc_server_grpc.pb.go -func (rpc *GRPCServer) TransferLeadership( - _ context.Context, - req *proto.CosignerGRPCTransferLeadershipRequest, -) (*proto.CosignerGRPCTransferLeadershipResponse, error) { - if rpc.raftStore.raft.State() != raft.Leader { - return &proto.CosignerGRPCTransferLeadershipResponse{}, nil - } - leaderID := req.GetLeaderID() - if leaderID != "" { - for _, c := range rpc.raftStore.thresholdValidator.peerCosigners { - shardID := fmt.Sprint(c.GetID()) - if shardID == leaderID { - raftAddress := p2pURLToRaftAddress(c.GetAddress()) - fmt.Printf("Transferring leadership to ID: %s - Address: %s\n", shardID, raftAddress) - rpc.raftStore.raft.LeadershipTransferToServer(raft.ServerID(shardID), raft.ServerAddress(raftAddress)) - return &proto.CosignerGRPCTransferLeadershipResponse{LeaderID: shardID, LeaderAddress: raftAddress}, nil - } - } - } - fmt.Printf("Transferring leadership to next candidate\n") - rpc.raftStore.raft.LeadershipTransfer() - return &proto.CosignerGRPCTransferLeadershipResponse{}, nil -} - -// GetLeader pseudo-implements the ICosignerGRPCServer interface in pkg/proto/cosigner_grpc_server_grpc.pb.go -// GetLeader gets the current raft cluster leader and send it as respons. -func (rpc *GRPCServer) GetLeader( - context.Context, - *proto.CosignerGRPCGetLeaderRequest, -) (*proto.CosignerGRPCGetLeaderResponse, error) { - leader := rpc.raftStore.GetLeader() - return &proto.CosignerGRPCGetLeaderResponse{Leader: string(leader)}, nil -} diff --git a/pkg/node/icosigner.go b/pkg/node/icosigner.go deleted file mode 100644 index 105cfec5..00000000 --- a/pkg/node/icosigner.go +++ /dev/null @@ -1,30 +0,0 @@ -package node - -import ( - cometcrypto "github.com/cometbft/cometbft/crypto" - "github.com/strangelove-ventures/horcrux/pkg/pcosigner" - "github.com/strangelove-ventures/horcrux/pkg/types" -) - -// ICosigner interface is a set of methods for an m-of-n threshold signature. -// This interface abstracts the underlying key storage and management -type ICosigner interface { - // GetID should return the id number of the cosigner - // The ID is the shamir index: 1, 2, etc... - GetID() int - - // GetAddress gets the P2P URL (GRPC and Raft) - GetAddress() string - - // GetPubKey gets the combined public key (permament) - // Not used by Remote Cosigner - GetPubKey(chainID string) (cometcrypto.PubKey, error) - - VerifySignature(chainID string, payload, signature []byte) bool - - // GetNonces requests nonce frpm the peer cosigners - GetNonces(chainID string, hrst types.HRSTKey) (*pcosigner.CosignerNoncesResponse, error) - - // Sign the requested bytes - SetNoncesAndSign(req pcosigner.CosignerSetNoncesAndSignRequest) (*pcosigner.CosignerSignResponse, error) -} diff --git a/pkg/node/leader_mock.go b/pkg/node/leader_mock.go index 40863e1e..78034d77 100644 --- a/pkg/node/leader_mock.go +++ b/pkg/node/leader_mock.go @@ -6,22 +6,23 @@ import ( "sync" "time" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner" "github.com/strangelove-ventures/horcrux/pkg/types" ) var _ ILeader = (*MockLeader)(nil) type MockLeader struct { - id int - - mu sync.Mutex - leader *ThresholdValidator + id int + cosigner pcosigner.ICosigner + mu sync.Mutex + leader *ThresholdValidator } func (m *MockLeader) IsLeader() bool { m.mu.Lock() defer m.mu.Unlock() - return m.leader != nil && m.leader.myCosigner.GetID() == m.id + return m.leader != nil && m.cosigner.GetID() == m.id } func (m *MockLeader) SetLeader(tv *ThresholdValidator) { diff --git a/pkg/node/raft_events.go b/pkg/node/raft_events.go index 43230d6c..cb80cb0f 100644 --- a/pkg/node/raft_events.go +++ b/pkg/node/raft_events.go @@ -1,11 +1,11 @@ package node import ( + "context" "encoding/json" "errors" "time" - "github.com/strangelove-ventures/horcrux/pkg/pcosigner" "github.com/strangelove-ventures/horcrux/pkg/types" "github.com/strangelove-ventures/horcrux/pkg/metrics" @@ -18,6 +18,14 @@ const ( raftEventLSS = "LSS" ) +const ( + rpcTimeout = 4 * time.Second +) + +func GetContext() (context.Context, context.CancelFunc) { + return context.WithTimeout(context.Background(), rpcTimeout) +} + func (f *fsm) getEventHandler(key string) func(string) { return map[string]func(string){ raftEventLSS: f.handleLSSEvent, @@ -48,10 +56,18 @@ func (f *fsm) handleLSSEvent(value string) { return } _ = f.thresholdValidator.SaveLastSignedState(lss.ChainID, lss.SignStateConsensus) - _ = f.thresholdValidator.myCosigner.SaveLastSignedState(lss.ChainID, lss.SignStateConsensus) + if err != nil { + f.logger.Error("[error] - thresholdValidator.SaveLastSignedState %v", err) + // Our ephemeral secret parts are required, cannot proceed + } + _ = f.thresholdValidator.thresholdalgorithm.SaveLastSignedState(lss.ChainID, lss.SignStateConsensus) + if err != nil { + f.logger.Error("[error] - thresholdalgorithm.SaveLastSignedState %v", err) + // Our ephemeral secret parts are required, cannot proceed + } } -func (s *RaftStore) getLeaderGRPCClient() (proto.ICosignerGRPCServerClient, *grpc.ClientConn, error) { +func (s *RaftStore) getLeaderGRPCClient() (proto.IRaftGRPCClient, *grpc.ClientConn, error) { var leader string for i := 0; i < 30; i++ { leader = string(s.GetLeader()) @@ -68,18 +84,20 @@ func (s *RaftStore) getLeaderGRPCClient() (proto.ICosignerGRPCServerClient, *grp if err != nil { return nil, nil, err } - return proto.NewICosignerGRPCServerClient(conn), conn, nil + + return proto.NewIRaftGRPCClient(conn), conn, nil } +// SignBlock implements the ILeader interface func (s *RaftStore) SignBlock(req ValidatorSignBlockRequest) (*ValidatorSignBlockResponse, error) { client, conn, err := s.getLeaderGRPCClient() if err != nil { return nil, err } defer conn.Close() - context, cancelFunc := pcosigner.GetContext() + context, cancelFunc := GetContext() defer cancelFunc() - res, err := client.SignBlock(context, &proto.CosignerGRPCSignBlockRequest{ + res, err := client.SignBlock(context, &proto.RaftGRPCSignBlockRequest{ ChainID: req.ChainID, Block: req.Block.toProto(), }) diff --git a/pkg/node/raft_store.go b/pkg/node/raft_store.go index 527ec0c2..babe5167 100644 --- a/pkg/node/raft_store.go +++ b/pkg/node/raft_store.go @@ -18,6 +18,8 @@ import ( "sync" "time" + "github.com/davecgh/go-spew/spew" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner" "github.com/strangelove-ventures/horcrux/pkg/types" "github.com/Jille/raft-grpc-leader-rpc/leaderhealth" @@ -67,7 +69,7 @@ type RaftStore struct { // NewRaftStore returns a new RaftStore. func NewRaftStore( nodeID string, directory string, bindAddress string, timeout time.Duration, - logger log.Logger) *RaftStore { + logger log.Logger, localCosigner *pcosigner.LocalCosigner, peers []pcosigner.ICosigner) *RaftStore { cosignerRaftStore := &RaftStore{ NodeID: nodeID, RaftDir: directory, @@ -76,7 +78,13 @@ func NewRaftStore( m: make(map[string]string), logger: logger, } + spew.Dump(peers) + spew.Dump(localCosigner) + err := cosignerRaftStore.onStart(localCosigner, peers) + if err != nil { + panic(err) + } cosignerRaftStore.BaseService = *service.NewBaseService(logger, "CosignerRaftStore", cosignerRaftStore) return cosignerRaftStore } @@ -85,7 +93,7 @@ func (s *RaftStore) SetThresholdValidator(thresholdValidator *ThresholdValidator s.thresholdValidator = thresholdValidator } -func (s *RaftStore) init() error { +func (s *RaftStore) init(localCosigner *pcosigner.LocalCosigner, peers []pcosigner.ICosigner) error { host := p2pURLToRaftAddress(s.RaftBind) _, port, err := net.SplitHostPort(host) if err != nil { @@ -96,14 +104,17 @@ func (s *RaftStore) init() error { if err != nil { return err } - transportManager, err := s.Open() + transportManager, err := s.Open(peers) if err != nil { return err } // Create a new gRPC server which is used by both the Raft, the threshold validator and the cosigner grpcServer := grpc.NewServer() - proto.RegisterICosignerGRPCServerServer(grpcServer, - NewGRPCServer(s.thresholdValidator.myCosigner, s.thresholdValidator, s)) + + // proto.RegisterIRaftGRPCServer(grpcServer, s) + proto.RegisterICosignerGRPCServer(grpcServer, + // FIX: this is a TempFIX get cosigner + NewGRPCServer(localCosigner, s)) transportManager.Register(grpcServer) leaderhealth.Setup(s.raft, grpcServer, []string{"Leader"}) raftadmin.Register(grpcServer, s.raft) @@ -113,10 +124,14 @@ func (s *RaftStore) init() error { // OnStart starts the raft server func (s *RaftStore) OnStart() error { + s.logger.Info("Starting RaftStore") + return nil +} +func (s *RaftStore) onStart(localCosign *pcosigner.LocalCosigner, peers []pcosigner.ICosigner) error { go func() { - err := s.init() + err := s.init(localCosign, peers) if err != nil { - fmt.Println("RAFTSTORE") + s.logger.Error("Starting RaftStore", "error", err) panic(err) } }() @@ -135,7 +150,7 @@ func p2pURLToRaftAddress(p2pURL string) string { // Open opens the store. If enableSingle is set, and there are no existing peers, // then this node becomes the first node, and therefore leader, of the cluster. // localID should be the server identifier for this node. -func (s *RaftStore) Open() (*raftgrpctransport.Manager, error) { +func (s *RaftStore) Open(peers []pcosigner.ICosigner) (*raftgrpctransport.Manager, error) { // Setup Raft configuration. config := raft.DefaultConfig() config.LocalID = raft.ServerID(s.NodeID) @@ -182,12 +197,13 @@ func (s *RaftStore) Open() (*raftgrpctransport.Manager, error) { }, }, } - - for _, c := range s.thresholdValidator.peerCosigners { - configuration.Servers = append(configuration.Servers, raft.Server{ - ID: raft.ServerID(fmt.Sprint(c.GetID())), - Address: raft.ServerAddress(p2pURLToRaftAddress(c.GetAddress())), - }) + // This is also a not so nice fix. + for _, c := range peers { + configuration.Servers = append(configuration.Servers, + raft.Server{ + ID: raft.ServerID(fmt.Sprint(c.GetID())), + Address: raft.ServerAddress(p2pURLToRaftAddress(c.GetAddress())), + }) } s.raft.BootstrapCluster(configuration) @@ -212,7 +228,7 @@ func (s *RaftStore) Emit(key string, value interface{}) error { // Set sets the value for the given key. func (s *RaftStore) Set(key, value string) error { if !s.IsLeader() { - return fmt.Errorf("not leader") + return fmt.Errorf("not leader, %v is leader", s.GetLeader()) } c := &command{ diff --git a/pkg/node/raft_store_test.go b/pkg/node/raft_store_test.go index 912dde3d..0756d420 100644 --- a/pkg/node/raft_store_test.go +++ b/pkg/node/raft_store_test.go @@ -1,4 +1,4 @@ -package node +package node_test import ( "crypto/rand" @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/strangelove-ventures/horcrux/pkg/node" "github.com/strangelove-ventures/horcrux/pkg/pcosigner" cometcryptoed25519 "github.com/cometbft/cometbft/crypto/ed25519" @@ -44,19 +45,22 @@ func Test_StoreInMemOpenSingleNode(t *testing.T) { "", ) - validator := &ThresholdValidator{ - myCosigner: cosigner} - s := &RaftStore{ - NodeID: "1", - RaftDir: tmpDir, - RaftBind: "127.0.0.1:0", - RaftTimeout: 1 * time.Second, - m: make(map[string]string), - logger: nil, - thresholdValidator: validator, - } + remoteCosigns := make([]pcosigner.IRemoteCosigner, 0) + remoteCosigns = append(remoteCosigns, pcosigner.NewRemoteCosigner(1, "temp")) + shadowRemoteCosign := pcosigner.ToIcosigner(remoteCosigns) + //spew.Dump(&remoteCosigns) + //spew.Dump(&shadowRemoteCosign) + + //fmt.Printf("remotecosign: %s \n", spew.Dump(&remoteCosigns)) + //fmt.Printf("shadowRemoteCosign: %v \n", spew.Dump(&shadowRemoteCosign)) + + s := node.NewRaftStore("1", tmpDir, "127.0.0.1:0", 1*time.Second, log.NewNopLogger(), cosigner, shadowRemoteCosign) + + validator := node.NewThresholdValidator(log.NewNopLogger(), nil, 0, 1, 1, cosigner, remoteCosigns, s) + + s.SetThresholdValidator(validator) - if _, err := s.Open(); err != nil { + if _, err := s.Open(shadowRemoteCosign); err != nil { t.Fatalf("failed to open store: %s", err) } diff --git a/pkg/node/threshold_validator.go b/pkg/node/threshold_validator.go index cb9054e2..7bbd8867 100644 --- a/pkg/node/threshold_validator.go +++ b/pkg/node/threshold_validator.go @@ -8,6 +8,9 @@ import ( "sync" "time" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner/signer" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner/signer/shamirCosign" + "github.com/strangelove-ventures/horcrux/pkg/pcosigner" "github.com/strangelove-ventures/horcrux/pkg/types" @@ -20,6 +23,7 @@ import ( "github.com/strangelove-ventures/horcrux/pkg/proto" ) +// Makes sure ThresholdValitator implements IPrivValidator, a wrapper around var _ IPrivValidator = &ThresholdValidator{} type ValidatorSignBlockRequest struct { @@ -37,15 +41,10 @@ type ThresholdValidator struct { grpcTimeout time.Duration - chainState sync.Map - - // our own cosigner - myCosigner *pcosigner.LocalCosigner - - // peer cosigners - peerCosigners []ICosigner + chainState sync.Map + thresholdalgorithm signer.ISigner // This is our threshold implementation. It should be injected - leader ILeader + leader ILeader // The Raft leader logger log.Logger @@ -72,19 +71,22 @@ func NewThresholdValidator( threshold int, grpcTimeout time.Duration, maxWaitForSameBlockAttempts int, - myCosigner *pcosigner.LocalCosigner, - peerCosigners []ICosigner, + myCosigner pcosigner.ILocalCosigner, + peerCosigners []pcosigner.IRemoteCosigner, leader ILeader, ) *ThresholdValidator { + return &ThresholdValidator{ logger: logger, config: config, threshold: threshold, grpcTimeout: grpcTimeout, maxWaitForSameBlockAttempts: maxWaitForSameBlockAttempts, - myCosigner: myCosigner, - peerCosigners: peerCosigners, - leader: leader, + thresholdalgorithm: shamirCosign.NewShamirCosign( + logger, + myCosigner, + peerCosigners), + leader: leader, } } @@ -228,28 +230,21 @@ func (pv *ThresholdValidator) notifyBlockSignError(chainID string, hrs types.HRS css.lastSignState.CondBroadcast() } -// waitForSignStatesToFlushToDisk waits for any sign states to finish writing to disk. -func (pv *ThresholdValidator) waitForSignStatesToFlushToDisk() { - pv.pendingDiskWG.Wait() - - pv.myCosigner.WaitForSignStatesToFlushToDisk() -} - // Stop safely shuts down the ThresholdValidator. // Stop implements IPrivValidator from threshold_remote_signer.go func (pv *ThresholdValidator) Stop() { - pv.waitForSignStatesToFlushToDisk() + pv.thresholdalgorithm.WaitForSignStatesToFlushToDisk() } // GetPubKey returns the public key of the validator. // Implements IPrivValidator fromt threshold_remote_signer.go func (pv *ThresholdValidator) GetPubKey(chainID string) (crypto.PubKey, error) { - return pv.myCosigner.GetPubKey(chainID) + return pv.thresholdalgorithm.GetPubKey(chainID) } // SignVote signs a canonical representation of the vote, along with the // chainID. -// SignVote implements IPrivValidator fromt threshold_remote_signer.go +// SignVote implements IPrivValidator from threshold_remote_signer.go func (pv *ThresholdValidator) SignVote(chainID string, vote *cometproto.Vote) error { block := &Block{ Height: vote.Height, @@ -367,117 +362,6 @@ func newSameBlockError(chainID string, hrs types.HRSKey) *SameBlockError { } } -func (pv *ThresholdValidator) waitForPeerNonces( - chainID string, - peer ICosigner, - hrst types.HRSTKey, - wg *sync.WaitGroup, - nonces map[ICosigner][]pcosigner.CosignerNonce, - thresholdPeersMutex *sync.Mutex, -) { - peerStartTime := time.Now() - peerNonces, err := peer.GetNonces(chainID, hrst) - if err != nil { - // Significant missing shares may lead to signature failure - metrics.MissedNonces.WithLabelValues(peer.GetAddress()).Add(float64(1)) - metrics.TotalMissedNonces.WithLabelValues(peer.GetAddress()).Inc() - pv.logger.Error("Error getting nonces", "cosigner", peer.GetID(), "err", err) - return - } - // Significant missing shares may lead to signature failure - metrics.MissedNonces.WithLabelValues(peer.GetAddress()).Set(0) - metrics.TimedCosignerNonceLag.WithLabelValues(peer.GetAddress()).Observe(time.Since(peerStartTime).Seconds()) - - // Check so that wg.Done is not called more than (threshold - 1) times which causes hardlock - thresholdPeersMutex.Lock() - if len(nonces) < pv.threshold-1 { - nonces[peer] = peerNonces.Nonces - defer wg.Done() - } - thresholdPeersMutex.Unlock() -} -func (pv *ThresholdValidator) waitForPeerSetNoncesAndSign( - chainID string, - peer ICosigner, - hrst types.HRSTKey, - noncesMap map[ICosigner][]pcosigner.CosignerNonce, - signBytes []byte, - shareSignatures *[][]byte, - shareSignaturesMutex *sync.Mutex, - wg *sync.WaitGroup, -) { - peerStartTime := time.Now() - defer wg.Done() - peerNonces := make([]pcosigner.CosignerNonce, 0, pv.threshold-1) - - peerID := peer.GetID() - - for _, nonces := range noncesMap { - for _, nonce := range nonces { - // if share is intended for peer, check to make sure source peer is included in threshold - if nonce.DestinationID != peerID { - continue - } - for thresholdPeer := range noncesMap { - if thresholdPeer.GetID() != nonce.SourceID { - continue - } - // source peer is included in threshold signature, include in sharing - peerNonces = append(peerNonces, nonce) - break - } - break - } - } - - sigRes, err := peer.SetNoncesAndSign(pcosigner.CosignerSetNoncesAndSignRequest{ - ChainID: chainID, - Nonces: peerNonces, - HRST: hrst, - SignBytes: signBytes, - }) - - if err != nil { - pv.logger.Error( - "Cosigner failed to set nonces and sign", - "id", peerID, - "err", err.Error(), - ) - return - } - - metrics.TimedCosignerSignLag.WithLabelValues(peer.GetAddress()).Observe(time.Since(peerStartTime).Seconds()) - pv.logger.Debug( - "Received signature part", - "cosigner", peerID, - "chain_id", chainID, - "height", hrst.Height, - "round", hrst.Round, - "step", hrst.Step, - ) - - shareSignaturesMutex.Lock() - defer shareSignaturesMutex.Unlock() - - peerIdx := peerID - 1 - (*shareSignatures)[peerIdx] = make([]byte, len(sigRes.Signature)) - copy((*shareSignatures)[peerIdx], sigRes.Signature) -} - -func waitUntilCompleteOrTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { - c := make(chan struct{}) - go func() { - defer close(c) - wg.Wait() - }() - select { - case <-c: - return false // completed normally - case <-time.After(timeout): - return true // timed out - } -} - func (pv *ThresholdValidator) LoadSignStateIfNecessary(chainID string) error { if _, ok := pv.chainState.Load(chainID); ok { return nil @@ -499,7 +383,7 @@ func (pv *ThresholdValidator) LoadSignStateIfNecessary(chainID string) error { lastSignStateInitiatedMutex: &sync.Mutex{}, }) - return pv.myCosigner.LoadSignStateIfNecessary(chainID) + return pv.LoadSignStateIfNecessary(chainID) } // getExistingBlockSignature returns the existing block signature and no error if the signature is valid for the block. @@ -578,7 +462,9 @@ func (pv *ThresholdValidator) compareBlockSignatureAgainstHRS( // SignBlock is the arranger of the whole signature/signing process. func (pv *ThresholdValidator) SignBlock(chainID string, block *Block) ([]byte, time.Time, error) { + // Placeholders height, round, step, stamp, signBytes := block.Height, block.Round, block.Step, block.Timestamp, block.SignBytes + threshold := pv.threshold if err := pv.LoadSignStateIfNecessary(chainID); err != nil { return nil, stamp, err @@ -633,6 +519,7 @@ func (pv *ThresholdValidator) SignBlock(chainID string, block *Block) ([]byte, t // Keep track of the last block that we began the signing process for. Only allow one attempt per block existingSignature, existingTimestamp, err := pv.SaveLastSignedStateInitiated(chainID, block) if err != nil { + pv.logger.Error("Error SaveLastSignedStateInitiated", "chainID", chainID, "block", block, "err", err) return nil, stamp, err } if existingSignature != nil { @@ -640,114 +527,11 @@ func (pv *ThresholdValidator) SignBlock(chainID string, block *Block) ([]byte, t return existingSignature, existingTimestamp, nil } - numPeers := len(pv.peerCosigners) - total := uint8(numPeers + 1) - getEphemeralWaitGroup := sync.WaitGroup{} - - // Only wait until we have enough threshold signatures - getEphemeralWaitGroup.Add(pv.threshold - 1) - // Used to track how close we are to threshold - - // Here the actual signing process starts from a cryptological perspective - // TODO: This process should be factored out. It is not the responsibility of the validator to know - // how to arrange signature of a block. It should be a separate component that is injected into the validator. - nonces := make(map[ICosigner][]pcosigner.CosignerNonce) - thresholdPeersMutex := sync.Mutex{} - - // From each cosigner peer we are requesting the nonce. - // This is done asynchronously. - // pv.waitForPeersNonces uses GRPC to get the nonce from the peer. - for _, c := range pv.peerCosigners { - go pv.waitForPeerNonces(chainID, c, hrst, &getEphemeralWaitGroup, - nonces, &thresholdPeersMutex) - } - - // Requesting a nonce from our own cosigner (a.k.a. the local cosigner) - myNonces, err := pv.myCosigner.GetNonces(chainID, hrst) - if err != nil { - pv.notifyBlockSignError(chainID, block.HRSKey()) - // Our ephemeral secret parts are required, cannot proceed - return nil, stamp, err - } - - // Wait for cosigners to be complete - // A Cosigner will either respond in time, or be cancelled with timeout - if waitUntilCompleteOrTimeout(&getEphemeralWaitGroup, pv.grpcTimeout) { - pv.notifyBlockSignError(chainID, block.HRSKey()) - return nil, stamp, errors.New("timed out waiting for ephemeral shares") - } - - thresholdPeersMutex.Lock() - nonces[pv.myCosigner] = myNonces.Nonces - thresholdPeersMutex.Unlock() - - metrics.TimedSignBlockThresholdLag.Observe(time.Since(timeStartSignBlock).Seconds()) - pv.logger.Debug( - "Have threshold peers", - "chain_id", chainID, - "height", hrst.Height, - "round", hrst.Round, - "step", hrst.Step, - ) - - setEphemeralAndSignWaitGroup := sync.WaitGroup{} - - // Only wait until we have threshold sigs - setEphemeralAndSignWaitGroup.Add(pv.threshold) - - // destination for share signatures - shareSignatures := make([][]byte, total) - - // share sigs is updated by goroutines - shareSignaturesMutex := sync.Mutex{} - - for cosigner := range nonces { - // set peerNonces and sign in single rpc call. - go pv.waitForPeerSetNoncesAndSign(chainID, cosigner, hrst, nonces, - signBytes, &shareSignatures, &shareSignaturesMutex, &setEphemeralAndSignWaitGroup) - } + signature, verified, err := pv.thresholdalgorithm.SignAndVerify( + chainID, threshold, hrst, pv.grpcTimeout, stamp, timeStartSignBlock, signBytes) - // Wait for threshold cosigners to be complete - // A Cosigner will either respond in time, or be cancelled with timeout - if waitUntilCompleteOrTimeout(&setEphemeralAndSignWaitGroup, 4*time.Second) { - pv.notifyBlockSignError(chainID, block.HRSKey()) - return nil, stamp, errors.New("timed out waiting for peers to sign") - } - - metrics.TimedSignBlockCosignerLag.Observe(time.Since(timeStartSignBlock).Seconds()) - pv.logger.Debug( - "Done waiting for cosigners, assembling signatures", - "chain_id", chainID, - "height", hrst.Height, - "round", hrst.Round, - "step", hrst.Step, - ) - - // collect all valid responses into array of partial signatures - shareSigs := make([]pcosigner.PartialSignature, 0, pv.threshold) - for idx, shareSig := range shareSignatures { - if len(shareSig) == 0 { - continue - } - - // we are ok to use the share signatures - complete boolean - // prevents future concurrent access - shareSigs = append(shareSigs, pcosigner.PartialSignature{ - ID: idx + 1, - Signature: shareSig, - }) - } - - if len(shareSigs) < pv.threshold { - metrics.TotalInsufficientCosigners.Inc() - pv.notifyBlockSignError(chainID, block.HRSKey()) - return nil, stamp, errors.New("not enough co-signers") - } - - // assemble the partial signatures into a valid signature - signature, err := pv.myCosigner.CombineSignatures(chainID, shareSigs) if err != nil { - pv.notifyBlockSignError(chainID, block.HRSKey()) + pv.logger.Error("Error pv.thresholdalgorithm.SignAndVerify", "chainID", chainID, "err", err) return nil, stamp, err } @@ -760,7 +544,8 @@ func (pv *ThresholdValidator) SignBlock(chainID string, block *Block) ([]byte, t ) // verify the combined signature before saving to watermark - if !pv.myCosigner.VerifySignature(chainID, signBytes, signature) { + // if !pv.myCosigner.VerifySignature(chainID, signBytes, signature) + if !verified { metrics.TotalInvalidSignature.Inc() pv.notifyBlockSignError(chainID, block.HRSKey()) return nil, stamp, errors.New("combined signature is not valid") diff --git a/pkg/node/threshold_validator_test.go b/pkg/node/threshold_validator_test.go index 8032b71b..bd316a74 100644 --- a/pkg/node/threshold_validator_test.go +++ b/pkg/node/threshold_validator_test.go @@ -73,7 +73,7 @@ func loadKeyForLocalCosigner( func testThresholdValidator(t *testing.T, threshold, total uint8) { cosigners, pubKey := getTestLocalCosigners(t, threshold, total) - thresholdCosigners := make([]ICosigner, 0, threshold-1) + thresholdCosigners := make([]pcosigner.IRemoteCosigner, 0, threshold-1) for i, cosigner := range cosigners { require.Equal(t, i+1, cosigner.GetID()) @@ -83,7 +83,7 @@ func testThresholdValidator(t *testing.T, threshold, total uint8) { } } - leader := &MockLeader{id: 1} + leader := &MockLeader{id: 1, cosigner: cosigners[0]} validator := NewThresholdValidator( cometlog.NewTMLogger(cometlog.NewSyncWriter(os.Stdout)).With("module", "validator"), @@ -100,6 +100,9 @@ func testThresholdValidator(t *testing.T, threshold, total uint8) { leader.leader = validator err := validator.LoadSignStateIfNecessary(testChainID) + if err != nil { + t.Logf("[error] %v", err) + } require.NoError(t, err) proposal := cometproto.Proposal{ @@ -111,8 +114,10 @@ func testThresholdValidator(t *testing.T, threshold, total uint8) { signBytes := comet.ProposalSignBytes(testChainID, &proposal) err = validator.SignProposal(testChainID, &proposal) + if err != nil { + t.Logf("SignProposal [error] %v", err) + } require.NoError(t, err) - require.True(t, pubKey.VerifySignature(signBytes, proposal.Signature)) firstSignature := proposal.Signature @@ -341,13 +346,13 @@ func testThresholdValidatorLeaderElection(t *testing.T, threshold, total uint8) var leader *ThresholdValidator leaders := make([]*MockLeader, total) for i, cosigner := range cosigners { - peers := make([]ICosigner, 0, len(cosigners)-1) + peers := make([]pcosigner.IRemoteCosigner, 0, len(cosigners)-1) for j, otherCosigner := range cosigners { if i != j { peers = append(peers, otherCosigner) } } - leaders[i] = &MockLeader{id: cosigner.GetID(), leader: leader} + leaders[i] = &MockLeader{id: cosigner.GetID(), leader: leader, cosigner: cosigner} tv := NewThresholdValidator( cometlog.NewTMLogger(cometlog.NewSyncWriter(os.Stdout)).With("module", "validator"), cosigner.Config, @@ -375,18 +380,17 @@ func testThresholdValidatorLeaderElection(t *testing.T, threshold, total uint8) for _, l := range leaders { l.SetLeader(nil) } - t.Log("No leader") + t.Log("No leader, will set a leader") rnd, err := rand.Int(rand.Reader, big.NewInt(50)) require.NoError(t, err) // time without a leader time.Sleep(time.Duration(int(rnd.Int64())+100) * time.Millisecond) - newLeader := thresholdValidators[i%len(thresholdValidators)] for _, l := range leaders { l.SetLeader(newLeader) + t.Logf("New leader: %d", l.cosigner.GetID()) } - t.Logf("New leader: %d", newLeader.myCosigner.GetID()) // time with new leader rnd, err = rand.Int(rand.Reader, big.NewInt(50)) diff --git a/pkg/pcosigner/cosigner_signer.go b/pkg/pcosigner/cosigner_signer.go index b4730833..f83e8ea1 100644 --- a/pkg/pcosigner/cosigner_signer.go +++ b/pkg/pcosigner/cosigner_signer.go @@ -1,6 +1,7 @@ package pcosigner // IThresholdSigner is interface for the cosigner_signer whether it's a soft sign or HSM +// Mainly for FUTURE use if and when we want to support HSM type IThresholdSigner interface { // GetPubKey returns the persistent public key GetPubKey() []byte @@ -13,6 +14,7 @@ type IThresholdSigner interface { Sign(nonces []Nonce, payload []byte) ([]byte, error) // CombineSignatures combines multiple partial signatures to a full signature. + // FIX: not necessary this need to be in a Thresholdsigner? CombineSignatures([]PartialSignature) ([]byte, error) } diff --git a/pkg/pcosigner/icosigner.go b/pkg/pcosigner/icosigner.go new file mode 100644 index 00000000..095ff7f2 --- /dev/null +++ b/pkg/pcosigner/icosigner.go @@ -0,0 +1,46 @@ +package pcosigner + +// TODO: Move back to Cosigner Package +import ( + cometcrypto "github.com/cometbft/cometbft/crypto" + "github.com/strangelove-ventures/horcrux/pkg/types" +) + +// An interface for a Cosigners, both local and remote +type ICosigner interface { + // GetID should return the id number of the cosigner + // The ID is the shamir index: 1, 2, etc... + GetID() int + // GetAddress gets the P2P URL (GRPC and Raft) + GetAddress() string +} + +type IRemoteCosigner interface { + ICosigner + // Sign the requested bytes + SetNoncesAndSign(req CosignerSetNoncesAndSignRequest) (*CosignerSignResponse, error) + // GetNonces requests nonce frpm the peer cosigners + GetNonces(chainID string, hrst types.HRSTKey) (*CosignerNoncesResponse, error) +} + +// ILocalCosigner interface is a set of methods for an m-of-n threshold signature. +// This interface abstracts the underlying key storage and management +type ILocalCosigner interface { + IRemoteCosigner + VerifySignature(chainID string, payload, signature []byte) bool + // GetPubKey gets the combined public key (permament) + // Not used by Remote Cosigner and should it be really used by the local cosigner? + GetPubKey(chainID string) (cometcrypto.PubKey, error) + WaitForSignStatesToFlushToDisk() + LoadSignStateIfNecessary(id string) error + SaveLastSignedState(id string, consensus types.SignStateConsensus) error + CombineSignatures(id string, sigs []PartialSignature) ([]byte, error) +} + +func ToIcosigner(iface []IRemoteCosigner) []ICosigner { + icosign := make([]ICosigner, 0) + for _, cosigner := range iface { + icosign = append(icosign, ICosigner(cosigner)) + } + return icosign +} diff --git a/pkg/pcosigner/remote_cosigner.go b/pkg/pcosigner/remote_cosigner.go index 41b31159..0d3e8c6f 100644 --- a/pkg/pcosigner/remote_cosigner.go +++ b/pkg/pcosigner/remote_cosigner.go @@ -1,6 +1,6 @@ package pcosigner -// RemoteCosigner is a Cosigner implementation that uses gRPC make to request from other Cosigners +// RemoteCosigner is a Cosigner implementation that uses gRPC make to request to other Cosigners import ( "context" "fmt" @@ -68,7 +68,7 @@ func (cosigner *RemoteCosigner) VerifySignature(_ string, _, _ []byte) bool { return false } -func (cosigner *RemoteCosigner) getGRPCClient() (proto.ICosignerGRPCServerClient, *grpc.ClientConn, error) { +func (cosigner *RemoteCosigner) getGRPCClient() (proto.ICosignerGRPCClient, *grpc.ClientConn, error) { var grpcAddress string url, err := url.Parse(cosigner.address) if err != nil { @@ -80,7 +80,7 @@ func (cosigner *RemoteCosigner) getGRPCClient() (proto.ICosignerGRPCServerClient if err != nil { return nil, nil, err } - return proto.NewICosignerGRPCServerClient(conn), conn, nil + return proto.NewICosignerGRPCClient(conn), conn, nil } // GetNonces implements the Cosigner interface diff --git a/pkg/pcosigner/signer/iSigner.go b/pkg/pcosigner/signer/iSigner.go new file mode 100644 index 00000000..cda82143 --- /dev/null +++ b/pkg/pcosigner/signer/iSigner.go @@ -0,0 +1,33 @@ +package signer + +import ( + "time" + + "github.com/cometbft/cometbft/crypto" + "github.com/strangelove-ventures/horcrux/pkg/types" +) + +type ISigner interface { + // SignAndVerify is responsible for signing the block and verifying the signature + // It absracts the threshold signature verification + SignAndVerify(chainID string, threshold int, hrst types.HRSTKey, grpcTimeout time.Duration, stamp time.Time, timeStartSignBlock time.Time, signBytes []byte) ([]byte, bool, error) + + GetPubKey(chainID string) (crypto.PubKey, error) + + WaitForSignStatesToFlushToDisk() + + // SaveLastSignedState saves the last sign state in the "high watermark" file at the cosigner lever. + SaveLastSignedState(chainID string, signStateConsensus types.SignStateConsensus) error + /* + + // TODO: This should be the job of ThresholdValidator + LoadSignStateIfNecessary(chainID string) error + + // FIX: Below should not be the responsibility of the ISigner + // Its a temporary hack to get the peers and localCosign + GetPeers() []pcosigner.ICosigner // Returns the remote peers (for use in GRPC) + GetLocalCosign() pcosigner.ILocalCosigner // + */ + // sign(chainID string, threshold int, hrst types.HRSTKey, grpcTimeout time.Duration, stamp time.Time, timeStartSignBlock time.Time, signBytes []byte) ([]byte, time.Time, error) + // verify(chainID string, signBytes []byte, signature []byte) (bool, error) +} diff --git a/pkg/pcosigner/signer/shamirCosign/shamirCosign.go b/pkg/pcosigner/signer/shamirCosign/shamirCosign.go new file mode 100644 index 00000000..805a3aa7 --- /dev/null +++ b/pkg/pcosigner/signer/shamirCosign/shamirCosign.go @@ -0,0 +1,302 @@ +package shamirCosign + +import ( + "errors" + logg "log" + "runtime" + "sync" + "time" + + "github.com/strangelove-ventures/horcrux/pkg/pcosigner" + + "github.com/strangelove-ventures/horcrux/pkg/types" + + "github.com/cometbft/cometbft/crypto" + "github.com/cometbft/cometbft/libs/log" + "github.com/strangelove-ventures/horcrux/pkg/metrics" +) + +func waitUntilCompleteOrTimeout(wg *sync.WaitGroup, timeout time.Duration) bool { + c := make(chan struct{}) + go func() { + defer close(c) + wg.Wait() + }() + select { + case <-c: + return false // completed normally + case <-time.After(timeout): + return true // timed out + } +} + +type ShamirCosign struct { + LocalCosign pcosigner.ILocalCosigner // The "Cosigner's" LocalCosigner + PeerCosigners []pcosigner.IRemoteCosigner // The "Cosigner's" RemoteCosigners + logger log.Logger + pendingDiskWG sync.WaitGroup +} + +func NewShamirCosign(logger log.Logger, myCosigner pcosigner.ILocalCosigner, peerCosigners []pcosigner.IRemoteCosigner) *ShamirCosign { + return &ShamirCosign{ + logger: logger, + LocalCosign: myCosigner, + PeerCosigners: peerCosigners, + } +} + +// waitForSignStatesToFlushToDisk waits for any sign states to finish writing to disk. +func (s *ShamirCosign) WaitForSignStatesToFlushToDisk() { + s.pendingDiskWG.Wait() + + s.LocalCosign.WaitForSignStatesToFlushToDisk() +} + +func (s *ShamirCosign) waitForPeerSetNoncesAndSign( + chainID string, + peer pcosigner.IRemoteCosigner, + hrst types.HRSTKey, + noncesMap map[pcosigner.IRemoteCosigner][]pcosigner.CosignerNonce, + signBytes []byte, + shareSignatures *[][]byte, + shareSignaturesMutex *sync.Mutex, + wg *sync.WaitGroup, + threshold int, +) { + peerStartTime := time.Now() + defer wg.Done() + peerNonces := make([]pcosigner.CosignerNonce, 0, threshold-1) + + peerID := peer.GetID() + + for _, nonces := range noncesMap { + for _, nonce := range nonces { + // if share is intended for peer, check to make sure source peer is included in threshold + if nonce.DestinationID != peerID { + continue + } + for thresholdPeer := range noncesMap { + if thresholdPeer.GetID() != nonce.SourceID { + continue + } + // source peer is included in threshold signature, include in sharing + peerNonces = append(peerNonces, nonce) + break + } + break + } + } + + sigRes, err := peer.SetNoncesAndSign(pcosigner.CosignerSetNoncesAndSignRequest{ + ChainID: chainID, + Nonces: peerNonces, + HRST: hrst, + SignBytes: signBytes, + }) + + if err != nil { + s.logger.Error( + "Cosigner failed to set nonces and sign", + "id", peerID, + "err", err.Error(), + ) + return + } + + metrics.TimedCosignerSignLag.WithLabelValues(peer.GetAddress()).Observe(time.Since(peerStartTime).Seconds()) + s.logger.Debug( + "ShamirCosign received signature part", + "cosigner", peerID, + "chain_id", chainID, + "height", hrst.Height, + "round", hrst.Round, + "step", hrst.Step, + ) + shareSignaturesMutex.Lock() + defer shareSignaturesMutex.Unlock() + + peerIdx := peerID - 1 + (*shareSignatures)[peerIdx] = make([]byte, len(sigRes.Signature)) + copy((*shareSignatures)[peerIdx], sigRes.Signature) +} +func (s *ShamirCosign) GetLocalCosign() pcosigner.ILocalCosigner { + return s.LocalCosign +} +func (s *ShamirCosign) GetPeers() []pcosigner.IRemoteCosigner { + return s.PeerCosigners +} +func (s *ShamirCosign) LoadSignStateIfNecessary(chainID string) error { + return s.LocalCosign.LoadSignStateIfNecessary(chainID) +} +func (s *ShamirCosign) SaveLastSignedState(chainID string, signStateConsensus types.SignStateConsensus) error { + return s.LocalCosign.SaveLastSignedState(chainID, signStateConsensus) +} +func (s *ShamirCosign) SignAndVerify(chainID string, threshold int, hrst types.HRSTKey, grpcTimeout time.Duration, stamp time.Time, timeStartSignBlock time.Time, signBytes []byte) ([]byte, bool, error) { + signature, _, err := s.sign(chainID, threshold, hrst, grpcTimeout, stamp, timeStartSignBlock, signBytes) + if err != nil { + _, filename, line, _ := runtime.Caller(1) + logg.Printf("sign [error] %s:%d %v", filename, line, err) + return nil, false, err + } + verified, err := s.verify(chainID, signBytes, signature) + if err != nil { + return signature, false, err + } + return signature, verified, err +} + +func (s *ShamirCosign) sign( + chainID string, threshold int, hrst types.HRSTKey, grpcTimeout time.Duration, stamp time.Time, timeStartSignBlock time.Time, signBytes []byte) ([]byte, time.Time, error) { + numPeers := len(s.PeerCosigners) + total := uint8(numPeers + 1) + getEphemeralWaitGroup := sync.WaitGroup{} + + // Only wait until we have enough threshold signatures + getEphemeralWaitGroup.Add(threshold - 1) + // Used to track how close we are to threshold + + // Here the actual signing process starts from a cryptological perspective + // TODO: This process should be factored out. It is not the responsibility of the validator to know + // how to arrange signature of a block. It should be a separate component that is injected into the validator. + nonces := make(map[pcosigner.IRemoteCosigner][]pcosigner.CosignerNonce) + thresholdPeersMutex := sync.Mutex{} + + // From each cosigner peer we are requesting the nonce. + // This is done asynchronously. + // pv.waitForPeersNonces uses GRPC to get the nonce from the peer. + + for _, c := range s.PeerCosigners { + // spew.Dump(c) + go s.waitForPeerNonces(chainID, c, hrst, &getEphemeralWaitGroup, + nonces, &thresholdPeersMutex, threshold) + } + + // Requesting a nonce from our own cosigner (a.k.a. the local cosigner) + myNonces, err := s.LocalCosign.GetNonces(chainID, hrst) + if err != nil { + s.logger.Error("Error getting nonces", "chainID", chainID, "err", err) + // TODO: pv.notifyBlockSignError(chainID, block.HRSKey()) + // Our ephemeral secret parts are required, cannot proceed + return nil, stamp, err + } + + // Wait for cosigners to be complete + // A Cosigner will either respond in time, or be cancelled with timeout + + if waitUntilCompleteOrTimeout(&getEphemeralWaitGroup, grpcTimeout) { + s.logger.Error("Error waitUntilCompleteOrTimeout", "chainID", chainID, "err", err) + // TODO: pv.notifyBlockSignError(chainID, block.HRSKey()) + return nil, stamp, errors.New("timed out waiting for ephemeral shares") + } + + thresholdPeersMutex.Lock() + nonces[s.LocalCosign] = myNonces.Nonces + thresholdPeersMutex.Unlock() + + metrics.TimedSignBlockThresholdLag.Observe(time.Since(timeStartSignBlock).Seconds()) + s.logger.Debug( + "Have threshold peers", + "chain_id", chainID, + "height", hrst.Height, + "round", hrst.Round, + "step", hrst.Step, + ) + setEphemeralAndSignWaitGroup := sync.WaitGroup{} + + // Only wait until we have threshold sigs + setEphemeralAndSignWaitGroup.Add(threshold) + + // destination for share signatures + shareSignatures := make([][]byte, total) + + // share sigs is updated by goroutines + shareSignaturesMutex := sync.Mutex{} + + for cosigner := range nonces { + // set peerNonces and sign in single rpc call in parallel using goroutines + // go + s.waitForPeerSetNoncesAndSign(chainID, cosigner, hrst, nonces, + signBytes, &shareSignatures, &shareSignaturesMutex, &setEphemeralAndSignWaitGroup, threshold) + } + + // Wait for threshold cosigners to be complete + // A Cosigner will either respond in time, or be cancelled with timeout + if waitUntilCompleteOrTimeout(&setEphemeralAndSignWaitGroup, 4*time.Second) { + // pv.notifyBlockSignError(chainID, block.HRSKey()) + return nil, stamp, errors.New("timed out waiting for peers to sign") + } + + metrics.TimedSignBlockCosignerLag.Observe(time.Since(timeStartSignBlock).Seconds()) + s.logger.Debug( + "Done waiting for cosigners, assembling signatures", + "chain_id", chainID, + "height", hrst.Height, + "round", hrst.Round, + "step", hrst.Step, + ) + // collect all valid responses into array of partial signatures + shareSigs := make([]pcosigner.PartialSignature, 0, threshold) + for idx, shareSig := range shareSignatures { + if len(shareSig) == 0 { + continue + } + + // we are ok to use the share signatures - complete boolean + // prevents future concurrent access + shareSigs = append(shareSigs, pcosigner.PartialSignature{ + ID: idx + 1, + Signature: shareSig, + }) + } + if len(shareSigs) < threshold { + metrics.TotalInsufficientCosigners.Inc() + // pv.notifyBlockSignError(chainID, block.HRSKey()) + return nil, stamp, errors.New("not enough co-signers") + } + // assemble the partial signatures into a valid signature + signature, err := s.LocalCosign.CombineSignatures(chainID, shareSigs) + if err != nil { + // pv.notifyBlockSignError(chainID, block.HRSKey()) + return nil, stamp, err + } + return signature, stamp, nil +} +func (s *ShamirCosign) verify(chainID string, signBytes []byte, signature []byte) (bool, error) { + verified := s.LocalCosign.VerifySignature(chainID, signBytes, signature) + return verified, nil +} + +func (s *ShamirCosign) GetPubKey(chainID string) (crypto.PubKey, error) { + return s.LocalCosign.GetPubKey(chainID) +} + +func (s *ShamirCosign) waitForPeerNonces( + chainID string, + peer pcosigner.IRemoteCosigner, + hrst types.HRSTKey, + wg *sync.WaitGroup, + nonces map[pcosigner.IRemoteCosigner][]pcosigner.CosignerNonce, + thresholdPeersMutex *sync.Mutex, + threshold int, +) { + peerStartTime := time.Now() + peerNonces, err := peer.GetNonces(chainID, hrst) + if err != nil { + // Significant missing shares may lead to signature failure + metrics.MissedNonces.WithLabelValues(peer.GetAddress()).Add(float64(1)) + metrics.TotalMissedNonces.WithLabelValues(peer.GetAddress()).Inc() + s.logger.Error("Error getting nonces", "cosigner", peer.GetID(), "err", err) + return + } + // Significant missing shares may lead to signature failure + metrics.MissedNonces.WithLabelValues(peer.GetAddress()).Set(0) + metrics.TimedCosignerNonceLag.WithLabelValues(peer.GetAddress()).Observe(time.Since(peerStartTime).Seconds()) + + // Check so that wg.Done is not called more than (threshold - 1) times which causes hardlock + thresholdPeersMutex.Lock() + if len(nonces) < threshold-1 { + nonces[peer] = peerNonces.Nonces + defer wg.Done() + } + thresholdPeersMutex.Unlock() +} diff --git a/pkg/proto/cosigner_grpc_server.pb.go b/pkg/proto/cosigner_grpc_server.pb.go index b481b631..5b512d63 100644 --- a/pkg/proto/cosigner_grpc_server.pb.go +++ b/pkg/proto/cosigner_grpc_server.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.29.0 +// protoc-gen-go v1.28.1 // protoc v3.13.0 // source: cosigner_grpc_server.proto @@ -99,7 +99,7 @@ func (x *Block) GetTimestamp() int64 { return 0 } -type CosignerGRPCSignBlockRequest struct { +type RaftGRPCSignBlockRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -108,8 +108,8 @@ type CosignerGRPCSignBlockRequest struct { Block *Block `protobuf:"bytes,2,opt,name=block,proto3" json:"block,omitempty"` } -func (x *CosignerGRPCSignBlockRequest) Reset() { - *x = CosignerGRPCSignBlockRequest{} +func (x *RaftGRPCSignBlockRequest) Reset() { + *x = RaftGRPCSignBlockRequest{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -117,13 +117,13 @@ func (x *CosignerGRPCSignBlockRequest) Reset() { } } -func (x *CosignerGRPCSignBlockRequest) String() string { +func (x *RaftGRPCSignBlockRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCSignBlockRequest) ProtoMessage() {} +func (*RaftGRPCSignBlockRequest) ProtoMessage() {} -func (x *CosignerGRPCSignBlockRequest) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCSignBlockRequest) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -135,26 +135,26 @@ func (x *CosignerGRPCSignBlockRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCSignBlockRequest.ProtoReflect.Descriptor instead. -func (*CosignerGRPCSignBlockRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCSignBlockRequest.ProtoReflect.Descriptor instead. +func (*RaftGRPCSignBlockRequest) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{1} } -func (x *CosignerGRPCSignBlockRequest) GetChainID() string { +func (x *RaftGRPCSignBlockRequest) GetChainID() string { if x != nil { return x.ChainID } return "" } -func (x *CosignerGRPCSignBlockRequest) GetBlock() *Block { +func (x *RaftGRPCSignBlockRequest) GetBlock() *Block { if x != nil { return x.Block } return nil } -type CosignerGRPCSignBlockResponse struct { +type RaftGRPCSignBlockResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -162,8 +162,8 @@ type CosignerGRPCSignBlockResponse struct { Signature []byte `protobuf:"bytes,1,opt,name=signature,proto3" json:"signature,omitempty"` } -func (x *CosignerGRPCSignBlockResponse) Reset() { - *x = CosignerGRPCSignBlockResponse{} +func (x *RaftGRPCSignBlockResponse) Reset() { + *x = RaftGRPCSignBlockResponse{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -171,13 +171,13 @@ func (x *CosignerGRPCSignBlockResponse) Reset() { } } -func (x *CosignerGRPCSignBlockResponse) String() string { +func (x *RaftGRPCSignBlockResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCSignBlockResponse) ProtoMessage() {} +func (*RaftGRPCSignBlockResponse) ProtoMessage() {} -func (x *CosignerGRPCSignBlockResponse) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCSignBlockResponse) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -189,12 +189,12 @@ func (x *CosignerGRPCSignBlockResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCSignBlockResponse.ProtoReflect.Descriptor instead. -func (*CosignerGRPCSignBlockResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCSignBlockResponse.ProtoReflect.Descriptor instead. +func (*RaftGRPCSignBlockResponse) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{2} } -func (x *CosignerGRPCSignBlockResponse) GetSignature() []byte { +func (x *RaftGRPCSignBlockResponse) GetSignature() []byte { if x != nil { return x.Signature } @@ -587,7 +587,7 @@ func (x *CosignerGRPCGetNoncesResponse) GetNonces() []*Nonce { return nil } -type CosignerGRPCTransferLeadershipRequest struct { +type RaftGRPCTransferLeadershipRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -595,8 +595,8 @@ type CosignerGRPCTransferLeadershipRequest struct { LeaderID string `protobuf:"bytes,1,opt,name=leaderID,proto3" json:"leaderID,omitempty"` } -func (x *CosignerGRPCTransferLeadershipRequest) Reset() { - *x = CosignerGRPCTransferLeadershipRequest{} +func (x *RaftGRPCTransferLeadershipRequest) Reset() { + *x = RaftGRPCTransferLeadershipRequest{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -604,13 +604,13 @@ func (x *CosignerGRPCTransferLeadershipRequest) Reset() { } } -func (x *CosignerGRPCTransferLeadershipRequest) String() string { +func (x *RaftGRPCTransferLeadershipRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCTransferLeadershipRequest) ProtoMessage() {} +func (*RaftGRPCTransferLeadershipRequest) ProtoMessage() {} -func (x *CosignerGRPCTransferLeadershipRequest) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCTransferLeadershipRequest) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -622,19 +622,19 @@ func (x *CosignerGRPCTransferLeadershipRequest) ProtoReflect() protoreflect.Mess return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCTransferLeadershipRequest.ProtoReflect.Descriptor instead. -func (*CosignerGRPCTransferLeadershipRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCTransferLeadershipRequest.ProtoReflect.Descriptor instead. +func (*RaftGRPCTransferLeadershipRequest) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{9} } -func (x *CosignerGRPCTransferLeadershipRequest) GetLeaderID() string { +func (x *RaftGRPCTransferLeadershipRequest) GetLeaderID() string { if x != nil { return x.LeaderID } return "" } -type CosignerGRPCTransferLeadershipResponse struct { +type RaftGRPCTransferLeadershipResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -643,8 +643,8 @@ type CosignerGRPCTransferLeadershipResponse struct { LeaderAddress string `protobuf:"bytes,2,opt,name=leaderAddress,proto3" json:"leaderAddress,omitempty"` } -func (x *CosignerGRPCTransferLeadershipResponse) Reset() { - *x = CosignerGRPCTransferLeadershipResponse{} +func (x *RaftGRPCTransferLeadershipResponse) Reset() { + *x = RaftGRPCTransferLeadershipResponse{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -652,13 +652,13 @@ func (x *CosignerGRPCTransferLeadershipResponse) Reset() { } } -func (x *CosignerGRPCTransferLeadershipResponse) String() string { +func (x *RaftGRPCTransferLeadershipResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCTransferLeadershipResponse) ProtoMessage() {} +func (*RaftGRPCTransferLeadershipResponse) ProtoMessage() {} -func (x *CosignerGRPCTransferLeadershipResponse) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCTransferLeadershipResponse) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -670,33 +670,33 @@ func (x *CosignerGRPCTransferLeadershipResponse) ProtoReflect() protoreflect.Mes return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCTransferLeadershipResponse.ProtoReflect.Descriptor instead. -func (*CosignerGRPCTransferLeadershipResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCTransferLeadershipResponse.ProtoReflect.Descriptor instead. +func (*RaftGRPCTransferLeadershipResponse) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{10} } -func (x *CosignerGRPCTransferLeadershipResponse) GetLeaderID() string { +func (x *RaftGRPCTransferLeadershipResponse) GetLeaderID() string { if x != nil { return x.LeaderID } return "" } -func (x *CosignerGRPCTransferLeadershipResponse) GetLeaderAddress() string { +func (x *RaftGRPCTransferLeadershipResponse) GetLeaderAddress() string { if x != nil { return x.LeaderAddress } return "" } -type CosignerGRPCGetLeaderRequest struct { +type RaftGRPCGetLeaderRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields } -func (x *CosignerGRPCGetLeaderRequest) Reset() { - *x = CosignerGRPCGetLeaderRequest{} +func (x *RaftGRPCGetLeaderRequest) Reset() { + *x = RaftGRPCGetLeaderRequest{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -704,13 +704,13 @@ func (x *CosignerGRPCGetLeaderRequest) Reset() { } } -func (x *CosignerGRPCGetLeaderRequest) String() string { +func (x *RaftGRPCGetLeaderRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCGetLeaderRequest) ProtoMessage() {} +func (*RaftGRPCGetLeaderRequest) ProtoMessage() {} -func (x *CosignerGRPCGetLeaderRequest) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCGetLeaderRequest) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -722,12 +722,12 @@ func (x *CosignerGRPCGetLeaderRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCGetLeaderRequest.ProtoReflect.Descriptor instead. -func (*CosignerGRPCGetLeaderRequest) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCGetLeaderRequest.ProtoReflect.Descriptor instead. +func (*RaftGRPCGetLeaderRequest) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{11} } -type CosignerGRPCGetLeaderResponse struct { +type RaftGRPCGetLeaderResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -735,8 +735,8 @@ type CosignerGRPCGetLeaderResponse struct { Leader string `protobuf:"bytes,1,opt,name=leader,proto3" json:"leader,omitempty"` } -func (x *CosignerGRPCGetLeaderResponse) Reset() { - *x = CosignerGRPCGetLeaderResponse{} +func (x *RaftGRPCGetLeaderResponse) Reset() { + *x = RaftGRPCGetLeaderResponse{} if protoimpl.UnsafeEnabled { mi := &file_cosigner_grpc_server_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -744,13 +744,13 @@ func (x *CosignerGRPCGetLeaderResponse) Reset() { } } -func (x *CosignerGRPCGetLeaderResponse) String() string { +func (x *RaftGRPCGetLeaderResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*CosignerGRPCGetLeaderResponse) ProtoMessage() {} +func (*RaftGRPCGetLeaderResponse) ProtoMessage() {} -func (x *CosignerGRPCGetLeaderResponse) ProtoReflect() protoreflect.Message { +func (x *RaftGRPCGetLeaderResponse) ProtoReflect() protoreflect.Message { mi := &file_cosigner_grpc_server_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) @@ -762,12 +762,12 @@ func (x *CosignerGRPCGetLeaderResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use CosignerGRPCGetLeaderResponse.ProtoReflect.Descriptor instead. -func (*CosignerGRPCGetLeaderResponse) Descriptor() ([]byte, []int) { +// Deprecated: Use RaftGRPCGetLeaderResponse.ProtoReflect.Descriptor instead. +func (*RaftGRPCGetLeaderResponse) Descriptor() ([]byte, []int) { return file_cosigner_grpc_server_proto_rawDescGZIP(), []int{12} } -func (x *CosignerGRPCGetLeaderResponse) GetLeader() string { +func (x *RaftGRPCGetLeaderResponse) GetLeader() string { if x != nil { return x.Leader } @@ -787,115 +787,113 @@ var file_cosigner_grpc_server_proto_rawDesc = []byte{ 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x42, 0x79, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x5c, 0x0a, 0x1c, 0x43, - 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, - 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, - 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, - 0x61, 0x69, 0x6e, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x22, 0x3d, 0x0a, 0x1d, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, - 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x95, 0x01, 0x0a, 0x05, 0x4e, 0x6f, 0x6e, - 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x12, 0x24, - 0x0a, 0x0d, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x75, 0x62, 0x4b, 0x65, 0x79, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x70, 0x75, 0x62, 0x4b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, - 0x73, 0x68, 0x61, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x73, 0x68, 0x61, - 0x72, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, - 0x22, 0x66, 0x0a, 0x04, 0x48, 0x52, 0x53, 0x54, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x69, 0x67, - 0x68, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, - 0x12, 0x14, 0x0a, 0x05, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x05, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0xa4, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, - 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x24, 0x0a, 0x06, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x52, 0x06, - 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x04, 0x68, 0x72, 0x73, 0x74, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x52, 0x53, - 0x54, 0x52, 0x04, 0x68, 0x72, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x42, - 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, - 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x22, - 0x84, 0x01, 0x0a, 0x24, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, - 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x6e, 0x6f, 0x6e, 0x63, - 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6e, - 0x6f, 0x6e, 0x63, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, - 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x59, 0x0a, 0x1c, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, - 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x04, 0x68, 0x72, 0x73, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x52, 0x53, - 0x54, 0x52, 0x04, 0x68, 0x72, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, - 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, - 0x44, 0x22, 0x45, 0x0a, 0x1d, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, - 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x4e, 0x6f, 0x6e, 0x63, 0x65, - 0x52, 0x06, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x22, 0x43, 0x0a, 0x25, 0x43, 0x6f, 0x73, 0x69, - 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, - 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x44, 0x22, 0x6a, 0x0a, - 0x26, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, + 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x58, 0x0a, 0x18, 0x52, + 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, + 0x44, 0x12, 0x22, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x05, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x22, 0x39, 0x0a, 0x19, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, + 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x22, 0x95, 0x01, 0x0a, 0x05, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x64, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x64, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, + 0x70, 0x75, 0x62, 0x4b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x70, 0x75, + 0x62, 0x4b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x68, 0x61, 0x72, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x05, 0x73, 0x68, 0x61, 0x72, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, + 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, + 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x66, 0x0a, 0x04, 0x48, 0x52, 0x53, 0x54, + 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x72, 0x6f, 0x75, 0x6e, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x12, 0x12, + 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x73, 0x74, + 0x65, 0x70, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x22, 0xa4, 0x01, 0x0a, 0x23, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, + 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x06, 0x6e, 0x6f, 0x6e, 0x63, + 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x52, 0x06, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x12, 0x1f, + 0x0a, 0x04, 0x68, 0x72, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x52, 0x53, 0x54, 0x52, 0x04, 0x68, 0x72, 0x73, 0x74, 0x12, + 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x42, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x22, 0x84, 0x01, 0x0a, 0x24, 0x43, 0x6f, 0x73, 0x69, + 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, + 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x20, 0x0a, 0x0b, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x50, 0x75, 0x62, 0x6c, + 0x69, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x22, 0x59, + 0x0a, 0x1c, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, + 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, + 0x0a, 0x04, 0x68, 0x72, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x48, 0x52, 0x53, 0x54, 0x52, 0x04, 0x68, 0x72, 0x73, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x44, 0x22, 0x45, 0x0a, 0x1d, 0x43, 0x6f, 0x73, + 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, + 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x24, 0x0a, 0x06, 0x6e, 0x6f, + 0x6e, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x52, 0x06, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x73, + 0x22, 0x3f, 0x0a, 0x21, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, + 0x44, 0x22, 0x66, 0x0a, 0x22, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x1e, 0x0a, 0x1c, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x37, 0x0a, 0x1d, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6c, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x32, 0x87, 0x04, 0x0a, 0x13, 0x49, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, - 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x58, 0x0a, 0x09, 0x53, 0x69, - 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x69, 0x67, 0x6e, - 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, + 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x61, 0x66, + 0x74, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x33, 0x0a, 0x19, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, + 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x32, 0xd8, 0x01, 0x0a, 0x0d, 0x49, + 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x12, 0x6d, 0x0a, 0x10, + 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, + 0x12, 0x2a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, + 0x64, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, - 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, - 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x12, 0x2a, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x74, - 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, - 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, - 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, - 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, - 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, - 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, - 0x12, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, - 0x68, 0x69, 0x70, 0x12, 0x2c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, - 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, - 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x2d, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, - 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, + 0x43, 0x53, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x41, 0x6e, 0x64, 0x53, 0x69, 0x67, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, + 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x12, 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, + 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, + 0x50, 0x43, 0x47, 0x65, 0x74, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0x9c, 0x02, 0x0a, 0x09, 0x49, 0x52, 0x61, 0x66, 0x74, 0x47, + 0x52, 0x50, 0x43, 0x12, 0x50, 0x0a, 0x09, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x12, 0x1f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, + 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, + 0x50, 0x43, 0x53, 0x69, 0x67, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x12, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, + 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x12, 0x28, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, + 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x68, 0x69, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, - 0x23, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x72, - 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x43, 0x6f, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x72, 0x47, 0x52, 0x50, 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x74, 0x72, 0x61, 0x6e, - 0x67, 0x65, 0x6c, 0x6f, 0x76, 0x65, 0x2d, 0x76, 0x65, 0x6e, 0x74, 0x75, 0x72, 0x65, 0x73, 0x2f, - 0x68, 0x6f, 0x72, 0x63, 0x72, 0x75, 0x78, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, + 0x1f, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, 0x43, + 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x20, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x47, 0x52, 0x50, + 0x43, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x73, 0x74, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x6c, 0x6f, 0x76, 0x65, 0x2d, 0x76, + 0x65, 0x6e, 0x74, 0x75, 0x72, 0x65, 0x73, 0x2f, 0x68, 0x6f, 0x72, 0x63, 0x72, 0x75, 0x78, 0x2f, + 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( @@ -912,36 +910,36 @@ func file_cosigner_grpc_server_proto_rawDescGZIP() []byte { var file_cosigner_grpc_server_proto_msgTypes = make([]protoimpl.MessageInfo, 13) var file_cosigner_grpc_server_proto_goTypes = []interface{}{ - (*Block)(nil), // 0: proto.Block - (*CosignerGRPCSignBlockRequest)(nil), // 1: proto.CosignerGRPCSignBlockRequest - (*CosignerGRPCSignBlockResponse)(nil), // 2: proto.CosignerGRPCSignBlockResponse - (*Nonce)(nil), // 3: proto.Nonce - (*HRST)(nil), // 4: proto.HRST - (*CosignerGRPCSetNoncesAndSignRequest)(nil), // 5: proto.CosignerGRPCSetNoncesAndSignRequest - (*CosignerGRPCSetNoncesAndSignResponse)(nil), // 6: proto.CosignerGRPCSetNoncesAndSignResponse - (*CosignerGRPCGetNoncesRequest)(nil), // 7: proto.CosignerGRPCGetNoncesRequest - (*CosignerGRPCGetNoncesResponse)(nil), // 8: proto.CosignerGRPCGetNoncesResponse - (*CosignerGRPCTransferLeadershipRequest)(nil), // 9: proto.CosignerGRPCTransferLeadershipRequest - (*CosignerGRPCTransferLeadershipResponse)(nil), // 10: proto.CosignerGRPCTransferLeadershipResponse - (*CosignerGRPCGetLeaderRequest)(nil), // 11: proto.CosignerGRPCGetLeaderRequest - (*CosignerGRPCGetLeaderResponse)(nil), // 12: proto.CosignerGRPCGetLeaderResponse + (*Block)(nil), // 0: proto.Block + (*RaftGRPCSignBlockRequest)(nil), // 1: proto.RaftGRPCSignBlockRequest + (*RaftGRPCSignBlockResponse)(nil), // 2: proto.RaftGRPCSignBlockResponse + (*Nonce)(nil), // 3: proto.Nonce + (*HRST)(nil), // 4: proto.HRST + (*CosignerGRPCSetNoncesAndSignRequest)(nil), // 5: proto.CosignerGRPCSetNoncesAndSignRequest + (*CosignerGRPCSetNoncesAndSignResponse)(nil), // 6: proto.CosignerGRPCSetNoncesAndSignResponse + (*CosignerGRPCGetNoncesRequest)(nil), // 7: proto.CosignerGRPCGetNoncesRequest + (*CosignerGRPCGetNoncesResponse)(nil), // 8: proto.CosignerGRPCGetNoncesResponse + (*RaftGRPCTransferLeadershipRequest)(nil), // 9: proto.RaftGRPCTransferLeadershipRequest + (*RaftGRPCTransferLeadershipResponse)(nil), // 10: proto.RaftGRPCTransferLeadershipResponse + (*RaftGRPCGetLeaderRequest)(nil), // 11: proto.RaftGRPCGetLeaderRequest + (*RaftGRPCGetLeaderResponse)(nil), // 12: proto.RaftGRPCGetLeaderResponse } var file_cosigner_grpc_server_proto_depIdxs = []int32{ - 0, // 0: proto.CosignerGRPCSignBlockRequest.block:type_name -> proto.Block + 0, // 0: proto.RaftGRPCSignBlockRequest.block:type_name -> proto.Block 3, // 1: proto.CosignerGRPCSetNoncesAndSignRequest.nonces:type_name -> proto.Nonce 4, // 2: proto.CosignerGRPCSetNoncesAndSignRequest.hrst:type_name -> proto.HRST 4, // 3: proto.CosignerGRPCGetNoncesRequest.hrst:type_name -> proto.HRST 3, // 4: proto.CosignerGRPCGetNoncesResponse.nonces:type_name -> proto.Nonce - 1, // 5: proto.ICosignerGRPCServer.SignBlock:input_type -> proto.CosignerGRPCSignBlockRequest - 5, // 6: proto.ICosignerGRPCServer.SetNoncesAndSign:input_type -> proto.CosignerGRPCSetNoncesAndSignRequest - 7, // 7: proto.ICosignerGRPCServer.GetNonces:input_type -> proto.CosignerGRPCGetNoncesRequest - 9, // 8: proto.ICosignerGRPCServer.TransferLeadership:input_type -> proto.CosignerGRPCTransferLeadershipRequest - 11, // 9: proto.ICosignerGRPCServer.GetLeader:input_type -> proto.CosignerGRPCGetLeaderRequest - 2, // 10: proto.ICosignerGRPCServer.SignBlock:output_type -> proto.CosignerGRPCSignBlockResponse - 6, // 11: proto.ICosignerGRPCServer.SetNoncesAndSign:output_type -> proto.CosignerGRPCSetNoncesAndSignResponse - 8, // 12: proto.ICosignerGRPCServer.GetNonces:output_type -> proto.CosignerGRPCGetNoncesResponse - 10, // 13: proto.ICosignerGRPCServer.TransferLeadership:output_type -> proto.CosignerGRPCTransferLeadershipResponse - 12, // 14: proto.ICosignerGRPCServer.GetLeader:output_type -> proto.CosignerGRPCGetLeaderResponse + 5, // 5: proto.ICosignerGRPC.SetNoncesAndSign:input_type -> proto.CosignerGRPCSetNoncesAndSignRequest + 7, // 6: proto.ICosignerGRPC.GetNonces:input_type -> proto.CosignerGRPCGetNoncesRequest + 1, // 7: proto.IRaftGRPC.SignBlock:input_type -> proto.RaftGRPCSignBlockRequest + 9, // 8: proto.IRaftGRPC.TransferLeadership:input_type -> proto.RaftGRPCTransferLeadershipRequest + 11, // 9: proto.IRaftGRPC.GetLeader:input_type -> proto.RaftGRPCGetLeaderRequest + 6, // 10: proto.ICosignerGRPC.SetNoncesAndSign:output_type -> proto.CosignerGRPCSetNoncesAndSignResponse + 8, // 11: proto.ICosignerGRPC.GetNonces:output_type -> proto.CosignerGRPCGetNoncesResponse + 2, // 12: proto.IRaftGRPC.SignBlock:output_type -> proto.RaftGRPCSignBlockResponse + 10, // 13: proto.IRaftGRPC.TransferLeadership:output_type -> proto.RaftGRPCTransferLeadershipResponse + 12, // 14: proto.IRaftGRPC.GetLeader:output_type -> proto.RaftGRPCGetLeaderResponse 10, // [10:15] is the sub-list for method output_type 5, // [5:10] is the sub-list for method input_type 5, // [5:5] is the sub-list for extension type_name @@ -968,7 +966,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCSignBlockRequest); i { + switch v := v.(*RaftGRPCSignBlockRequest); i { case 0: return &v.state case 1: @@ -980,7 +978,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCSignBlockResponse); i { + switch v := v.(*RaftGRPCSignBlockResponse); i { case 0: return &v.state case 1: @@ -1064,7 +1062,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCTransferLeadershipRequest); i { + switch v := v.(*RaftGRPCTransferLeadershipRequest); i { case 0: return &v.state case 1: @@ -1076,7 +1074,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCTransferLeadershipResponse); i { + switch v := v.(*RaftGRPCTransferLeadershipResponse); i { case 0: return &v.state case 1: @@ -1088,7 +1086,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCGetLeaderRequest); i { + switch v := v.(*RaftGRPCGetLeaderRequest); i { case 0: return &v.state case 1: @@ -1100,7 +1098,7 @@ func file_cosigner_grpc_server_proto_init() { } } file_cosigner_grpc_server_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CosignerGRPCGetLeaderResponse); i { + switch v := v.(*RaftGRPCGetLeaderResponse); i { case 0: return &v.state case 1: @@ -1120,7 +1118,7 @@ func file_cosigner_grpc_server_proto_init() { NumEnums: 0, NumMessages: 13, NumExtensions: 0, - NumServices: 1, + NumServices: 2, }, GoTypes: file_cosigner_grpc_server_proto_goTypes, DependencyIndexes: file_cosigner_grpc_server_proto_depIdxs, diff --git a/pkg/proto/cosigner_grpc_server.proto b/pkg/proto/cosigner_grpc_server.proto index 1fcfc588..a0dfe514 100644 --- a/pkg/proto/cosigner_grpc_server.proto +++ b/pkg/proto/cosigner_grpc_server.proto @@ -3,16 +3,26 @@ syntax = "proto3"; option go_package = "github.com/strangelove-ventures/horcrux/pkg/proto"; package proto; -// TODO: Rename it as GPRCserver. It makes more sense since we are using it for all connections. -// CosignerGRPC handles all the services (the server) -service ICosignerGRPCServer { - rpc SignBlock (CosignerGRPCSignBlockRequest) returns (CosignerGRPCSignBlockResponse) {} - rpc SetNoncesAndSign (CosignerGRPCSetNoncesAndSignRequest) returns (CosignerGRPCSetNoncesAndSignResponse) {} - rpc GetNonces (CosignerGRPCGetNoncesRequest) returns (CosignerGRPCGetNoncesResponse) {} - rpc TransferLeadership (CosignerGRPCTransferLeadershipRequest) returns (CosignerGRPCTransferLeadershipResponse) {} - rpc GetLeader (CosignerGRPCGetLeaderRequest) returns (CosignerGRPCGetLeaderResponse) {} +// TODO: Should we split the service(?) into 2 one for "RAFT" and one for "Cosigner" +// CosignerGRPC handles all the cosigner level "services" +// All RPCs in ICosignerGRPCServer are Unary. This means the client sends a single request to the +// server and gets a single response back, just like a normal function call. +service ICosignerGRPC { + rpc SetNoncesAndSign (CosignerGRPCSetNoncesAndSignRequest) returns (CosignerGRPCSetNoncesAndSignResponse) {} + rpc GetNonces (CosignerGRPCGetNoncesRequest) returns (CosignerGRPCGetNoncesResponse) {} + +} + +// IRaftGRPC handles all the cosigner level "services" +// All RPCs in ICosignerGRPCServer are Unary. This means the client sends a single request to the +// server and gets a single response back, just like a normal function call +service IRaftGRPC{ + rpc SignBlock (RaftGRPCSignBlockRequest) returns (RaftGRPCSignBlockResponse) {} + rpc TransferLeadership (RaftGRPCTransferLeadershipRequest) returns (RaftGRPCTransferLeadershipResponse) {} + rpc GetLeader (RaftGRPCGetLeaderRequest) returns (RaftGRPCGetLeaderResponse) {} } + message Block { int64 height = 1; int64 round = 2; @@ -21,12 +31,12 @@ message Block { int64 timestamp = 5; } -message CosignerGRPCSignBlockRequest { +message RaftGRPCSignBlockRequest { string chainID = 1; Block block = 2; } -message CosignerGRPCSignBlockResponse { +message RaftGRPCSignBlockResponse { bytes signature = 1; } @@ -41,15 +51,15 @@ message Nonce { message HRST { int64 height = 1; int64 round = 2; - int32 step = 3; - int64 timestamp = 4; + int32 step = 3; + int64 timestamp = 4; } message CosignerGRPCSetNoncesAndSignRequest { repeated Nonce nonces = 1; HRST hrst = 2; bytes signBytes = 3; - string chainID = 4; + string chainID = 4; } message CosignerGRPCSetNoncesAndSignResponse { @@ -60,24 +70,24 @@ message CosignerGRPCSetNoncesAndSignResponse { message CosignerGRPCGetNoncesRequest { HRST hrst = 1; - string chainID = 2; + string chainID = 2; } message CosignerGRPCGetNoncesResponse { repeated Nonce nonces = 1; } -message CosignerGRPCTransferLeadershipRequest { - string leaderID = 1; +message RaftGRPCTransferLeadershipRequest { + string leaderID = 1; } -message CosignerGRPCTransferLeadershipResponse { - string leaderID = 1; - string leaderAddress = 2; +message RaftGRPCTransferLeadershipResponse { + string leaderID = 1; + string leaderAddress = 2; } -message CosignerGRPCGetLeaderRequest {} +message RaftGRPCGetLeaderRequest {} -message CosignerGRPCGetLeaderResponse { - string leader = 1; +message RaftGRPCGetLeaderResponse { + string leader = 1; } diff --git a/pkg/proto/cosigner_grpc_server_grpc.pb.go b/pkg/proto/cosigner_grpc_server_grpc.pb.go index f7c0b5df..79a36a62 100644 --- a/pkg/proto/cosigner_grpc_server_grpc.pb.go +++ b/pkg/proto/cosigner_grpc_server_grpc.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: -// - protoc-gen-go-grpc v1.3.0 +// - protoc-gen-go-grpc v1.2.0 // - protoc v3.13.0 // source: cosigner_grpc_server.proto @@ -18,238 +18,280 @@ import ( // Requires gRPC-Go v1.32.0 or later. const _ = grpc.SupportPackageIsVersion7 -const ( - ICosignerGRPCServer_SignBlock_FullMethodName = "/proto.ICosignerGRPCServer/SignBlock" - ICosignerGRPCServer_SetNoncesAndSign_FullMethodName = "/proto.ICosignerGRPCServer/SetNoncesAndSign" - ICosignerGRPCServer_GetNonces_FullMethodName = "/proto.ICosignerGRPCServer/GetNonces" - ICosignerGRPCServer_TransferLeadership_FullMethodName = "/proto.ICosignerGRPCServer/TransferLeadership" - ICosignerGRPCServer_GetLeader_FullMethodName = "/proto.ICosignerGRPCServer/GetLeader" -) - -// ICosignerGRPCServerClient is the client API for ICosignerGRPCServer service. +// ICosignerGRPCClient is the client API for ICosignerGRPC 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. -type ICosignerGRPCServerClient interface { - SignBlock(ctx context.Context, in *CosignerGRPCSignBlockRequest, opts ...grpc.CallOption) (*CosignerGRPCSignBlockResponse, error) +type ICosignerGRPCClient interface { SetNoncesAndSign(ctx context.Context, in *CosignerGRPCSetNoncesAndSignRequest, opts ...grpc.CallOption) (*CosignerGRPCSetNoncesAndSignResponse, error) GetNonces(ctx context.Context, in *CosignerGRPCGetNoncesRequest, opts ...grpc.CallOption) (*CosignerGRPCGetNoncesResponse, error) - TransferLeadership(ctx context.Context, in *CosignerGRPCTransferLeadershipRequest, opts ...grpc.CallOption) (*CosignerGRPCTransferLeadershipResponse, error) - GetLeader(ctx context.Context, in *CosignerGRPCGetLeaderRequest, opts ...grpc.CallOption) (*CosignerGRPCGetLeaderResponse, error) } -type iCosignerGRPCServerClient struct { +type iCosignerGRPCClient struct { cc grpc.ClientConnInterface } -func NewICosignerGRPCServerClient(cc grpc.ClientConnInterface) ICosignerGRPCServerClient { - return &iCosignerGRPCServerClient{cc} -} - -func (c *iCosignerGRPCServerClient) SignBlock(ctx context.Context, in *CosignerGRPCSignBlockRequest, opts ...grpc.CallOption) (*CosignerGRPCSignBlockResponse, error) { - out := new(CosignerGRPCSignBlockResponse) - err := c.cc.Invoke(ctx, ICosignerGRPCServer_SignBlock_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil +func NewICosignerGRPCClient(cc grpc.ClientConnInterface) ICosignerGRPCClient { + return &iCosignerGRPCClient{cc} } -func (c *iCosignerGRPCServerClient) SetNoncesAndSign(ctx context.Context, in *CosignerGRPCSetNoncesAndSignRequest, opts ...grpc.CallOption) (*CosignerGRPCSetNoncesAndSignResponse, error) { +func (c *iCosignerGRPCClient) SetNoncesAndSign(ctx context.Context, in *CosignerGRPCSetNoncesAndSignRequest, opts ...grpc.CallOption) (*CosignerGRPCSetNoncesAndSignResponse, error) { out := new(CosignerGRPCSetNoncesAndSignResponse) - err := c.cc.Invoke(ctx, ICosignerGRPCServer_SetNoncesAndSign_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, "/proto.ICosignerGRPC/SetNoncesAndSign", in, out, opts...) if err != nil { return nil, err } return out, nil } -func (c *iCosignerGRPCServerClient) GetNonces(ctx context.Context, in *CosignerGRPCGetNoncesRequest, opts ...grpc.CallOption) (*CosignerGRPCGetNoncesResponse, error) { +func (c *iCosignerGRPCClient) GetNonces(ctx context.Context, in *CosignerGRPCGetNoncesRequest, opts ...grpc.CallOption) (*CosignerGRPCGetNoncesResponse, error) { out := new(CosignerGRPCGetNoncesResponse) - err := c.cc.Invoke(ctx, ICosignerGRPCServer_GetNonces_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *iCosignerGRPCServerClient) TransferLeadership(ctx context.Context, in *CosignerGRPCTransferLeadershipRequest, opts ...grpc.CallOption) (*CosignerGRPCTransferLeadershipResponse, error) { - out := new(CosignerGRPCTransferLeadershipResponse) - err := c.cc.Invoke(ctx, ICosignerGRPCServer_TransferLeadership_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - -func (c *iCosignerGRPCServerClient) GetLeader(ctx context.Context, in *CosignerGRPCGetLeaderRequest, opts ...grpc.CallOption) (*CosignerGRPCGetLeaderResponse, error) { - out := new(CosignerGRPCGetLeaderResponse) - err := c.cc.Invoke(ctx, ICosignerGRPCServer_GetLeader_FullMethodName, in, out, opts...) + err := c.cc.Invoke(ctx, "/proto.ICosignerGRPC/GetNonces", in, out, opts...) if err != nil { return nil, err } return out, nil } -// ICosignerGRPCServerServer is the server API for ICosignerGRPCServer service. -// All implementations must embed UnimplementedICosignerGRPCServerServer +// ICosignerGRPCServer is the server API for ICosignerGRPC service. +// All implementations must embed UnimplementedICosignerGRPCServer // for forward compatibility -type ICosignerGRPCServerServer interface { - SignBlock(context.Context, *CosignerGRPCSignBlockRequest) (*CosignerGRPCSignBlockResponse, error) +type ICosignerGRPCServer interface { SetNoncesAndSign(context.Context, *CosignerGRPCSetNoncesAndSignRequest) (*CosignerGRPCSetNoncesAndSignResponse, error) GetNonces(context.Context, *CosignerGRPCGetNoncesRequest) (*CosignerGRPCGetNoncesResponse, error) - TransferLeadership(context.Context, *CosignerGRPCTransferLeadershipRequest) (*CosignerGRPCTransferLeadershipResponse, error) - GetLeader(context.Context, *CosignerGRPCGetLeaderRequest) (*CosignerGRPCGetLeaderResponse, error) - mustEmbedUnimplementedICosignerGRPCServerServer() + mustEmbedUnimplementedICosignerGRPCServer() } -// UnimplementedICosignerGRPCServerServer must be embedded to have forward compatible implementations. -type UnimplementedICosignerGRPCServerServer struct { +// UnimplementedICosignerGRPCServer must be embedded to have forward compatible implementations. +type UnimplementedICosignerGRPCServer struct { } -func (UnimplementedICosignerGRPCServerServer) SignBlock(context.Context, *CosignerGRPCSignBlockRequest) (*CosignerGRPCSignBlockResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method SignBlock not implemented") -} -func (UnimplementedICosignerGRPCServerServer) SetNoncesAndSign(context.Context, *CosignerGRPCSetNoncesAndSignRequest) (*CosignerGRPCSetNoncesAndSignResponse, error) { +func (UnimplementedICosignerGRPCServer) SetNoncesAndSign(context.Context, *CosignerGRPCSetNoncesAndSignRequest) (*CosignerGRPCSetNoncesAndSignResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method SetNoncesAndSign not implemented") } -func (UnimplementedICosignerGRPCServerServer) GetNonces(context.Context, *CosignerGRPCGetNoncesRequest) (*CosignerGRPCGetNoncesResponse, error) { +func (UnimplementedICosignerGRPCServer) GetNonces(context.Context, *CosignerGRPCGetNoncesRequest) (*CosignerGRPCGetNoncesResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method GetNonces not implemented") } -func (UnimplementedICosignerGRPCServerServer) TransferLeadership(context.Context, *CosignerGRPCTransferLeadershipRequest) (*CosignerGRPCTransferLeadershipResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method TransferLeadership not implemented") -} -func (UnimplementedICosignerGRPCServerServer) GetLeader(context.Context, *CosignerGRPCGetLeaderRequest) (*CosignerGRPCGetLeaderResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method GetLeader not implemented") -} -func (UnimplementedICosignerGRPCServerServer) mustEmbedUnimplementedICosignerGRPCServerServer() {} +func (UnimplementedICosignerGRPCServer) mustEmbedUnimplementedICosignerGRPCServer() {} -// UnsafeICosignerGRPCServerServer may be embedded to opt out of forward compatibility for this service. -// Use of this interface is not recommended, as added methods to ICosignerGRPCServerServer will +// UnsafeICosignerGRPCServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to ICosignerGRPCServer will // result in compilation errors. -type UnsafeICosignerGRPCServerServer interface { - mustEmbedUnimplementedICosignerGRPCServerServer() +type UnsafeICosignerGRPCServer interface { + mustEmbedUnimplementedICosignerGRPCServer() } -func RegisterICosignerGRPCServerServer(s grpc.ServiceRegistrar, srv ICosignerGRPCServerServer) { - s.RegisterService(&ICosignerGRPCServer_ServiceDesc, srv) +func RegisterICosignerGRPCServer(s grpc.ServiceRegistrar, srv ICosignerGRPCServer) { + s.RegisterService(&ICosignerGRPC_ServiceDesc, srv) } -func _ICosignerGRPCServer_SignBlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CosignerGRPCSignBlockRequest) +func _ICosignerGRPC_SetNoncesAndSign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CosignerGRPCSetNoncesAndSignRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(ICosignerGRPCServerServer).SignBlock(ctx, in) + return srv.(ICosignerGRPCServer).SetNoncesAndSign(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: ICosignerGRPCServer_SignBlock_FullMethodName, + FullMethod: "/proto.ICosignerGRPC/SetNoncesAndSign", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ICosignerGRPCServerServer).SignBlock(ctx, req.(*CosignerGRPCSignBlockRequest)) + return srv.(ICosignerGRPCServer).SetNoncesAndSign(ctx, req.(*CosignerGRPCSetNoncesAndSignRequest)) } return interceptor(ctx, in, info, handler) } -func _ICosignerGRPCServer_SetNoncesAndSign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CosignerGRPCSetNoncesAndSignRequest) +func _ICosignerGRPC_GetNonces_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CosignerGRPCGetNoncesRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(ICosignerGRPCServerServer).SetNoncesAndSign(ctx, in) + return srv.(ICosignerGRPCServer).GetNonces(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: ICosignerGRPCServer_SetNoncesAndSign_FullMethodName, + FullMethod: "/proto.ICosignerGRPC/GetNonces", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ICosignerGRPCServerServer).SetNoncesAndSign(ctx, req.(*CosignerGRPCSetNoncesAndSignRequest)) + return srv.(ICosignerGRPCServer).GetNonces(ctx, req.(*CosignerGRPCGetNoncesRequest)) } return interceptor(ctx, in, info, handler) } -func _ICosignerGRPCServer_GetNonces_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CosignerGRPCGetNoncesRequest) +// ICosignerGRPC_ServiceDesc is the grpc.ServiceDesc for ICosignerGRPC service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var ICosignerGRPC_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "proto.ICosignerGRPC", + HandlerType: (*ICosignerGRPCServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "SetNoncesAndSign", + Handler: _ICosignerGRPC_SetNoncesAndSign_Handler, + }, + { + MethodName: "GetNonces", + Handler: _ICosignerGRPC_GetNonces_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "cosigner_grpc_server.proto", +} + +// IRaftGRPCClient is the client API for IRaftGRPC 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. +type IRaftGRPCClient interface { + SignBlock(ctx context.Context, in *RaftGRPCSignBlockRequest, opts ...grpc.CallOption) (*RaftGRPCSignBlockResponse, error) + TransferLeadership(ctx context.Context, in *RaftGRPCTransferLeadershipRequest, opts ...grpc.CallOption) (*RaftGRPCTransferLeadershipResponse, error) + GetLeader(ctx context.Context, in *RaftGRPCGetLeaderRequest, opts ...grpc.CallOption) (*RaftGRPCGetLeaderResponse, error) +} + +type iRaftGRPCClient struct { + cc grpc.ClientConnInterface +} + +func NewIRaftGRPCClient(cc grpc.ClientConnInterface) IRaftGRPCClient { + return &iRaftGRPCClient{cc} +} + +func (c *iRaftGRPCClient) SignBlock(ctx context.Context, in *RaftGRPCSignBlockRequest, opts ...grpc.CallOption) (*RaftGRPCSignBlockResponse, error) { + out := new(RaftGRPCSignBlockResponse) + err := c.cc.Invoke(ctx, "/proto.IRaftGRPC/SignBlock", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *iRaftGRPCClient) TransferLeadership(ctx context.Context, in *RaftGRPCTransferLeadershipRequest, opts ...grpc.CallOption) (*RaftGRPCTransferLeadershipResponse, error) { + out := new(RaftGRPCTransferLeadershipResponse) + err := c.cc.Invoke(ctx, "/proto.IRaftGRPC/TransferLeadership", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *iRaftGRPCClient) GetLeader(ctx context.Context, in *RaftGRPCGetLeaderRequest, opts ...grpc.CallOption) (*RaftGRPCGetLeaderResponse, error) { + out := new(RaftGRPCGetLeaderResponse) + err := c.cc.Invoke(ctx, "/proto.IRaftGRPC/GetLeader", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// IRaftGRPCServer is the server API for IRaftGRPC service. +// All implementations must embed UnimplementedIRaftGRPCServer +// for forward compatibility +type IRaftGRPCServer interface { + SignBlock(context.Context, *RaftGRPCSignBlockRequest) (*RaftGRPCSignBlockResponse, error) + TransferLeadership(context.Context, *RaftGRPCTransferLeadershipRequest) (*RaftGRPCTransferLeadershipResponse, error) + GetLeader(context.Context, *RaftGRPCGetLeaderRequest) (*RaftGRPCGetLeaderResponse, error) + mustEmbedUnimplementedIRaftGRPCServer() +} + +// UnimplementedIRaftGRPCServer must be embedded to have forward compatible implementations. +type UnimplementedIRaftGRPCServer struct { +} + +func (UnimplementedIRaftGRPCServer) SignBlock(context.Context, *RaftGRPCSignBlockRequest) (*RaftGRPCSignBlockResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SignBlock not implemented") +} +func (UnimplementedIRaftGRPCServer) TransferLeadership(context.Context, *RaftGRPCTransferLeadershipRequest) (*RaftGRPCTransferLeadershipResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method TransferLeadership not implemented") +} +func (UnimplementedIRaftGRPCServer) GetLeader(context.Context, *RaftGRPCGetLeaderRequest) (*RaftGRPCGetLeaderResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetLeader not implemented") +} +func (UnimplementedIRaftGRPCServer) mustEmbedUnimplementedIRaftGRPCServer() {} + +// UnsafeIRaftGRPCServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to IRaftGRPCServer will +// result in compilation errors. +type UnsafeIRaftGRPCServer interface { + mustEmbedUnimplementedIRaftGRPCServer() +} + +func RegisterIRaftGRPCServer(s grpc.ServiceRegistrar, srv IRaftGRPCServer) { + s.RegisterService(&IRaftGRPC_ServiceDesc, srv) +} + +func _IRaftGRPC_SignBlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RaftGRPCSignBlockRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(ICosignerGRPCServerServer).GetNonces(ctx, in) + return srv.(IRaftGRPCServer).SignBlock(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: ICosignerGRPCServer_GetNonces_FullMethodName, + FullMethod: "/proto.IRaftGRPC/SignBlock", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ICosignerGRPCServerServer).GetNonces(ctx, req.(*CosignerGRPCGetNoncesRequest)) + return srv.(IRaftGRPCServer).SignBlock(ctx, req.(*RaftGRPCSignBlockRequest)) } return interceptor(ctx, in, info, handler) } -func _ICosignerGRPCServer_TransferLeadership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CosignerGRPCTransferLeadershipRequest) +func _IRaftGRPC_TransferLeadership_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RaftGRPCTransferLeadershipRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(ICosignerGRPCServerServer).TransferLeadership(ctx, in) + return srv.(IRaftGRPCServer).TransferLeadership(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: ICosignerGRPCServer_TransferLeadership_FullMethodName, + FullMethod: "/proto.IRaftGRPC/TransferLeadership", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ICosignerGRPCServerServer).TransferLeadership(ctx, req.(*CosignerGRPCTransferLeadershipRequest)) + return srv.(IRaftGRPCServer).TransferLeadership(ctx, req.(*RaftGRPCTransferLeadershipRequest)) } return interceptor(ctx, in, info, handler) } -func _ICosignerGRPCServer_GetLeader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(CosignerGRPCGetLeaderRequest) +func _IRaftGRPC_GetLeader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(RaftGRPCGetLeaderRequest) if err := dec(in); err != nil { return nil, err } if interceptor == nil { - return srv.(ICosignerGRPCServerServer).GetLeader(ctx, in) + return srv.(IRaftGRPCServer).GetLeader(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: ICosignerGRPCServer_GetLeader_FullMethodName, + FullMethod: "/proto.IRaftGRPC/GetLeader", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(ICosignerGRPCServerServer).GetLeader(ctx, req.(*CosignerGRPCGetLeaderRequest)) + return srv.(IRaftGRPCServer).GetLeader(ctx, req.(*RaftGRPCGetLeaderRequest)) } return interceptor(ctx, in, info, handler) } -// ICosignerGRPCServer_ServiceDesc is the grpc.ServiceDesc for ICosignerGRPCServer service. +// IRaftGRPC_ServiceDesc is the grpc.ServiceDesc for IRaftGRPC service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) -var ICosignerGRPCServer_ServiceDesc = grpc.ServiceDesc{ - ServiceName: "proto.ICosignerGRPCServer", - HandlerType: (*ICosignerGRPCServerServer)(nil), +var IRaftGRPC_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "proto.IRaftGRPC", + HandlerType: (*IRaftGRPCServer)(nil), Methods: []grpc.MethodDesc{ { MethodName: "SignBlock", - Handler: _ICosignerGRPCServer_SignBlock_Handler, - }, - { - MethodName: "SetNoncesAndSign", - Handler: _ICosignerGRPCServer_SetNoncesAndSign_Handler, - }, - { - MethodName: "GetNonces", - Handler: _ICosignerGRPCServer_GetNonces_Handler, + Handler: _IRaftGRPC_SignBlock_Handler, }, { MethodName: "TransferLeadership", - Handler: _ICosignerGRPCServer_TransferLeadership_Handler, + Handler: _IRaftGRPC_TransferLeadership_Handler, }, { MethodName: "GetLeader", - Handler: _ICosignerGRPCServer_GetLeader_Handler, + Handler: _IRaftGRPC_GetLeader_Handler, }, }, Streams: []grpc.StreamDesc{}, diff --git a/pkg/proto/proto.sh b/pkg/proto/proto.sh new file mode 100644 index 00000000..c788027a --- /dev/null +++ b/pkg/proto/proto.sh @@ -0,0 +1 @@ +protoc --go_out=. --go_opt=paths=source_relative --go-grpc_out=. --go-grpc_opt=paths=source_relative cosigner_grpc_server.proto \ No newline at end of file diff --git a/pkg/types/sign_state.go b/pkg/types/sign_state.go index 66e4c88a..c1f05aef 100644 --- a/pkg/types/sign_state.go +++ b/pkg/types/sign_state.go @@ -409,8 +409,8 @@ func LoadOrCreateSignState(filepath string) (*SignState, error) { return nil, fmt.Errorf("unexpected error checking file existence (%s): %w", filepath, err) } - fmt.Printf("\n\tNo filepath for Signstate exist for:\n\t\t%s.", filepath) - fmt.Printf("\n\tThis means NO(!) Signstate exists so we will create a new Signstate.\n") + // fmt.Printf("\n\tNo filepath for Signstate exist for:\n\t\t%s.", filepath) + //fmt.Printf("\n\tThis means NO(!) Signstate exists so we will create a new Signstate.\n") // the only scenario where we want to create a new sign state file is when the file does not exist. // Make an empty sign state and save it. diff --git a/test/validator.go b/test/validator.go index 3d293e7c..6d7998fa 100644 --- a/test/validator.go +++ b/test/validator.go @@ -268,9 +268,9 @@ func getLeader(ctx context.Context, cosigner *cosmos.SidecarProcess) (string, er ctx, cancelFunc := context.WithTimeout(ctx, 10*time.Second) defer cancelFunc() - grpcClient := proto.NewICosignerGRPCServerClient(conn) + grpcClient := proto.NewIRaftGRPCClient(conn) - res, err := grpcClient.GetLeader(ctx, &proto.CosignerGRPCGetLeaderRequest{}) + res, err := grpcClient.GetLeader(ctx, &proto.RaftGRPCGetLeaderRequest{}) if err != nil { return "", err }