diff --git a/orderer/localconfig/config.go b/orderer/localconfig/config.go index c7f747ae876..83ecdbf942e 100644 --- a/orderer/localconfig/config.go +++ b/orderer/localconfig/config.go @@ -101,6 +101,20 @@ type Kafka struct { Version sarama.KafkaVersion } +// Sbft contains config for the SBFT orderer +type Sbft struct { + PeerCommAddr string + CertFile string + KeyFile string + DataDir string + N uint64 + F uint64 + BatchDurationNsec uint64 + BatchSizeBytes uint64 + RequestTimeoutNsec uint64 + Peers map[string]string // Address to Cert mapping +} + // Retry contains config for the reconnection attempts to the Kafka brokers type Retry struct { Period time.Duration @@ -118,6 +132,7 @@ type TopLevel struct { FileLedger FileLedger Kafka Kafka Genesis Genesis + Sbft Sbft } var defaults = TopLevel{ @@ -161,6 +176,20 @@ var defaults = TopLevel{ PreferredMaxBytes: 512 * 1024, }, }, + // TODO move the appropriate parts to Genesis + // and use BatchSizeMaxBytes + Sbft: Sbft{ + PeerCommAddr: ":6101", + CertFile: "sbft/testdata/cert1.pem", + KeyFile: "sbft/testdata/key.pem", + DataDir: "/tmp", + N: 1, + F: 0, + BatchDurationNsec: 1000, + BatchSizeBytes: 1000000000, + RequestTimeoutNsec: 1000000000, + Peers: map[string]string{":6101": "sbft/testdata/cert1.pem"}, + }, } func (c *TopLevel) completeInitialization() { diff --git a/orderer/localconfig/config_util.go b/orderer/localconfig/config_util.go index 6b753c9c1f6..ad3e761c3ac 100644 --- a/orderer/localconfig/config_util.go +++ b/orderer/localconfig/config_util.go @@ -25,6 +25,8 @@ import ( "strings" "time" + "encoding/json" + "github.com/mitchellh/mapstructure" "github.com/spf13/viper" ) @@ -48,6 +50,9 @@ func getKeysRecursively(base string, v *viper.Viper, nodeKeys map[string]interfa } else if m, ok := val.(map[string]interface{}); ok { logger.Debugf("Found map[string]interface{} value for %s", fqKey) result[key] = getKeysRecursively(fqKey+".", v, m) + } else if m, ok := unmarshalJson(val); ok { + logger.Debugf("Found real value for %s setting to map[string]string %v", fqKey, m) + result[key] = m } else { logger.Debugf("Found real value for %s setting to %T %v", fqKey, val, val) result[key] = val @@ -56,6 +61,21 @@ func getKeysRecursively(base string, v *viper.Viper, nodeKeys map[string]interfa return result } +func unmarshalJson(val interface{}) (map[string]string, bool) { + mp := map[string]string{} + s, ok := val.(string) + if !ok { + logger.Debugf("Unmarshal JSON: value is not a string: %v", val) + return nil, false + } + err := json.Unmarshal([]byte(s), &mp) + if err != nil { + logger.Debugf("Unmarshal JSON: value cannot be unmarshalled: %s", err) + return nil, false + } + return mp, true +} + // customDecodeHook adds the additional functions of parsing durations from strings // as well as parsing strings of the format "[thing1, thing2, thing3]" into string slices // Note that whitespace around slice elements is removed diff --git a/orderer/main.go b/orderer/main.go index 3884879bd67..89c5d42f587 100644 --- a/orderer/main.go +++ b/orderer/main.go @@ -35,6 +35,10 @@ import ( ramledger "github.com/hyperledger/fabric/orderer/ledger/ram" "github.com/hyperledger/fabric/orderer/localconfig" "github.com/hyperledger/fabric/orderer/multichain" + "github.com/hyperledger/fabric/orderer/sbft" + "github.com/hyperledger/fabric/orderer/sbft/backend" + sbftcrypto "github.com/hyperledger/fabric/orderer/sbft/crypto" + "github.com/hyperledger/fabric/orderer/sbft/simplebft" "github.com/hyperledger/fabric/orderer/solo" cb "github.com/hyperledger/fabric/protos/common" ab "github.com/hyperledger/fabric/protos/orderer" @@ -144,6 +148,7 @@ func main() { consenters := make(map[string]multichain.Consenter) consenters["solo"] = solo.New() consenters["kafka"] = kafka.New(conf.Kafka.Version, conf.Kafka.Retry) + consenters["sbft"] = sbft.New(makeSbftConsensusConfig(conf), makeSbftStackConfig(conf)) manager := multichain.NewManagerImpl(lf, consenters, localmsp.NewSigner()) @@ -157,3 +162,21 @@ func main() { logger.Infof("Beginning to serve requests") grpcServer.Start() } + +func makeSbftConsensusConfig(conf *config.TopLevel) *sbft.ConsensusConfig { + cfg := simplebft.Config{N: conf.Sbft.N, F: conf.Sbft.F, BatchDurationNsec: conf.Sbft.BatchDurationNsec, + BatchSizeBytes: conf.Sbft.BatchSizeBytes, + RequestTimeoutNsec: conf.Sbft.RequestTimeoutNsec} + peers := make(map[string][]byte) + for addr, cert := range conf.Sbft.Peers { + peers[addr], _ = sbftcrypto.ParseCertPEM(cert) + } + return &sbft.ConsensusConfig{Consensus: &cfg, Peers: peers} +} + +func makeSbftStackConfig(conf *config.TopLevel) *backend.StackConfig { + return &backend.StackConfig{ListenAddr: conf.Sbft.PeerCommAddr, + CertFile: conf.Sbft.CertFile, + KeyFile: conf.Sbft.KeyFile, + DataDir: conf.Sbft.DataDir} +} diff --git a/orderer/mocks/multichain/multichain.go b/orderer/mocks/multichain/multichain.go index 7fffd9630b4..10afd81a6dc 100644 --- a/orderer/mocks/multichain/multichain.go +++ b/orderer/mocks/multichain/multichain.go @@ -83,7 +83,9 @@ func (mcs *ConsenterSupport) WriteBlock(block *cb.Block, _committers []filter.Co umtxs[i] = utils.UnmarshalEnvelopeOrPanic(block.Data.Data[i]) } mcs.Batches <- umtxs - block.Metadata.Metadata[cb.BlockMetadataIndex_ORDERER] = utils.MarshalOrPanic(&cb.Metadata{Value: encodedMetadataValue}) + if encodedMetadataValue != nil { + block.Metadata.Metadata[cb.BlockMetadataIndex_ORDERER] = utils.MarshalOrPanic(&cb.Metadata{Value: encodedMetadataValue}) + } mcs.WriteBlockVal = block return block } diff --git a/orderer/sbft/main/network_test.go b/orderer/network_test.go similarity index 73% rename from orderer/sbft/main/network_test.go rename to orderer/network_test.go index dc4251ab619..ae3db2c1327 100644 --- a/orderer/sbft/main/network_test.go +++ b/orderer/network_test.go @@ -20,7 +20,6 @@ import ( "crypto/rand" "crypto/x509" "crypto/x509/pkix" - "encoding/json" "encoding/pem" "fmt" "io/ioutil" @@ -31,20 +30,35 @@ import ( "testing" "time" + "encoding/json" + "github.com/golang/protobuf/proto" "github.com/hyperledger/fabric/orderer/common/bootstrap/provisional" - pb "github.com/hyperledger/fabric/orderer/sbft/simplebft" cb "github.com/hyperledger/fabric/protos/common" ab "github.com/hyperledger/fabric/protos/orderer" "github.com/hyperledger/fabric/protos/utils" + "github.com/op/go-logging" "golang.org/x/net/context" "google.golang.org/grpc" ) -const keyfile = "testdata/key.pem" -const maindir = "github.com/hyperledger/fabric/orderer/sbft/main" - -var mainexe = os.TempDir() + "/" + "sbft" +const keyfile = "sbft/testdata/key.pem" +const maindir = "github.com/hyperledger/fabric/orderer" + +var ordererDir string +var mainexe string + +type flags struct { + listenAddr string + grpcAddr string + telemetryAddr string + certFile string + keyFile string + dataDir string + genesisFile string + verbose string + init string +} type Peer struct { id uint64 @@ -77,6 +91,13 @@ func deleteExe() { } func TestMain(m *testing.M) { + var err error + ordererDir, err = os.Getwd() + if err != nil { + panic(err) + } + mainexe = os.TempDir() + "/" + "orderer" + build() code := m.Run() deleteExe() @@ -89,6 +110,30 @@ func TestTwoReplicasBroadcastAndDeliverUsingTheSame(t *testing.T) { skipInShortMode(t) peers := InitPeers(2, startingPort) StartPeers(peers) + r, err := Receive(peers[0], startingPort) + defer r.Stop() + defer StopPeers(peers) + if err != nil { + t.Errorf("Failed to start up receiver: %s", err) + } + WaitForConnection(peers) + if berr := Broadcast(peers[0], startingPort, []byte{0, 1, 2, 3, 4}); berr != nil { + t.Errorf("Failed to broadcast message: %s", berr) + } + if !AssertWithTimeout(func() bool { + return r.Received() == 2 + }, 30) { + t.Errorf("Failed to receive some messages. (Received %d)", r.Received()) + } +} + +func TestTwoReplicasBroadcastAndDeliverUsingDifferent(t *testing.T) { + t.Parallel() + logging.SetLevel(logging.DEBUG, "sbft") + startingPort := 2500 + skipInShortMode(t) + peers := InitPeers(2, startingPort) + StartPeers(peers) r, err := Receive(peers[1], startingPort) defer r.Stop() defer StopPeers(peers) @@ -227,12 +272,18 @@ func TestTenReplicasBombedWithBroadcastsIfLedgersConsistent(t *testing.T) { func InitPeers(num uint64, startingPort int) []*Peer { peers := make([]*Peer, 0, num) certFiles := make([]string, 0, num) + peersWithCerts := map[string]string{} for i := uint64(0); i < num; i++ { - certFiles = append(certFiles, generateCertificate(i, keyfile)) + certFile := generateCertificate(i, keyfile) + certFiles = append(certFiles, certFile) + peerCommPort := listenAddress(i, startingPort) + peersWithCerts[peerCommPort] = certFile } - configFile := generateConfig(num, startingPort, certFiles) for i := uint64(0); i < num; i++ { - peers = append(peers, initPeer(i, startingPort, configFile, certFiles[i])) + peerCommPort := listenAddress(i, startingPort) + grpcPort := grpcPort(i, startingPort) + configEnv := generateConfigEnv(num, grpcPort, peerCommPort, certFiles[i], peersWithCerts) + peers = append(peers, initPeer(i, configEnv)) } return peers } @@ -249,51 +300,34 @@ func StopPeers(peers []*Peer) { } } -func generateConfig(peerNum uint64, startingPort int, certFiles []string) string { +func generateConfigEnv(peerNum uint64, grpcPort int, peerCommPort string, certFile string, peersWithCerts map[string]string) []string { tempDir, err := ioutil.TempDir("", "sbft_test_config") panicOnError(err) - c := pb.Config{ - N: peerNum, - F: (peerNum - 1) / 3, - BatchDurationNsec: 1000, - BatchSizeBytes: 1000000000, - RequestTimeoutNsec: 1000000000} - peerconfigs := make([]map[string]string, 0, peerNum) - for i := uint64(0); i < peerNum; i++ { - pc := make(map[string]string) - pc["Id"] = fmt.Sprintf("%d", i) - pc["Address"] = listenAddress(i, startingPort) - pc["Cert"] = certFiles[i] - peerconfigs = append(peerconfigs, pc) - } - consconfig := make(map[string]interface{}) - consconfig["consensus"] = c - consconfig["peers"] = peerconfigs - stringconf, err := json.Marshal(consconfig) - panicOnError(err) - conffilepath := tempDir + "/jsonconfig" - ioutil.WriteFile(conffilepath, []byte(stringconf), 0644) - return conffilepath + envs := []string{} + envs = append(envs, fmt.Sprintf("ORDERER_CFG_PATH=%s", ordererDir)) + envs = append(envs, fmt.Sprintf("ORDERER_GENESIS_ORDERERTYPE=%s", "sbft")) + envs = append(envs, fmt.Sprintf("ORDERER_GENERAL_LISTENPORT=%d", grpcPort)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_PEERCOMMADDR=%s", peerCommPort)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_CERTFILE=%s", certFile)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_KEYFILE=%s", keyfile)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_DATADIR=%s", tempDir)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_BATCHDURATIONNSEC=%d", 1000)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_BATCHSIZEBYTES=%d", 1000000000)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_REQUESTTIMEOUTNSEC=%d", 1000000000)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_N=%d", peerNum)) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_F=%d", (peerNum-1)/3)) + js, _ := json.Marshal(peersWithCerts) + envs = append(envs, fmt.Sprintf("ORDERER_SBFT_PEERS=%s", js)) + return envs } -func initPeer(uid uint64, startingPort int, configFile string, certFile string) (p *Peer) { - tempDir, err := ioutil.TempDir("", "sbft_test") - panicOnError(err) - os.RemoveAll(tempDir) - c := flags{init: configFile, - listenAddr: listenAddress(uid, startingPort), - grpcAddr: grpcAddress(uid, startingPort), - certFile: certFile, - keyFile: keyfile, - dataDir: tempDir} +func initPeer(uid uint64, configEnv []string) (p *Peer) { ctx, cancel := context.WithCancel(context.Background()) - p = &Peer{id: uid, cancel: cancel, config: c} - err = initInstance(c) - panicOnError(err) - p.cmd = exec.CommandContext(ctx, mainexe, "-addr", p.config.listenAddr, "-gaddr", p.config.grpcAddr, "-cert", p.config.certFile, "-key", - p.config.keyFile, "-data-dir", p.config.dataDir, "-verbose", "debug") + p = &Peer{id: uid, cancel: cancel} + p.cmd = exec.CommandContext(ctx, mainexe) p.cmd.Stdout = os.Stdout p.cmd.Stderr = os.Stderr + p.cmd.Env = append(configEnv, os.Environ()...) return } @@ -308,10 +342,12 @@ func (p *Peer) stop() { } func Broadcast(p *Peer, startingPort int, bytes []byte) error { - timeout := 10 * time.Second + timeout := 5 * time.Second grpcAddress := grpcAddress(p.id, startingPort) + logger.Warningf("Broadcast - dialing: %s", grpcAddress) clientconn, err := grpc.Dial(grpcAddress, grpc.WithBlock(), grpc.WithTimeout(timeout), grpc.WithInsecure()) if err != nil { + logger.Warning("Broadcast - failure") return err } defer clientconn.Close() @@ -320,14 +356,17 @@ func Broadcast(p *Peer, startingPort int, bytes []byte) error { if err != nil { return err } - pl := &cb.Payload{Data: bytes} + h := &cb.Header{ChainHeader: &cb.ChainHeader{ChainID: provisional.TestChainID}, SignatureHeader: &cb.SignatureHeader{}} + pl := &cb.Payload{Data: bytes, Header: h} mpl, err := proto.Marshal(pl) panicOnError(err) + logger.Warningf("Broadcast - sending: %s", grpcAddress) if e := bstream.Send(&cb.Envelope{Payload: mpl}); e != nil { return e } _, err = bstream.Recv() panicOnError(err) + logger.Warningf("Broadcast - done: %s", grpcAddress) return nil } @@ -336,6 +375,7 @@ func Receive(p *Peer, startingPort int) (*Receiver, error) { signals := make(chan bool, 100) timeout := 4 * time.Second grpcAddress := grpcAddress(p.id, startingPort) + logger.Warning("Receiver - dial") clientconn, err := grpc.Dial(grpcAddress, grpc.WithBlock(), grpc.WithTimeout(timeout), grpc.WithInsecure()) if err != nil { return nil, err @@ -375,13 +415,17 @@ func Receive(p *Peer, startingPort int) (*Receiver, error) { clientconn.Close() return } - b := m.Type.(*ab.DeliverResponse_Block) + b, ok := m.Type.(*ab.DeliverResponse_Block) + if !ok { + continue + } for _, tx := range b.Block.Data.Data { pl := &cb.Payload{} e := &cb.Envelope{} merr1 := proto.Unmarshal(tx, e) merr2 := proto.Unmarshal(e.Payload, pl) if merr1 == nil && merr2 == nil { + logger.Warning("Receiver - received a message") retch <- tx num++ } @@ -418,7 +462,11 @@ func listenAddress(id uint64, startingPort int) string { } func grpcAddress(id uint64, startingPort int) string { - return fmt.Sprintf(":%d", startingPort+1+2*int(id)) + return fmt.Sprintf(":%d", grpcPort(id, startingPort)) +} + +func grpcPort(id uint64, startingPort int) int { + return startingPort + 1 + 2*int(id) } func generateCertificate(id uint64, keyFile string) string { diff --git a/orderer/orderer.yaml b/orderer/orderer.yaml index 5c0ea88f21d..c914f4d6a99 100644 --- a/orderer/orderer.yaml +++ b/orderer/orderer.yaml @@ -113,6 +113,32 @@ Kafka: # NOTE: Use IP:port notation Brokers: - 127.0.0.1:9092 + +################################################################################ +# +# SECTION: Sbft +# +# - This section applies to the configuration of the Sbft-backed orderer +# +################################################################################ +Sbft: + # Address to use for SBFT internal communication + PeerCommAddr: ":6101" + CertFile: "sbft/testdata/cert1.pem" + KeyFile: "sbft/testdata/key.pem" + # Directory for SBFT data (persistence) + DataDir: "/tmp" + # Number of peers + "N": 1 + # Fault tolerance + F: 0 + BatchDurationNsec: 1000 + BatchSizeBytes: 1000000000 + RequestTimeoutNsec: 1000000000 + # Peers (PeerCommAddr) with the path of their cert + Peers: + ":6101": "sbft/testdata/cert1.pem" + ################################################################################ # # SECTION: Genesis diff --git a/orderer/sbft/backend/backend.go b/orderer/sbft/backend/backend.go index edc4a1e846a..6fdf252dbd7 100644 --- a/orderer/sbft/backend/backend.go +++ b/orderer/sbft/backend/backend.go @@ -40,7 +40,8 @@ import ( "encoding/gob" "github.com/golang/protobuf/proto" - ordererledger "github.com/hyperledger/fabric/orderer/ledger" + commonfilter "github.com/hyperledger/fabric/orderer/common/filter" + "github.com/hyperledger/fabric/orderer/multichain" "github.com/hyperledger/fabric/orderer/sbft/connection" "github.com/hyperledger/fabric/orderer/sbft/persist" s "github.com/hyperledger/fabric/orderer/sbft/simplebft" @@ -55,25 +56,31 @@ const metadataLen = 2 var logger = logging.MustGetLogger("backend") type Backend struct { - consensus s.Receiver - conn *connection.Manager - - lock sync.Mutex - peers map[uint64]chan<- *s.Msg + conn *connection.Manager + lock sync.Mutex + peers map[uint64]chan<- *s.MultiChainMsg + queue chan Executable + persistence *persist.Persist - self *PeerInfo + self *PeerInfo + // address to PeerInfo mapping peerInfo map[string]*PeerInfo - queue chan Executable - - persistence *persist.Persist - ledger ordererledger.ReadWriter - - lastBatch *s.Batch + // chainId to instance mapping + consensus map[string]s.Receiver + lastBatches map[string]*s.Batch + supports map[string]multichain.ConsenterSupport } type consensusConn Backend +type StackConfig struct { + ListenAddr string + CertFile string + KeyFile string + DataDir string +} + type PeerInfo struct { info connection.PeerInfo id uint64 @@ -93,12 +100,14 @@ func (pi peerInfoSlice) Swap(i, j int) { pi[i], pi[j] = pi[j], pi[i] } -func NewBackend(peers map[string][]byte, conn *connection.Manager, rl ordererledger.ReadWriter, persist *persist.Persist) (*Backend, error) { +func NewBackend(peers map[string][]byte, conn *connection.Manager, persist *persist.Persist) (*Backend, error) { c := &Backend{ - conn: conn, - peers: make(map[uint64]chan<- *s.Msg), - peerInfo: make(map[string]*PeerInfo), - ledger: rl, + conn: conn, + peers: make(map[uint64]chan<- *s.MultiChainMsg), + peerInfo: make(map[string]*PeerInfo), + supports: make(map[string]multichain.ConsenterSupport), + consensus: make(map[string]s.Receiver), + lastBatches: make(map[string]*s.Batch), } var peerInfo []*PeerInfo @@ -135,17 +144,27 @@ func NewBackend(peers map[string][]byte, conn *connection.Manager, rl ordererled } RegisterConsensusServer(conn.Server, (*consensusConn)(c)) c.persistence = persist - c.lastBatch = &s.Batch{Header: nil, Signatures: nil, Payloads: [][]byte{}} c.queue = make(chan Executable) go c.run() return c, nil } -func (c *Backend) GetMyId() uint64 { - return c.self.id +// GetMyId returns the ID of the backend in the SFTT network (1..N) +func (b *Backend) GetMyId() uint64 { + return b.self.id +} + +// Enqueue enqueues an Envelope for a chainId for ordering, marshalling it first +func (b *Backend) Enqueue(chainID string, env *cb.Envelope) bool { + requestbytes, err := proto.Marshal(env) + if err != nil { + return false + } + b.enqueueRequest(chainID, requestbytes) + return true } -func (c *Backend) connectWorker(peer *PeerInfo) { +func (b *Backend) connectWorker(peer *PeerInfo) { timeout := 1 * time.Second delay := time.After(0) @@ -157,7 +176,7 @@ func (c *Backend) connectWorker(peer *PeerInfo) { delay = time.After(timeout) logger.Infof("connecting to replica %d (%s)", peer.id, peer.info) - conn, err := c.conn.DialPeer(peer.info, grpc.WithBlock(), grpc.WithTimeout(timeout)) + conn, err := b.conn.DialPeer(peer.info, grpc.WithBlock(), grpc.WithTimeout(timeout)) if err != nil { logger.Warningf("could not connect to replica %d (%s): %s", peer.id, peer.info, err) continue @@ -182,26 +201,26 @@ func (c *Backend) connectWorker(peer *PeerInfo) { logger.Warningf("consensus stream with replica %d (%s) broke: %v", peer.id, peer.info, err) break } - c.enqueueForReceive(msg, peer.id) + b.enqueueForReceive(msg.ChainID, msg.Msg, peer.id) } } } -func (b *Backend) enqueueConnection(peerid uint64) { +func (b *Backend) enqueueConnection(chainID string, peerid uint64) { go func() { - b.queue <- &connectionEvent{peerid: peerid} + b.queue <- &connectionEvent{chainID: chainID, peerid: peerid} }() } -func (b *Backend) enqueueRequest(request []byte) { +func (b *Backend) enqueueRequest(chainID string, request []byte) { go func() { - b.queue <- &requestEvent{req: request} + b.queue <- &requestEvent{chainId: chainID, req: request} }() } -func (b *Backend) enqueueForReceive(msg *s.Msg, src uint64) { +func (b *Backend) enqueueForReceive(chainID string, msg *s.Msg, src uint64) { go func() { - b.queue <- &msgEvent{msg: msg, src: src} + b.queue <- &msgEvent{chainId: chainID, msg: msg, src: src} }() } @@ -221,7 +240,13 @@ func (b *Backend) run() { } } -// gRPC interface +// AddSbftPeer adds a new SBFT peer for the given chainId using the given support and configuration +func (b *Backend) AddSbftPeer(chainID string, support multichain.ConsenterSupport, config *s.Config) (*s.SBFT, error) { + b.supports[chainID] = support + return s.New(b.GetMyId(), chainID, config, b) +} + +// Consensus implements the SBFT consensus gRPC interface func (c *consensusConn) Consensus(_ *Handshake, srv Consensus_ConsensusServer) error { pi := connection.GetPeerInfo(srv) peer, ok := c.peerInfo[pi.Fingerprint()] @@ -232,7 +257,7 @@ func (c *consensusConn) Consensus(_ *Handshake, srv Consensus_ConsensusServer) e } logger.Infof("connection from replica %d (%s)", peer.id, pi) - ch := make(chan *s.Msg) + ch := make(chan *s.MultiChainMsg) c.lock.Lock() if oldch, ok := c.peers[peer.id]; ok { logger.Debugf("replacing connection from replica %d", peer.id) @@ -240,7 +265,10 @@ func (c *consensusConn) Consensus(_ *Handshake, srv Consensus_ConsensusServer) e } c.peers[peer.id] = ch c.lock.Unlock() - ((*Backend)(c)).enqueueConnection(peer.id) + + for chainID, _ := range c.supports { + ((*Backend)(c)).enqueueConnection(chainID, peer.id) + } var err error for msg := range ch { @@ -257,49 +285,55 @@ func (c *consensusConn) Consensus(_ *Handshake, srv Consensus_ConsensusServer) e return err } -func (c *Backend) Broadcast(msg *s.Msg) error { - c.lock.Lock() - for _, ch := range c.peers { +// Unicast sends to all external SBFT peers +func (b *Backend) Broadcast(msg *s.MultiChainMsg) error { + b.lock.Lock() + for _, ch := range b.peers { ch <- msg } - c.lock.Unlock() + b.lock.Unlock() return nil } -func (c *Backend) Unicast(msg *s.Msg, dest uint64) error { - c.lock.Lock() - ch, ok := c.peers[dest] - c.lock.Unlock() +// Unicast sends to a specific external SBFT peer identified by chainId and dest +func (b *Backend) Unicast(chainID string, msg *s.Msg, dest uint64) error { + b.lock.Lock() + ch, ok := b.peers[dest] + b.lock.Unlock() if !ok { err := fmt.Errorf("peer not found: %v", dest) logger.Debug(err) return err } - ch <- msg + ch <- &s.MultiChainMsg{Msg: msg, ChainID: chainID} return nil } -func (t *Backend) SetReceiver(recv s.Receiver) { - t.consensus = recv +// AddReceiver adds a receiver instance for a given chainId +func (b *Backend) AddReceiver(chainId string, recv s.Receiver) { + b.consensus[chainId] = recv + b.lastBatches[chainId] = &s.Batch{Header: nil, Signatures: nil, Payloads: [][]byte{}} } -func (t *Backend) Send(msg *s.Msg, dest uint64) { - if dest == t.self.id { - t.enqueueForReceive(msg, t.self.id) +// Send sends to a specific SBFT peer identified by chainId and dest +func (b *Backend) Send(chainID string, msg *s.Msg, dest uint64) { + if dest == b.self.id { + b.enqueueForReceive(chainID, msg, b.self.id) return } - t.Unicast(msg, dest) + b.Unicast(chainID, msg, dest) } -func (t *Backend) Timer(d time.Duration, tf func()) s.Canceller { +// Timer starts a timer +func (b *Backend) Timer(d time.Duration, tf func()) s.Canceller { tm := &Timer{tf: tf, execute: true} - t.initTimer(tm, d) + b.initTimer(tm, d) return tm } -// Deliver writes the ledger -func (t *Backend) Deliver(batch *s.Batch) { +// Deliver writes a block +func (b *Backend) Deliver(chainId string, batch *s.Batch) { blockContents := make([]*cb.Envelope, 0, len(batch.Payloads)) for _, p := range batch.Payloads { envelope := &cb.Envelope{} @@ -310,29 +344,38 @@ func (t *Backend) Deliver(batch *s.Batch) { logger.Warningf("Payload cannot be unmarshalled.") } } - block := ordererledger.CreateNextBlock(t.ledger, blockContents) - // This a quick and dirty solution to make it work. - // SBFT needs to use Ordererledger's structures and signatures over the Block. - block.Metadata.Metadata[headerIndex] = batch.Header - block.Metadata.Metadata[signaturesIndex] = encodeSignatures(batch.Signatures) - t.lastBatch = batch - t.ledger.Append(block) -} + block := b.supports[chainId].CreateNextBlock(blockContents) -func (t *Backend) Persist(key string, data proto.Message) { + // TODO SBFT needs to use Rawledger's structures and signatures over the Block. + // This a quick and dirty solution to make it work. + block.Metadata = &cb.BlockMetadata{} + metadata := make([][]byte, metadataLen) + metadata[headerIndex] = batch.Header + metadata[signaturesIndex] = encodeSignatures(batch.Signatures) + block.Metadata.Metadata = metadata + b.lastBatches[chainId] = batch + committers := []commonfilter.Committer{} + b.supports[chainId].WriteBlock(block, committers, nil) +} + +// Persist persists data identified by a chainId and a key +func (b *Backend) Persist(chainId string, key string, data proto.Message) { + compk := fmt.Sprintf("chain-%s-%s", chainId, key) if data == nil { - t.persistence.DelState(key) + b.persistence.DelState(compk) } else { bytes, err := proto.Marshal(data) if err != nil { panic(err) } - t.persistence.StoreState(key, bytes) + b.persistence.StoreState(compk, bytes) } } -func (t *Backend) Restore(key string, out proto.Message) bool { - val, err := t.persistence.ReadState(key) +// Restore loads persisted data identified by chainId and key +func (b *Backend) Restore(chainId string, key string, out proto.Message) bool { + compk := fmt.Sprintf("chain-%s-%s", chainId, key) + val, err := b.persistence.ReadState(compk) if err != nil { return false } @@ -340,26 +383,31 @@ func (t *Backend) Restore(key string, out proto.Message) bool { return (err == nil) } -func (t *Backend) LastBatch() *s.Batch { - return t.lastBatch +// LastBatch returns the last batch for a given chain identified by its ID +func (b *Backend) LastBatch(chainId string) *s.Batch { + return b.lastBatches[chainId] } -func (t *Backend) Sign(data []byte) []byte { - return Sign(t.conn.Cert.PrivateKey, data) +// Sign signs a given data +func (b *Backend) Sign(data []byte) []byte { + return Sign(b.conn.Cert.PrivateKey, data) } -func (t *Backend) CheckSig(data []byte, src uint64, sig []byte) error { - leaf := t.conn.Cert.Leaf +// CheckSig checks a signature +func (b *Backend) CheckSig(data []byte, src uint64, sig []byte) error { + leaf := b.conn.Cert.Leaf if leaf == nil { panic("No public key found: certificate leaf is nil.") } return CheckSig(leaf.PublicKey, data, sig) } -func (t *Backend) Reconnect(replica uint64) { - t.enqueueConnection(replica) +// Reconnect requests connection to a replica identified by its ID and chainId +func (b *Backend) Reconnect(chainId string, replica uint64) { + b.enqueueConnection(chainId, replica) } +// Sign signs a given data func Sign(privateKey crypto.PrivateKey, data []byte) []byte { var err error var encsig []byte @@ -385,6 +433,7 @@ func Sign(privateKey crypto.PrivateKey, data []byte) []byte { return encsig } +// CheckSig checks a signature func CheckSig(publicKey crypto.PublicKey, data []byte, sig []byte) error { hash := sha256.Sum256(data) switch p := publicKey.(type) { @@ -410,20 +459,6 @@ func CheckSig(publicKey crypto.PublicKey, data []byte, sig []byte) error { } } -func getHeader(metadata *cb.BlockMetadata) []byte { - if metadata == nil || len(metadata.Metadata) < metadataLen { - return nil - } - return metadata.Metadata[headerIndex] -} - -func getEncodedSignatures(metadata *cb.BlockMetadata) []byte { - if metadata == nil || len(metadata.Metadata) < metadataLen { - return nil - } - return metadata.Metadata[signaturesIndex] -} - func encodeSignatures(signatures map[uint64][]byte) []byte { var buf bytes.Buffer enc := gob.NewEncoder(&buf) diff --git a/orderer/sbft/backend/backend_test.go b/orderer/sbft/backend/backend_test.go index 646fff28b54..e8b90bea7ff 100644 --- a/orderer/sbft/backend/backend_test.go +++ b/orderer/sbft/backend/backend_test.go @@ -25,9 +25,8 @@ import ( "testing" "github.com/golang/protobuf/proto" - "github.com/hyperledger/fabric/orderer/common/bootstrap/provisional" - ramledger "github.com/hyperledger/fabric/orderer/ledger/ram" - localconfig "github.com/hyperledger/fabric/orderer/localconfig" + "github.com/hyperledger/fabric/orderer/mocks/multichain" + mc "github.com/hyperledger/fabric/orderer/multichain" "github.com/hyperledger/fabric/orderer/sbft/simplebft" cb "github.com/hyperledger/fabric/protos/common" ) @@ -69,13 +68,14 @@ func TestSignAndVerifyEcdsa(t *testing.T) { } func TestLedgerReadWrite(t *testing.T) { - localConf := localconfig.Load() - localConf.Genesis.OrdererType = provisional.ConsensusTypeSbft - genesis := provisional.New(localConf).GenesisBlock() - rlf := ramledger.New(10) - rl, _ := rlf.GetOrCreate(provisional.TestChainID) - rl.Append(genesis) - b := Backend{ledger: rl} + testChainID1 := "testID1" + testChainID2 := "testID2" + testChainID3 := "testID2" + b := Backend{supports: map[string]mc.ConsenterSupport{}, lastBatches: map[string]*simplebft.Batch{}} + + b.supports[testChainID1] = &multichain.ConsenterSupport{Batches: make(chan []*cb.Envelope, 10)} + b.supports[testChainID2] = &multichain.ConsenterSupport{Batches: make(chan []*cb.Envelope, 10)} + b.supports[testChainID3] = &multichain.ConsenterSupport{Batches: make(chan []*cb.Envelope, 10)} header := []byte("header") e1 := &cb.Envelope{Payload: []byte("data1")} @@ -88,11 +88,20 @@ func TestLedgerReadWrite(t *testing.T) { sgns[uint64(22)] = []byte("sgn22") batch := simplebft.Batch{Header: header, Payloads: data, Signatures: sgns} - b.Deliver(&batch) - batch2 := b.LastBatch() + b.Deliver(testChainID1, &batch) + batch1 := b.LastBatch(testChainID1) + batch2 := b.LastBatch(testChainID2) + b.Deliver(testChainID3, &batch) + batch3 := b.LastBatch(testChainID3) - if !reflect.DeepEqual(batch, *batch2) { - t.Errorf("The wrong batch was returned by LastBatch after Deliver: %v (original was: %v)", batch2, &batch) + if !reflect.DeepEqual(batch, *batch1) { + t.Errorf("The wrong batch was returned by LastBatch(chainID1) after Deliver: %v (original was: %v)", batch1, &batch) + } + if !reflect.DeepEqual(batch, *batch3) { + t.Errorf("The wrong batch was returned by LastBatch(chainID3) after Deliver: %v (original was: %v)", batch3, &batch) + } + if batch2 != nil { + t.Error("The second chain should be empty.") } } diff --git a/orderer/sbft/backend/backendab.go b/orderer/sbft/backend/backendab.go deleted file mode 100644 index 3b59c6767d1..00000000000 --- a/orderer/sbft/backend/backendab.go +++ /dev/null @@ -1,136 +0,0 @@ -/* -Copyright Digital Asset Holdings, LLC 2016 All Rights Reserved. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package backend - -import ( - "github.com/hyperledger/fabric/common/policies" - "github.com/hyperledger/fabric/orderer/common/deliver" - "github.com/hyperledger/fabric/orderer/common/sharedconfig" - ordererledger "github.com/hyperledger/fabric/orderer/ledger" - mockpolicies "github.com/hyperledger/fabric/orderer/mocks/policies" - mocksharedconfig "github.com/hyperledger/fabric/orderer/mocks/sharedconfig" - cb "github.com/hyperledger/fabric/protos/common" - ab "github.com/hyperledger/fabric/protos/orderer" - - "github.com/golang/protobuf/proto" -) - -type xxxSupportManager struct { - chainID string - support *xxxSupport -} - -func (xxx *xxxSupportManager) GetChain(id string) (deliver.Support, bool) { - if id != xxx.chainID { - return nil, false - } - return xxx.support, true -} - -type xxxSupport struct { - reader ordererledger.Reader - sharedConfig sharedconfig.Manager - policyManager policies.Manager -} - -func (xxx *xxxSupport) PolicyManager() policies.Manager { - return xxx.policyManager -} - -func (xxx *xxxSupport) SharedConfig() sharedconfig.Manager { - return xxx.sharedConfig -} - -func (xxx *xxxSupport) Reader() ordererledger.Reader { - return xxx.reader -} - -type BackendAB struct { - backend *Backend - deliverserver deliver.Handler -} - -func NewBackendAB(backend *Backend) *BackendAB { - - // XXX All the code below is a hacky shim until sbft can be adapter to the new multichain interface - it, _ := backend.ledger.Iterator(&ab.SeekPosition{Type: &ab.SeekPosition_Oldest{}}) - block, status := it.Next() - if status != cb.Status_SUCCESS { - panic("Error getting a block from the ledger") - } - env := &cb.Envelope{} - err := proto.Unmarshal(block.Data.Data[0], env) - if err != nil { - panic(err) - } - - payload := &cb.Payload{} - err = proto.Unmarshal(env.Payload, payload) - if err != nil { - panic(err) - } - - manager := &xxxSupportManager{ - chainID: payload.Header.ChainHeader.ChainID, - support: &xxxSupport{ - reader: backend.ledger, - sharedConfig: &mocksharedconfig.Manager{}, - policyManager: &mockpolicies.Manager{Policy: &mockpolicies.Policy{}}, - }, - } - - manager.support.sharedConfig.(*mocksharedconfig.Manager).EgressPolicyNamesVal = []string{"XXX"} - manager.support.sharedConfig.(*mocksharedconfig.Manager).IngressPolicyNamesVal = []string{"XXX"} - // XXX End hackiness - - bab := &BackendAB{ - backend: backend, - deliverserver: deliver.NewHandlerImpl(manager), - } - return bab -} - -// Broadcast receives a stream of messages from a client for ordering -func (b *BackendAB) Broadcast(srv ab.AtomicBroadcast_BroadcastServer) error { - for { - envelope, err := srv.Recv() - if err != nil { - return err - } - - if envelope.Payload == nil { - err = srv.Send(&ab.BroadcastResponse{Status: cb.Status_BAD_REQUEST}) - if err != nil { - return err - } - } - req, err := proto.Marshal(envelope) - if err != nil { - panic(err) - } - b.backend.enqueueRequest(req) - err = srv.Send(&ab.BroadcastResponse{Status: cb.Status_SUCCESS}) - if err != nil { - return err - } - } -} - -// Deliver sends a stream of blocks to a client after ordering -func (b *BackendAB) Deliver(srv ab.AtomicBroadcast_DeliverServer) error { - return b.deliverserver.Handle(srv) -} diff --git a/orderer/sbft/backend/consensus.pb.go b/orderer/sbft/backend/consensus.pb.go index b2bad42309b..b55f36f7d93 100644 --- a/orderer/sbft/backend/consensus.pb.go +++ b/orderer/sbft/backend/consensus.pb.go @@ -84,7 +84,7 @@ func (c *consensusClient) Consensus(ctx context.Context, in *Handshake, opts ... } type Consensus_ConsensusClient interface { - Recv() (*simplebft.Msg, error) + Recv() (*simplebft.MultiChainMsg, error) grpc.ClientStream } @@ -92,8 +92,8 @@ type consensusConsensusClient struct { grpc.ClientStream } -func (x *consensusConsensusClient) Recv() (*simplebft.Msg, error) { - m := new(simplebft.Msg) +func (x *consensusConsensusClient) Recv() (*simplebft.MultiChainMsg, error) { + m := new(simplebft.MultiChainMsg) if err := x.ClientStream.RecvMsg(m); err != nil { return nil, err } @@ -119,7 +119,7 @@ func _Consensus_Consensus_Handler(srv interface{}, stream grpc.ServerStream) err } type Consensus_ConsensusServer interface { - Send(*simplebft.Msg) error + Send(*simplebft.MultiChainMsg) error grpc.ServerStream } @@ -127,7 +127,7 @@ type consensusConsensusServer struct { grpc.ServerStream } -func (x *consensusConsensusServer) Send(m *simplebft.Msg) error { +func (x *consensusConsensusServer) Send(m *simplebft.MultiChainMsg) error { return x.ServerStream.SendMsg(m) } @@ -148,16 +148,16 @@ var _Consensus_serviceDesc = grpc.ServiceDesc{ func init() { proto.RegisterFile("backend/consensus.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 164 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0x12, 0x4f, 0x4a, 0x4c, 0xce, - 0x4e, 0xcd, 0x4b, 0xd1, 0x4f, 0xce, 0xcf, 0x2b, 0x4e, 0xcd, 0x2b, 0x2e, 0x2d, 0xd6, 0x2b, 0x28, - 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x87, 0x4a, 0x48, 0x49, 0x16, 0x67, 0xe6, 0x16, 0xe4, 0xa4, 0x26, - 0xa5, 0x95, 0xe8, 0xc3, 0x59, 0x10, 0x35, 0x4a, 0xdc, 0x5c, 0x9c, 0x19, 0x89, 0x79, 0x29, 0xc5, - 0x19, 0x89, 0xd9, 0xa9, 0x46, 0x0e, 0x5c, 0x9c, 0x70, 0x33, 0x84, 0x8c, 0x91, 0x39, 0x42, 0x7a, - 0x50, 0xb3, 0xf4, 0xe0, 0xaa, 0xa5, 0xf8, 0xf4, 0x10, 0x86, 0xf9, 0x16, 0xa7, 0x2b, 0x31, 0x18, - 0x30, 0x3a, 0x99, 0x44, 0x19, 0xa5, 0x67, 0x96, 0x64, 0x94, 0x26, 0xe9, 0x25, 0xe7, 0xe7, 0xea, - 0x67, 0x54, 0x16, 0xa4, 0x16, 0xe5, 0xa4, 0xa6, 0xa4, 0xa7, 0x16, 0xe9, 0xa7, 0x25, 0x26, 0x15, - 0x65, 0x26, 0xeb, 0xe7, 0x17, 0xa5, 0xa4, 0x16, 0xa5, 0x16, 0xe9, 0x17, 0x83, 0x1c, 0x03, 0x35, - 0x33, 0x89, 0x0d, 0xec, 0x16, 0x63, 0x40, 0x00, 0x00, 0x00, 0xff, 0xff, 0xff, 0x8b, 0x41, 0xd6, - 0xca, 0x00, 0x00, 0x00, + // 172 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x4c, 0xce, 0xbd, 0xae, 0x82, 0x40, + 0x10, 0x05, 0xe0, 0x7b, 0x1b, 0x0d, 0xd8, 0xd1, 0xa8, 0x94, 0x3c, 0xc0, 0x8e, 0x41, 0x5b, 0x1b, + 0xed, 0x4c, 0x78, 0x01, 0xbb, 0xfd, 0x19, 0xd8, 0x0d, 0xb0, 0x4b, 0x66, 0x96, 0xc2, 0xb7, 0x37, + 0x2a, 0x59, 0xed, 0x66, 0x72, 0x4e, 0xbe, 0x9c, 0x7c, 0xab, 0xa4, 0xee, 0xd1, 0x1b, 0xd0, 0xc1, + 0x33, 0x7a, 0x9e, 0x59, 0x4c, 0x14, 0x62, 0x28, 0xd6, 0x4b, 0x50, 0xee, 0xd9, 0x8d, 0xd3, 0x80, + 0xaa, 0x8d, 0x90, 0xae, 0x4f, 0xa7, 0xda, 0xe4, 0x99, 0x95, 0xde, 0xb0, 0x95, 0x3d, 0xd6, 0xb7, + 0x3c, 0x4b, 0x46, 0x71, 0xfe, 0x7d, 0x0a, 0xb1, 0x58, 0x22, 0xb5, 0xcb, 0x9d, 0xf8, 0x62, 0xcd, + 0x3c, 0x44, 0x77, 0xb5, 0xd2, 0xf9, 0x86, 0xbb, 0xea, 0xef, 0xf0, 0x7f, 0x39, 0xdd, 0xeb, 0xce, + 0x45, 0x3b, 0x2b, 0xa1, 0xc3, 0x08, 0xf6, 0x31, 0x21, 0x0d, 0x68, 0x3a, 0x24, 0x68, 0xa5, 0x22, + 0xa7, 0x21, 0x90, 0x41, 0x42, 0x02, 0x7e, 0xcd, 0x5a, 0x74, 0xb5, 0x7a, 0xaf, 0x3a, 0x3e, 0x03, + 0x00, 0x00, 0xff, 0xff, 0x1d, 0x79, 0xa9, 0x9a, 0xd4, 0x00, 0x00, 0x00, } diff --git a/orderer/sbft/backend/consensus.proto b/orderer/sbft/backend/consensus.proto index df6e4b97681..26b2c32c91d 100644 --- a/orderer/sbft/backend/consensus.proto +++ b/orderer/sbft/backend/consensus.proto @@ -7,7 +7,7 @@ option go_package = "github.com/hyperledger/fabric/orderer/sbft/backend"; import "simplebft/simplebft.proto"; service consensus { - rpc consensus(handshake) returns (stream simplebft.Msg) {} + rpc consensus(handshake) returns (stream simplebft.MultiChainMsg) {} } message handshake { diff --git a/orderer/sbft/backend/events.go b/orderer/sbft/backend/events.go index b167ce61144..2da4ba79d0b 100644 --- a/orderer/sbft/backend/events.go +++ b/orderer/sbft/backend/events.go @@ -40,26 +40,29 @@ func (t *Timer) Execute(backend *Backend) { } type msgEvent struct { - msg *s.Msg - src uint64 + chainId string + msg *s.Msg + src uint64 } func (m *msgEvent) Execute(backend *Backend) { - backend.consensus.Receive(m.msg, m.src) + backend.consensus[m.chainId].Receive(m.msg, m.src) } type requestEvent struct { - req []byte + chainId string + req []byte } func (r *requestEvent) Execute(backend *Backend) { - backend.consensus.Request(r.req) + backend.consensus[r.chainId].Request(r.req) } type connectionEvent struct { - peerid uint64 + chainID string + peerid uint64 } func (c *connectionEvent) Execute(backend *Backend) { - backend.consensus.Connection(c.peerid) + backend.consensus[c.chainID].Connection(c.peerid) } diff --git a/orderer/sbft/consenter.go b/orderer/sbft/consenter.go new file mode 100644 index 00000000000..ab29ac745e7 --- /dev/null +++ b/orderer/sbft/consenter.go @@ -0,0 +1,120 @@ +/* +Copyright IBM Corp. 2016 All Rights Reserved. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package sbft + +import ( + "github.com/hyperledger/fabric/orderer/multichain" + "github.com/hyperledger/fabric/orderer/sbft/backend" + "github.com/hyperledger/fabric/orderer/sbft/connection" + "github.com/hyperledger/fabric/orderer/sbft/persist" + "github.com/hyperledger/fabric/orderer/sbft/simplebft" + cb "github.com/hyperledger/fabric/protos/common" + "github.com/op/go-logging" +) + +type consensusStack struct { + persist *persist.Persist + backend *backend.Backend +} + +var logger = logging.MustGetLogger("orderer/main") + +// Consenter interface implementation for new main application +type consenter struct { + config *ConsensusConfig + consensusStack *consensusStack + sbftStackConfig *backend.StackConfig + sbftPeers map[string]*simplebft.SBFT +} + +type chain struct { + chainID string + consensusStack *consensusStack +} + +// New creates a new consenter for the SBFT consensus scheme. +// It accepts messages being delivered via Enqueue, orders them, and then uses the blockcutter to form the messages +// into blocks before writing to the given ledger. +func New(c *ConsensusConfig, sc *backend.StackConfig) multichain.Consenter { + return &consenter{config: c, sbftStackConfig: sc} +} + +func (sbft *consenter) HandleChain(support multichain.ConsenterSupport, metadata *cb.Metadata) (multichain.Chain, error) { + return newChain(sbft, support), nil +} + +func newChain(sbft *consenter, support multichain.ConsenterSupport) *chain { + logger.Infof("Starting a chain: %d", support.ChainID()) + + if sbft.sbftPeers == nil { + sbft.consensusStack = createConsensusStack(sbft) + sbft.sbftPeers = make(map[string]*simplebft.SBFT) + } + sbft.sbftPeers[support.ChainID()] = initSbftPeer(support.ChainID(), sbft, support) + + return &chain{ + chainID: support.ChainID(), + consensusStack: sbft.consensusStack, + } +} + +func createConsensusStack(sbft *consenter) *consensusStack { + logger.Infof("%v %v %v", sbft.sbftStackConfig.ListenAddr, sbft.sbftStackConfig.CertFile, sbft.sbftStackConfig.KeyFile) + conn, err := connection.New(sbft.sbftStackConfig.ListenAddr, sbft.sbftStackConfig.CertFile, sbft.sbftStackConfig.KeyFile) + if err != nil { + logger.Errorf("Error when trying to connect: %s", err) + panic(err) + } + persist := persist.New(sbft.sbftStackConfig.DataDir) + backend, err := backend.NewBackend(sbft.config.Peers, conn, persist) + if err != nil { + logger.Errorf("Backend instantiation error.") + panic(err) + } + return &consensusStack{ + backend: backend, + persist: persist, + } +} + +func initSbftPeer(chainID string, sbft *consenter, support multichain.ConsenterSupport) *simplebft.SBFT { + sbftPeer, err := sbft.consensusStack.backend.AddSbftPeer(support.ChainID(), support, sbft.config.Consensus) + if err != nil { + logger.Errorf("SBFT peer instantiation error.") + panic(err) + } + return sbftPeer +} + +// Chain interface implementation: + +// Start allocates the necessary resources for staying up to date with this Chain. +// It implements the multichain.Chain interface. It is called by multichain.NewManagerImpl() +// which is invoked when the ordering process is launched, before the call to NewServer(). +func (ch *chain) Start() { + +} + +// Halt frees the resources which were allocated for this Chain +func (ch *chain) Halt() { + panic("There is no way to halt SBFT") +} + +// Enqueue accepts a message and returns true on acceptance, or false on shutdown +func (ch *chain) Enqueue(env *cb.Envelope) bool { + return ch.consensusStack.backend.Enqueue(ch.chainID, env) +} diff --git a/orderer/sbft/main/main.go b/orderer/sbft/main/main.go deleted file mode 100644 index ac02d91489c..00000000000 --- a/orderer/sbft/main/main.go +++ /dev/null @@ -1,208 +0,0 @@ -/* -Copyright IBM Corp. 2016 All Rights Reserved. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package main - -import ( - "flag" - "net" - _ "net/http/pprof" - "os" - - "github.com/hyperledger/fabric/orderer/common/bootstrap/provisional" - fileledger "github.com/hyperledger/fabric/orderer/ledger/file" - localconfig "github.com/hyperledger/fabric/orderer/localconfig" - "github.com/hyperledger/fabric/orderer/multichain" - "github.com/hyperledger/fabric/orderer/sbft" - "github.com/hyperledger/fabric/orderer/sbft/backend" - "github.com/hyperledger/fabric/orderer/sbft/connection" - "github.com/hyperledger/fabric/orderer/sbft/persist" - pb "github.com/hyperledger/fabric/orderer/sbft/simplebft" - cb "github.com/hyperledger/fabric/protos/common" - ab "github.com/hyperledger/fabric/protos/orderer" - "github.com/op/go-logging" - "google.golang.org/grpc" -) - -type consensusStack struct { - persist *persist.Persist - backend *backend.Backend -} - -type flags struct { - listenAddr string - grpcAddr string - telemetryAddr string - certFile string - keyFile string - dataDir string - genesisFile string - verbose string - init string -} - -var logger = logging.MustGetLogger("orderer/main") - -// TODO move to_test after integration with common components -func init() { - logging.SetLevel(logging.DEBUG, "") -} - -// Consenter interface implementation for new main application: -// This part is Work In Progress - -type consenter struct{} - -type chain struct { - support multichain.ConsenterSupport -} - -// New creates a new consenter for the SBFT consensus scheme. -// It accepts messages being delivered via Enqueue, orders them, and then -// uses the blockcutter to form the messages into blocks before writing to -// the given ledger. -func New() multichain.Consenter { - return &consenter{} -} - -// HandleChain creates/returns a reference to a Chain for the given set of support resources. -func (solo *consenter) HandleChain(support multichain.ConsenterSupport, metadata *cb.Metadata) (multichain.Chain, error) { - return newChain(support), nil -} - -func newChain(support multichain.ConsenterSupport) *chain { - return &chain{ - support: support, - } -} - -// Chain interface implementation: - -// TODO -// Start allocates whatever resources are needed for staying up to date with the chain -func (ch *chain) Start() { - -} - -// TODO -// Halt frees the resources which were allocated for this Chain -func (ch *chain) Halt() { - -} - -// TODO -// Enqueue accepts a message and returns true on acceptance, or false on shutdown -func (ch *chain) Enqueue(env *cb.Envelope) bool { - return false -} - -// The "old", SBFT only application: -func main() { - var c flags - - flag.StringVar(&c.init, "init", "", "initialized instance from pbft config `file`") - flag.StringVar(&c.listenAddr, "addr", ":6100", "`addr`ess/port of service") - flag.StringVar(&c.grpcAddr, "gaddr", ":7100", "`addr`ess/port of GRPC atomic broadcast server") - flag.StringVar(&c.telemetryAddr, "telemetry", ":7100", "`addr`ess of telemetry/profiler") - flag.StringVar(&c.certFile, "cert", "", "certificate `file`") - flag.StringVar(&c.keyFile, "key", "", "key `file`") - flag.StringVar(&c.dataDir, "data-dir", "", "data `dir`ectory") - flag.StringVar(&c.genesisFile, "genesis-file", "", "`gen`esis block file") - flag.StringVar(&c.verbose, "verbose", "info", "set verbosity `level` (critical, error, warning, notice, info, debug)") - - flag.Parse() - - level, err := logging.LogLevel(c.verbose) - if err != nil { - logger.Panicf("Failed to set loglevel: %s", err) - } - logging.SetLevel(level, "") - - if c.init != "" { - err = initInstance(c) - if err != nil { - logger.Panicf("Failed to initialize SBFT instance: %s", err) - } - return - } - - serve(c) -} - -func initInstance(c flags) error { - config, err := sbft.ReadJsonConfig(c.init) - if err != nil { - return err - } - - err = os.Mkdir(c.dataDir, 0755) - if err != nil { - return err - } - - p := persist.New(c.dataDir) - err = sbft.SaveConfig(p, config) - if err != nil { - return err - } - - logger.Infof("Initialized new peer: listening at %v GRPC at %v", c.listenAddr, c.grpcAddr) - return nil -} - -func serve(c flags) { - if c.dataDir == "" { - logger.Panic("No data directory was given.") - } - - persist := persist.New(c.dataDir) - config, err := sbft.RestoreConfig(persist) - if err != nil { - logger.Panicf("Failed to restore configuration: %s", err) - } - - conn, err := connection.New(c.listenAddr, c.certFile, c.keyFile) - if err != nil { - logger.Panicf("Error when trying to connect: %s", err) - } - s := &consensusStack{ - persist: nil, - } - - localConf := localconfig.Load() - localConf.Genesis.OrdererType = provisional.ConsensusTypeSbft - genesisBlock := provisional.New(localConf).GenesisBlock() - - flf := fileledger.New(c.dataDir) - ledger, _ := flf.GetOrCreate(provisional.TestChainID) - ledger.Append(genesisBlock) - s.backend, err = backend.NewBackend(config.Peers, conn, ledger, persist) - if err != nil { - logger.Panicf("Failed to create a new backend instance: %s", err) - } - - sbft, _ := pb.New(s.backend.GetMyId(), config.Consensus, s.backend) - s.backend.SetReceiver(sbft) - - grpcServer := grpc.NewServer() - lis, err := net.Listen("tcp", c.grpcAddr) - if err != nil { - logger.Panicf("Failed to listen: %s", err) - } - broadcastab := backend.NewBackendAB(s.backend) - ab.RegisterAtomicBroadcastServer(grpcServer, broadcastab) - grpcServer.Serve(lis) -} diff --git a/orderer/sbft/simplebft/backlog.go b/orderer/sbft/simplebft/backlog.go index aa7be6b1eb4..7dfd0eab682 100644 --- a/orderer/sbft/simplebft/backlog.go +++ b/orderer/sbft/simplebft/backlog.go @@ -54,7 +54,7 @@ func (s *SBFT) recordBacklogMsg(m *Msg, src uint64) { if len(s.replicaState[src].backLog) > maxBacklogSeq*msgPerSeq { log.Debugf("replica %d: backlog for %d full, discarding and reconnecting", s.id, src) s.discardBacklog(src) - s.sys.Reconnect(src) + s.sys.Reconnect(s.chainId, src) } } diff --git a/orderer/sbft/simplebft/commit.go b/orderer/sbft/simplebft/commit.go index 47f5df54aa9..957fd95cdf9 100644 --- a/orderer/sbft/simplebft/commit.go +++ b/orderer/sbft/simplebft/commit.go @@ -29,7 +29,7 @@ func (s *SBFT) maybeSendCommit() { func (s *SBFT) sendCommit() { s.cur.prepared = true c := s.cur.subject - s.sys.Persist(prepared, &c) + s.sys.Persist(s.chainId, prepared, &c) s.broadcast(&Msg{&Msg_Commit{&c}}) } @@ -58,7 +58,7 @@ func (s *SBFT) handleCommit(c *Subject, src uint64) { s.cur.committed = true log.Noticef("replica %d: executing %v %x", s.id, s.cur.subject.Seq, s.cur.subject.Digest) - s.sys.Persist(committed, &s.cur.subject) + s.sys.Persist(s.chainId, committed, &s.cur.subject) s.sendCheckpoint() s.processBacklog() diff --git a/orderer/sbft/simplebft/connection.go b/orderer/sbft/simplebft/connection.go index 70eddf91d56..11be8f68d52 100644 --- a/orderer/sbft/simplebft/connection.go +++ b/orderer/sbft/simplebft/connection.go @@ -21,17 +21,17 @@ package simplebft // On connection, we send our latest (weak) checkpoint, and we expect // to receive one from replica. func (s *SBFT) Connection(replica uint64) { - batch := *s.sys.LastBatch() + batch := *s.sys.LastBatch(s.chainId) batch.Payloads = nil // don't send the big payload hello := &Hello{Batch: &batch} if s.isPrimary() && s.activeView && s.lastNewViewSent != nil { hello.NewView = s.lastNewViewSent } - s.sys.Send(&Msg{&Msg_Hello{hello}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_Hello{hello}}, replica) svc := s.replicaState[s.id].signedViewchange if svc != nil { - s.sys.Send(&Msg{&Msg_ViewChange{svc}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_ViewChange{svc}}, replica) } // A reconnecting replica can play forward its blockchain to @@ -51,15 +51,15 @@ func (s *SBFT) Connection(replica uint64) { if s.cur.subject.Seq.Seq > batchheader.Seq && s.activeView { if s.isPrimary() { - s.sys.Send(&Msg{&Msg_Preprepare{s.cur.preprep}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_Preprepare{s.cur.preprep}}, replica) } else { - s.sys.Send(&Msg{&Msg_Prepare{&s.cur.subject}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_Prepare{&s.cur.subject}}, replica) } if s.cur.prepared { - s.sys.Send(&Msg{&Msg_Commit{&s.cur.subject}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_Commit{&s.cur.subject}}, replica) } if s.cur.committed { - s.sys.Send(&Msg{&Msg_Checkpoint{s.makeCheckpoint()}}, replica) + s.sys.Send(s.chainId, &Msg{&Msg_Checkpoint{s.makeCheckpoint()}}, replica) } } } @@ -73,7 +73,7 @@ func (s *SBFT) handleHello(h *Hello, src uint64) { return } - if s.sys.LastBatch().DecodeHeader().Seq < bh.Seq { + if s.sys.LastBatch(s.chainId).DecodeHeader().Seq < bh.Seq { log.Debugf("replica %d: delivering batch %d after hello from replica %d", s.id, bh.Seq, src) s.deliverBatch(h.Batch) } diff --git a/orderer/sbft/simplebft/newview.go b/orderer/sbft/simplebft/newview.go index 98103392bec..b3bb757f0b6 100644 --- a/orderer/sbft/simplebft/newview.go +++ b/orderer/sbft/simplebft/newview.go @@ -147,7 +147,7 @@ func (s *SBFT) handleNewView(nv *NewView, src uint64) { s.discardBacklog(s.primaryID()) // maybe deliver previous batch - if s.sys.LastBatch().DecodeHeader().Seq < prevBatch.DecodeHeader().Seq { + if s.sys.LastBatch(s.chainId).DecodeHeader().Seq < prevBatch.DecodeHeader().Seq { if prevBatch.DecodeHeader().Seq == s.cur.subject.Seq.Seq { // we just received a signature set for a request which we preprepared, but never delivered. // check first if the locally preprepared request matches the signature set diff --git a/orderer/sbft/simplebft/preprepare.go b/orderer/sbft/simplebft/preprepare.go index 981a97c4b61..300050421f1 100644 --- a/orderer/sbft/simplebft/preprepare.go +++ b/orderer/sbft/simplebft/preprepare.go @@ -29,14 +29,14 @@ func (s *SBFT) sendPreprepare(batch []*Request) { data[i] = req.Payload } - lasthash := hash(s.sys.LastBatch().Header) + lasthash := hash(s.sys.LastBatch(s.chainId).Header) m := &Preprepare{ Seq: &seq, Batch: s.makeBatch(seq.Seq, lasthash, data), } - s.sys.Persist(preprepared, m) + s.sys.Persist(s.chainId, preprepared, m) s.broadcast(&Msg{&Msg_Preprepare{m}}) s.handleCheckedPreprepare(m) } @@ -70,7 +70,7 @@ func (s *SBFT) handlePreprepare(pp *Preprepare, src uint64) { return } - prevhash := s.sys.LastBatch().Hash() + prevhash := s.sys.LastBatch(s.chainId).Hash() if !bytes.Equal(batchheader.PrevHash, prevhash) { log.Infof("replica %d: preprepare batch prev hash does not match expected %s, got %s", s.id, hash2str(batchheader.PrevHash), hash2str(prevhash)) return @@ -83,7 +83,7 @@ func (s *SBFT) acceptPreprepare(pp *Preprepare) { sub := Subject{Seq: pp.Seq, Digest: pp.Batch.Hash()} log.Infof("replica %d: accepting preprepare for %v, %x", s.id, sub.Seq, sub.Digest) - s.sys.Persist(preprepared, pp) + s.sys.Persist(s.chainId, preprepared, pp) s.cur = reqInfo{ subject: sub, diff --git a/orderer/sbft/simplebft/request.go b/orderer/sbft/simplebft/request.go index 2594296638e..0b96f3f8c6b 100644 --- a/orderer/sbft/simplebft/request.go +++ b/orderer/sbft/simplebft/request.go @@ -20,6 +20,7 @@ import "time" // Request proposes a new request to the BFT network. func (s *SBFT) Request(req []byte) { + log.Debugf("replica %d: broadcasting a request", s.id) s.broadcast(&Msg{&Msg_Request{&Request{req}}}) } diff --git a/orderer/sbft/simplebft/simplebft.go b/orderer/sbft/simplebft/simplebft.go index 6f267628fae..3dc748842cc 100644 --- a/orderer/sbft/simplebft/simplebft.go +++ b/orderer/sbft/simplebft/simplebft.go @@ -36,20 +36,21 @@ type Receiver interface { Receive(msg *Msg, src uint64) Request(req []byte) Connection(replica uint64) + GetChainId() string } // System defines the API that needs to be provided for SBFT. type System interface { - Send(msg *Msg, dest uint64) + Send(chainId string, msg *Msg, dest uint64) Timer(d time.Duration, f func()) Canceller - Deliver(batch *Batch) - SetReceiver(receiver Receiver) - Persist(key string, data proto.Message) - Restore(key string, out proto.Message) bool - LastBatch() *Batch + Deliver(chainId string, batch *Batch) + AddReceiver(chainId string, receiver Receiver) + Persist(chainId string, key string, data proto.Message) + Restore(chainId string, key string, out proto.Message) bool + LastBatch(chainId string) *Batch Sign(data []byte) []byte CheckSig(data []byte, src uint64, sig []byte) error - Reconnect(replica uint64) + Reconnect(chainId string, replica uint64) } // Canceller allows cancelling of a scheduled timer event. @@ -73,6 +74,7 @@ type SBFT struct { viewChangeTimer Canceller replicaState []replicaInfo pending map[string]*Request + chainId string } type reqInfo struct { @@ -101,20 +103,21 @@ type dummyCanceller struct{} func (d dummyCanceller) Cancel() {} // New creates a new SBFT instance. -func New(id uint64, config *Config, sys System) (*SBFT, error) { +func New(id uint64, chainID string, config *Config, sys System) (*SBFT, error) { if config.F*3+1 > config.N { - return nil, fmt.Errorf("invalid combination of N and F") + return nil, fmt.Errorf("invalid combination of N (%d) and F (%d)", config.N, config.F) } s := &SBFT{ config: *config, sys: sys, id: id, + chainId: chainID, viewChangeTimer: dummyCanceller{}, replicaState: make([]replicaInfo, config.N), pending: make(map[string]*Request), } - s.sys.SetReceiver(s) + s.sys.AddReceiver(chainID, s) s.view = 0 s.cur.subject.Seq = &SeqView{} @@ -125,7 +128,7 @@ func New(id uint64, config *Config, sys System) (*SBFT, error) { s.activeView = true svc := &Signed{} - if s.sys.Restore(viewchange, svc) { + if s.sys.Restore(s.chainId, viewchange, svc) { vc := &ViewChange{} err := proto.Unmarshal(svc.Data, vc) if err != nil { @@ -137,7 +140,7 @@ func New(id uint64, config *Config, sys System) (*SBFT, error) { } pp := &Preprepare{} - if s.sys.Restore(preprepared, pp) && pp.Seq.View >= s.view { + if s.sys.Restore(s.chainId, preprepared, pp) && pp.Seq.View >= s.view { s.view = pp.Seq.View s.activeView = true if pp.Seq.Seq > s.seq() { @@ -145,11 +148,11 @@ func New(id uint64, config *Config, sys System) (*SBFT, error) { } } c := &Subject{} - if s.sys.Restore(prepared, c) && reflect.DeepEqual(c, &s.cur.subject) && c.Seq.View >= s.view { + if s.sys.Restore(s.chainId, prepared, c) && reflect.DeepEqual(c, &s.cur.subject) && c.Seq.View >= s.view { s.cur.prepared = true } ex := &Subject{} - if s.sys.Restore(committed, ex) && reflect.DeepEqual(c, &s.cur.subject) && ex.Seq.View >= s.view { + if s.sys.Restore(s.chainId, committed, ex) && reflect.DeepEqual(c, &s.cur.subject) && ex.Seq.View >= s.view { s.cur.committed = true } @@ -159,6 +162,10 @@ func New(id uint64, config *Config, sys System) (*SBFT, error) { //////////////////////////////////////////////// +func (s *SBFT) GetChainId() string { + return s.chainId +} + func (s *SBFT) primaryIDView(v uint64) uint64 { return v % s.config.N } @@ -172,7 +179,7 @@ func (s *SBFT) isPrimary() bool { } func (s *SBFT) seq() uint64 { - return s.sys.LastBatch().DecodeHeader().Seq + return s.sys.LastBatch(s.chainId).DecodeHeader().Seq } func (s *SBFT) nextSeq() SeqView { @@ -203,7 +210,7 @@ func (s *SBFT) oneCorrectQuorum() int { func (s *SBFT) broadcast(m *Msg) { for i := uint64(0); i < s.config.N; i++ { - s.sys.Send(m, i) + s.sys.Send(s.chainId, m, i) } } @@ -257,7 +264,7 @@ func (s *SBFT) handleQueueableMessage(m *Msg, src uint64) { func (s *SBFT) deliverBatch(batch *Batch) { s.cur.checkpointDone = true s.cur.timeout.Cancel() - s.sys.Deliver(batch) + s.sys.Deliver(s.chainId, batch) for _, req := range batch.Payloads { key := hash2str(hash(req)) diff --git a/orderer/sbft/simplebft/simplebft.pb.go b/orderer/sbft/simplebft/simplebft.pb.go index edc350277b3..38d202f5f62 100644 --- a/orderer/sbft/simplebft/simplebft.pb.go +++ b/orderer/sbft/simplebft/simplebft.pb.go @@ -1,15 +1,16 @@ // Code generated by protoc-gen-go. -// source: simplebft/simplebft.proto +// source: simplebft.proto // DO NOT EDIT! /* Package simplebft is a generated protocol buffer package. It is generated from these files: - simplebft/simplebft.proto + simplebft.proto It has these top-level messages: Config + MultiChainMsg Msg Request SeqView @@ -53,6 +54,23 @@ func (m *Config) String() string { return proto.CompactTextString(m) func (*Config) ProtoMessage() {} func (*Config) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} } +type MultiChainMsg struct { + ChainID string `protobuf:"bytes,1,opt,name=chainId" json:"chainId,omitempty"` + Msg *Msg `protobuf:"bytes,2,opt,name=msg" json:"msg,omitempty"` +} + +func (m *MultiChainMsg) Reset() { *m = MultiChainMsg{} } +func (m *MultiChainMsg) String() string { return proto.CompactTextString(m) } +func (*MultiChainMsg) ProtoMessage() {} +func (*MultiChainMsg) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } + +func (m *MultiChainMsg) GetMsg() *Msg { + if m != nil { + return m.Msg + } + return nil +} + type Msg struct { // Types that are valid to be assigned to Type: // *Msg_Request @@ -69,7 +87,7 @@ type Msg struct { func (m *Msg) Reset() { *m = Msg{} } func (m *Msg) String() string { return proto.CompactTextString(m) } func (*Msg) ProtoMessage() {} -func (*Msg) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} } +func (*Msg) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } type isMsg_Type interface { isMsg_Type() @@ -367,7 +385,7 @@ type Request struct { func (m *Request) Reset() { *m = Request{} } func (m *Request) String() string { return proto.CompactTextString(m) } func (*Request) ProtoMessage() {} -func (*Request) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} } +func (*Request) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } type SeqView struct { View uint64 `protobuf:"varint,1,opt,name=view" json:"view,omitempty"` @@ -377,7 +395,7 @@ type SeqView struct { func (m *SeqView) Reset() { *m = SeqView{} } func (m *SeqView) String() string { return proto.CompactTextString(m) } func (*SeqView) ProtoMessage() {} -func (*SeqView) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} } +func (*SeqView) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } type BatchHeader struct { Seq uint64 `protobuf:"varint,1,opt,name=seq" json:"seq,omitempty"` @@ -388,7 +406,7 @@ type BatchHeader struct { func (m *BatchHeader) Reset() { *m = BatchHeader{} } func (m *BatchHeader) String() string { return proto.CompactTextString(m) } func (*BatchHeader) ProtoMessage() {} -func (*BatchHeader) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} } +func (*BatchHeader) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } type Batch struct { Header []byte `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"` @@ -399,7 +417,7 @@ type Batch struct { func (m *Batch) Reset() { *m = Batch{} } func (m *Batch) String() string { return proto.CompactTextString(m) } func (*Batch) ProtoMessage() {} -func (*Batch) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} } +func (*Batch) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } func (m *Batch) GetSignatures() map[uint64][]byte { if m != nil { @@ -416,7 +434,7 @@ type Preprepare struct { func (m *Preprepare) Reset() { *m = Preprepare{} } func (m *Preprepare) String() string { return proto.CompactTextString(m) } func (*Preprepare) ProtoMessage() {} -func (*Preprepare) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} } +func (*Preprepare) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } func (m *Preprepare) GetSeq() *SeqView { if m != nil { @@ -440,7 +458,7 @@ type Subject struct { func (m *Subject) Reset() { *m = Subject{} } func (m *Subject) String() string { return proto.CompactTextString(m) } func (*Subject) ProtoMessage() {} -func (*Subject) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} } +func (*Subject) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } func (m *Subject) GetSeq() *SeqView { if m != nil { @@ -459,7 +477,7 @@ type ViewChange struct { func (m *ViewChange) Reset() { *m = ViewChange{} } func (m *ViewChange) String() string { return proto.CompactTextString(m) } func (*ViewChange) ProtoMessage() {} -func (*ViewChange) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} } +func (*ViewChange) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } func (m *ViewChange) GetPset() []*Subject { if m != nil { @@ -490,7 +508,7 @@ type Signed struct { func (m *Signed) Reset() { *m = Signed{} } func (m *Signed) String() string { return proto.CompactTextString(m) } func (*Signed) ProtoMessage() {} -func (*Signed) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} } +func (*Signed) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } type NewView struct { View uint64 `protobuf:"varint,1,opt,name=view" json:"view,omitempty"` @@ -502,7 +520,7 @@ type NewView struct { func (m *NewView) Reset() { *m = NewView{} } func (m *NewView) String() string { return proto.CompactTextString(m) } func (*NewView) ProtoMessage() {} -func (*NewView) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} } +func (*NewView) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } func (m *NewView) GetVset() map[uint64]*Signed { if m != nil { @@ -534,7 +552,7 @@ type Checkpoint struct { func (m *Checkpoint) Reset() { *m = Checkpoint{} } func (m *Checkpoint) String() string { return proto.CompactTextString(m) } func (*Checkpoint) ProtoMessage() {} -func (*Checkpoint) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} } +func (*Checkpoint) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } type Hello struct { Batch *Batch `protobuf:"bytes,1,opt,name=batch" json:"batch,omitempty"` @@ -544,7 +562,7 @@ type Hello struct { func (m *Hello) Reset() { *m = Hello{} } func (m *Hello) String() string { return proto.CompactTextString(m) } func (*Hello) ProtoMessage() {} -func (*Hello) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} } +func (*Hello) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} } func (m *Hello) GetBatch() *Batch { if m != nil { @@ -562,6 +580,7 @@ func (m *Hello) GetNewView() *NewView { func init() { proto.RegisterType((*Config)(nil), "simplebft.Config") + proto.RegisterType((*MultiChainMsg)(nil), "simplebft.MultiChainMsg") proto.RegisterType((*Msg)(nil), "simplebft.Msg") proto.RegisterType((*Request)(nil), "simplebft.Request") proto.RegisterType((*SeqView)(nil), "simplebft.SeqView") @@ -576,58 +595,61 @@ func init() { proto.RegisterType((*Hello)(nil), "simplebft.Hello") } -func init() { proto.RegisterFile("simplebft/simplebft.proto", fileDescriptor0) } +func init() { proto.RegisterFile("simplebft.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 799 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x8c, 0x55, 0xdd, 0x6e, 0xf3, 0x44, - 0x10, 0xad, 0x63, 0xc7, 0x49, 0x26, 0x15, 0xe4, 0x5b, 0x4a, 0x65, 0x4a, 0x2f, 0x22, 0x83, 0x4a, - 0x2e, 0x20, 0xa9, 0x42, 0x05, 0xa8, 0x12, 0x12, 0x4a, 0x41, 0x44, 0x48, 0x54, 0xc8, 0xad, 0x2a, - 0xd1, 0x0b, 0x22, 0xc7, 0x9e, 0xd8, 0xa6, 0x89, 0xed, 0x78, 0x37, 0x49, 0xd3, 0x87, 0xe1, 0x82, - 0x27, 0xe0, 0x01, 0x78, 0x23, 0x5e, 0x02, 0xed, 0x4f, 0x6c, 0x37, 0x3f, 0xd5, 0x27, 0xf9, 0x62, - 0x77, 0xce, 0xd9, 0xdd, 0x39, 0xb3, 0x67, 0xd6, 0xf0, 0x09, 0x8d, 0x66, 0xe9, 0x14, 0xc7, 0x13, - 0xd6, 0xcb, 0x47, 0xdd, 0x34, 0x4b, 0x58, 0x42, 0x1a, 0x79, 0xc0, 0xfe, 0x47, 0x03, 0xf3, 0x26, - 0x89, 0x27, 0x51, 0x40, 0x8e, 0x41, 0x8b, 0x2d, 0xad, 0xad, 0x75, 0x0c, 0x47, 0x8b, 0xf9, 0x6c, - 0x62, 0x55, 0xe4, 0x6c, 0x42, 0xba, 0xf0, 0xd1, 0xd8, 0x65, 0x5e, 0x38, 0xf2, 0x17, 0x99, 0xcb, - 0xa2, 0x24, 0x1e, 0xc5, 0x14, 0x3d, 0x4b, 0x17, 0xf8, 0x3b, 0x01, 0xfd, 0xa8, 0x90, 0x5b, 0x8a, - 0x1e, 0xe9, 0x40, 0x4b, 0xf2, 0x69, 0xf4, 0x82, 0xa3, 0xf1, 0x9a, 0x21, 0xb5, 0x0c, 0x41, 0xfe, - 0x40, 0xc4, 0xef, 0xa2, 0x17, 0x1c, 0xf0, 0x28, 0xb9, 0x84, 0x93, 0x0c, 0xe7, 0x0b, 0xa4, 0x6c, - 0xc4, 0xa2, 0x19, 0x26, 0x0b, 0x26, 0xb7, 0xae, 0x0a, 0x36, 0x51, 0xd8, 0xbd, 0x84, 0xf8, 0xde, - 0xf6, 0x5f, 0x3a, 0xe8, 0xbf, 0xd2, 0x80, 0x74, 0xa1, 0xa6, 0x50, 0x91, 0x75, 0xb3, 0x4f, 0xba, - 0x85, 0x50, 0x47, 0x22, 0xc3, 0x23, 0x67, 0x43, 0x22, 0xdf, 0x02, 0xa4, 0x19, 0xf2, 0xcf, 0xcd, - 0x50, 0x48, 0x6b, 0xf6, 0x3f, 0x2e, 0x2d, 0xf9, 0x2d, 0x07, 0x87, 0x47, 0x4e, 0x89, 0xca, 0x0f, - 0xda, 0xac, 0xd2, 0x77, 0x0e, 0xba, 0x5b, 0x8c, 0xff, 0x44, 0x4f, 0x1c, 0xb4, 0xe1, 0x7f, 0x09, - 0xa6, 0x97, 0xcc, 0x66, 0x11, 0x13, 0x92, 0x0f, 0xd1, 0x15, 0x87, 0x5c, 0x41, 0x73, 0x19, 0xe1, - 0x6a, 0xe4, 0x85, 0x6e, 0x1c, 0xa0, 0xd0, 0xdd, 0xec, 0xbf, 0x2b, 0x2f, 0x89, 0x82, 0x18, 0x7d, - 0x9e, 0x13, 0xe7, 0xdd, 0x08, 0x1a, 0xe9, 0x41, 0x3d, 0xc6, 0xd5, 0x88, 0x47, 0x2c, 0x73, 0xe7, - 0x94, 0x5b, 0x5c, 0x3d, 0x44, 0xb8, 0xe2, 0x49, 0xc5, 0x72, 0xc8, 0xd5, 0x7b, 0x21, 0x7a, 0x4f, - 0x69, 0x12, 0xc5, 0xcc, 0xaa, 0xed, 0xa8, 0xbf, 0xc9, 0x41, 0x7e, 0x52, 0x41, 0x25, 0x1d, 0xa8, - 0x86, 0x38, 0x9d, 0x26, 0x56, 0x5d, 0xac, 0x69, 0x95, 0xd6, 0x0c, 0x79, 0x7c, 0x78, 0xe4, 0x48, - 0xc2, 0xc0, 0x04, 0x83, 0xad, 0x53, 0xb4, 0x3f, 0x83, 0x9a, 0x2a, 0x3f, 0xb1, 0xa0, 0x96, 0xba, - 0xeb, 0x69, 0xe2, 0xfa, 0xe2, 0x8e, 0x8e, 0x9d, 0xcd, 0xd4, 0xee, 0x41, 0xed, 0x0e, 0xe7, 0x22, - 0x35, 0x02, 0x86, 0xd0, 0x21, 0xbd, 0x27, 0xc6, 0xa4, 0x05, 0x3a, 0xc5, 0xb9, 0x32, 0x20, 0x1f, - 0xda, 0xbf, 0x43, 0x73, 0xc0, 0xad, 0x33, 0x44, 0xd7, 0xc7, 0x6c, 0x43, 0xd0, 0x72, 0x02, 0xf9, - 0x14, 0x1a, 0x69, 0x86, 0xcb, 0x51, 0xe8, 0xd2, 0x50, 0x2c, 0x3c, 0x76, 0xea, 0x3c, 0x30, 0x74, - 0x69, 0xc8, 0x41, 0xdf, 0x65, 0xae, 0x04, 0x75, 0x09, 0xf2, 0x00, 0x07, 0xed, 0x7f, 0x35, 0xa8, - 0x8a, 0xbd, 0xc9, 0x29, 0x98, 0xa1, 0xd8, 0x5f, 0xa5, 0xab, 0x66, 0xe4, 0x0c, 0xea, 0x2a, 0x71, - 0x6a, 0x55, 0xda, 0xba, 0xd8, 0x5a, 0xcd, 0xc9, 0x0f, 0x00, 0x34, 0x0a, 0x62, 0x97, 0x2d, 0x32, - 0xa4, 0x96, 0xde, 0xd6, 0x3b, 0xcd, 0x7e, 0xbb, 0x54, 0x25, 0xb1, 0xb3, 0xb8, 0x45, 0x49, 0xf9, - 0x29, 0x66, 0xd9, 0xda, 0x29, 0xad, 0x39, 0xfb, 0x1e, 0x3e, 0xdc, 0x82, 0xb9, 0xbc, 0x27, 0x5c, - 0x6f, 0xe4, 0x3d, 0xe1, 0x9a, 0x9c, 0x40, 0x75, 0xe9, 0x4e, 0x17, 0xa8, 0xa4, 0xc9, 0xc9, 0x75, - 0xe5, 0x3b, 0xcd, 0x7e, 0x04, 0x28, 0xbc, 0x4b, 0x3e, 0x2f, 0x0a, 0xb3, 0x65, 0x3d, 0x59, 0x6e, - 0x59, 0xac, 0x0b, 0xa8, 0x8a, 0x46, 0x54, 0x7d, 0xd0, 0xda, 0xce, 0xd7, 0x91, 0xb0, 0xfd, 0x33, - 0xd4, 0x94, 0x65, 0xdf, 0x73, 0xe3, 0x53, 0x30, 0xfd, 0x28, 0xe0, 0x4d, 0x29, 0xf3, 0x54, 0x33, - 0xfb, 0x6f, 0x0d, 0xe0, 0xa1, 0xf0, 0xef, 0xbe, 0x3b, 0xbf, 0x00, 0x23, 0xa5, 0xc8, 0x44, 0x81, - 0xf7, 0x76, 0x8d, 0x23, 0x70, 0xce, 0x9b, 0x73, 0x9e, 0x7e, 0x98, 0xc7, 0x71, 0x72, 0xf9, 0xca, - 0xf2, 0xc6, 0x01, 0xa1, 0x25, 0x8e, 0x7d, 0x0d, 0xa6, 0xec, 0x36, 0x9e, 0x1f, 0xb7, 0x87, 0xb2, - 0x81, 0x18, 0x93, 0x73, 0x68, 0xe4, 0x97, 0xa6, 0xd4, 0x15, 0x01, 0xfb, 0x3f, 0x0d, 0x6a, 0xaa, - 0xef, 0xf6, 0xaa, 0xbb, 0x04, 0x63, 0x59, 0xa8, 0x3b, 0xdf, 0xed, 0xd6, 0xee, 0x03, 0x45, 0x26, - 0xcd, 0x21, 0x98, 0x5c, 0xe7, 0xb3, 0xd4, 0xa9, 0x1d, 0xd2, 0xf9, 0x2c, 0x79, 0xea, 0x2e, 0x8d, - 0x37, 0xef, 0xf2, 0xec, 0x17, 0x68, 0xe4, 0x47, 0xec, 0x31, 0xd8, 0x17, 0x65, 0x83, 0xed, 0x7b, - 0x82, 0xca, 0x9e, 0xbb, 0x07, 0x28, 0x5e, 0x8c, 0x3d, 0xcd, 0x78, 0xc0, 0x06, 0xaf, 0x6b, 0xa8, - 0x6f, 0xd7, 0xf0, 0x0f, 0xa8, 0x8a, 0x37, 0xa5, 0x90, 0xa4, 0xbd, 0x29, 0x89, 0x7c, 0x55, 0x7a, - 0x06, 0x2b, 0x87, 0x9e, 0xc1, 0xfc, 0x11, 0x1c, 0x7c, 0xf3, 0x78, 0x15, 0x44, 0x2c, 0x5c, 0x8c, - 0xbb, 0x5e, 0x32, 0xeb, 0x85, 0xeb, 0x14, 0xb3, 0x29, 0xfa, 0x01, 0x66, 0xbd, 0x89, 0x3b, 0xce, - 0x22, 0xaf, 0x97, 0x64, 0x3e, 0x66, 0x98, 0xf5, 0xe8, 0xab, 0xdf, 0xe6, 0xd8, 0x14, 0xff, 0xcd, - 0xaf, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x6d, 0x7b, 0xcb, 0xdb, 0x54, 0x07, 0x00, 0x00, + // 842 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x8c, 0x55, 0xdd, 0x8e, 0xdb, 0x54, + 0x10, 0x5e, 0xaf, 0x1d, 0x67, 0x33, 0x59, 0xda, 0xed, 0xa1, 0x54, 0xd6, 0xd2, 0x8b, 0xc8, 0xa0, + 0xb2, 0x17, 0x90, 0xac, 0x96, 0x0a, 0x50, 0x25, 0x24, 0x94, 0x14, 0x11, 0x40, 0x5b, 0x21, 0x6f, + 0xb5, 0x12, 0xbd, 0x20, 0xf2, 0xcf, 0xc4, 0x3e, 0x6c, 0x62, 0x3b, 0x3e, 0xc7, 0x49, 0xd3, 0x87, + 0xe1, 0x82, 0x27, 0xe0, 0x01, 0x78, 0x23, 0x5e, 0x02, 0x9d, 0x9f, 0xd8, 0x6e, 0x7e, 0x56, 0x95, + 0x72, 0xe1, 0x99, 0xef, 0x3b, 0x33, 0xf3, 0xcd, 0x99, 0x39, 0x81, 0x87, 0x8c, 0xce, 0xf3, 0x19, + 0x06, 0x53, 0xde, 0xcf, 0x8b, 0x8c, 0x67, 0xa4, 0x53, 0x39, 0xdc, 0x7f, 0x0c, 0xb0, 0x47, 0x59, + 0x3a, 0xa5, 0x31, 0x39, 0x05, 0x23, 0x75, 0x8c, 0x9e, 0x71, 0x61, 0x79, 0x46, 0x2a, 0xac, 0xa9, + 0x73, 0xac, 0xac, 0x29, 0xe9, 0xc3, 0xc7, 0x81, 0xcf, 0xc3, 0x64, 0x12, 0x95, 0x85, 0xcf, 0x69, + 0x96, 0x4e, 0x52, 0x86, 0xa1, 0x63, 0x4a, 0xfc, 0x91, 0x84, 0x5e, 0x6a, 0xe4, 0x15, 0xc3, 0x90, + 0x5c, 0xc0, 0x99, 0xe2, 0x33, 0xfa, 0x0e, 0x27, 0xc1, 0x9a, 0x23, 0x73, 0x2c, 0x49, 0x7e, 0x20, + 0xfd, 0x37, 0xf4, 0x1d, 0x0e, 0x85, 0x97, 0x5c, 0xc2, 0xe3, 0x02, 0x17, 0x25, 0x32, 0x3e, 0xe1, + 0x74, 0x8e, 0x59, 0xc9, 0x55, 0xe8, 0x96, 0x64, 0x13, 0x8d, 0xbd, 0x56, 0x90, 0x88, 0xed, 0xfe, + 0x0a, 0x1f, 0x5d, 0x97, 0x33, 0x4e, 0x47, 0x89, 0x4f, 0xd3, 0x6b, 0x16, 0x13, 0x07, 0xda, 0xa1, + 0xf8, 0xfe, 0xf9, 0xa5, 0x2c, 0xbf, 0xe3, 0x6d, 0x4c, 0xd2, 0x03, 0x73, 0xce, 0x62, 0x29, 0xa3, + 0x7b, 0xf5, 0xa0, 0x5f, 0xf7, 0xe1, 0x9a, 0xc5, 0x9e, 0x80, 0xdc, 0xbf, 0x4c, 0x30, 0x45, 0x8c, + 0x3e, 0xb4, 0x75, 0x2a, 0x19, 0xa3, 0x7b, 0x45, 0x1a, 0x6c, 0x4f, 0x21, 0xe3, 0x23, 0x6f, 0x43, + 0x22, 0xdf, 0x02, 0xe4, 0x05, 0x8a, 0x9f, 0x5f, 0xa0, 0x4e, 0xf0, 0x49, 0xe3, 0xc8, 0x6f, 0x15, + 0x38, 0x3e, 0xf2, 0x1a, 0x54, 0x91, 0x68, 0x73, 0xca, 0xdc, 0x49, 0x74, 0x53, 0x06, 0x7f, 0x62, + 0x28, 0x13, 0x6d, 0xf8, 0x5f, 0x82, 0x1d, 0x66, 0xf3, 0x39, 0xe5, 0xb2, 0x7f, 0x87, 0xe8, 0x9a, + 0x43, 0x9e, 0x43, 0x77, 0x49, 0x71, 0x35, 0x09, 0x13, 0x3f, 0x8d, 0x51, 0x36, 0xb1, 0x7b, 0xf5, + 0xa8, 0x79, 0x84, 0xc6, 0x29, 0x46, 0xa2, 0x26, 0xc1, 0x1b, 0x49, 0x1a, 0x19, 0xc0, 0x49, 0x8a, + 0xab, 0x89, 0xf0, 0x38, 0xf6, 0x4e, 0x96, 0x57, 0xb8, 0xba, 0xa5, 0xb8, 0x12, 0x45, 0xa5, 0xea, + 0x53, 0xa8, 0x0f, 0x13, 0x0c, 0xef, 0xf2, 0x8c, 0xa6, 0xdc, 0x69, 0xef, 0xa8, 0x1f, 0x55, 0xa0, + 0xc8, 0x54, 0x53, 0xc9, 0x05, 0xb4, 0x12, 0x9c, 0xcd, 0x32, 0xe7, 0x44, 0x9e, 0x39, 0x6b, 0x9c, + 0x19, 0x0b, 0xff, 0xf8, 0xc8, 0x53, 0x84, 0xa1, 0x0d, 0x16, 0x5f, 0xe7, 0xe8, 0x7e, 0x06, 0x6d, + 0xdd, 0x7e, 0x71, 0xcf, 0xb9, 0xbf, 0x9e, 0x65, 0x7e, 0x24, 0xef, 0xe8, 0xd4, 0xdb, 0x98, 0xee, + 0x00, 0xda, 0x37, 0xb8, 0x90, 0xa5, 0x11, 0xb0, 0xa4, 0x0e, 0x35, 0xc8, 0xf2, 0x9b, 0x9c, 0x81, + 0xc9, 0x70, 0xa1, 0xa7, 0x59, 0x7c, 0xba, 0xbf, 0x43, 0x77, 0x28, 0xe6, 0x70, 0x8c, 0x7e, 0x84, + 0xc5, 0x86, 0x60, 0x54, 0x04, 0xf2, 0x29, 0x74, 0xf2, 0x02, 0x97, 0x93, 0xc4, 0x67, 0x89, 0x3c, + 0x78, 0xea, 0x9d, 0x08, 0xc7, 0xd8, 0x67, 0x89, 0x00, 0x23, 0x9f, 0xfb, 0x0a, 0x34, 0x15, 0x28, + 0x1c, 0x02, 0x74, 0xff, 0x35, 0xa0, 0x25, 0x63, 0x93, 0x27, 0x60, 0x27, 0x32, 0xbe, 0x2e, 0x57, + 0x5b, 0xe4, 0x1c, 0x4e, 0x74, 0xe1, 0xcc, 0x39, 0xee, 0x99, 0x32, 0xb4, 0xb6, 0xc9, 0x0f, 0x00, + 0x8c, 0xc6, 0xa9, 0xcf, 0xcb, 0x02, 0x99, 0x63, 0xf6, 0xcc, 0x8b, 0xee, 0x55, 0xaf, 0xd1, 0x25, + 0x19, 0x59, 0xde, 0xa2, 0xa2, 0xfc, 0x98, 0xf2, 0x62, 0xed, 0x35, 0xce, 0x9c, 0x7f, 0x0f, 0x0f, + 0xb7, 0x60, 0x21, 0xef, 0x0e, 0xd7, 0x1b, 0x79, 0x77, 0xb8, 0x26, 0x8f, 0xa1, 0xb5, 0xf4, 0x67, + 0x25, 0x6a, 0x69, 0xca, 0x78, 0x71, 0xfc, 0x9d, 0xe1, 0xbe, 0x01, 0xa8, 0x67, 0x97, 0x7c, 0x5e, + 0x37, 0x66, 0x6b, 0xf4, 0x54, 0xbb, 0x55, 0xb3, 0x9e, 0x41, 0x4b, 0x6e, 0xb5, 0xde, 0x83, 0xb3, + 0xed, 0x7a, 0x3d, 0x05, 0xbb, 0x3f, 0x41, 0x5b, 0x8f, 0xec, 0x07, 0x06, 0x7e, 0x02, 0x76, 0x44, + 0x63, 0xb1, 0x94, 0xaa, 0x4e, 0x6d, 0xb9, 0x7f, 0x1b, 0x00, 0xb7, 0xf5, 0xfc, 0xee, 0xbb, 0xf3, + 0x67, 0x60, 0xe5, 0x0c, 0xb9, 0x6c, 0xf0, 0xde, 0xad, 0xf1, 0x24, 0x2e, 0x78, 0x0b, 0xc1, 0x33, + 0x0f, 0xf3, 0x04, 0x4e, 0x2e, 0xdf, 0x1b, 0x79, 0xeb, 0x80, 0xd0, 0x06, 0xc7, 0x7d, 0x01, 0xb6, + 0xda, 0x36, 0x51, 0x9f, 0x18, 0x0f, 0x3d, 0x06, 0xf2, 0x9b, 0x3c, 0x85, 0x4e, 0x75, 0x69, 0x5a, + 0x5d, 0xed, 0x70, 0xff, 0x33, 0xa0, 0xad, 0xf7, 0x6e, 0xaf, 0xba, 0x4b, 0xb0, 0x96, 0xb5, 0xba, + 0xa7, 0xbb, 0xdb, 0xda, 0xbf, 0x65, 0xc8, 0xd5, 0x70, 0x48, 0xa6, 0xd0, 0xf9, 0x56, 0xe9, 0x34, + 0x0e, 0xe9, 0x7c, 0xab, 0x78, 0xfa, 0x2e, 0xad, 0x7b, 0xef, 0xf2, 0xfc, 0x17, 0xe8, 0x54, 0x29, + 0xf6, 0x0c, 0xd8, 0x17, 0xcd, 0x01, 0xdb, 0xf7, 0x04, 0x35, 0x67, 0xee, 0x35, 0x40, 0xfd, 0x62, + 0xec, 0x59, 0xc6, 0x03, 0x63, 0xf0, 0x7e, 0x0f, 0xcd, 0xed, 0x1e, 0xfe, 0x01, 0x2d, 0xf9, 0xa6, + 0xd4, 0x92, 0x8c, 0x7b, 0x25, 0x91, 0xaf, 0x1a, 0xcf, 0xe0, 0xf1, 0xa1, 0x67, 0xb0, 0x7a, 0x04, + 0x87, 0xdf, 0xbc, 0x79, 0x1e, 0x53, 0x9e, 0x94, 0x41, 0x3f, 0xcc, 0xe6, 0x83, 0x64, 0x9d, 0x63, + 0x31, 0xc3, 0x28, 0xc6, 0x62, 0x30, 0xf5, 0x83, 0x82, 0x86, 0x83, 0xac, 0x88, 0xb0, 0xc0, 0x62, + 0xc0, 0x82, 0x29, 0x1f, 0x54, 0x81, 0x02, 0x5b, 0xfe, 0x09, 0x7f, 0xfd, 0x7f, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x91, 0x61, 0x36, 0x77, 0x97, 0x07, 0x00, 0x00, } diff --git a/orderer/sbft/simplebft/simplebft.proto b/orderer/sbft/simplebft/simplebft.proto index 9a3cde17eb5..13013003009 100644 --- a/orderer/sbft/simplebft/simplebft.proto +++ b/orderer/sbft/simplebft/simplebft.proto @@ -28,6 +28,11 @@ message Config { uint64 request_timeout_nsec = 5; }; +message MultiChainMsg { + string chainID = 1; + Msg msg = 2; +} + message Msg { oneof type { Request request = 1; diff --git a/orderer/sbft/simplebft/simplebft_bench_test.go b/orderer/sbft/simplebft/simplebft_bench_test.go index d1902809052..6797faf2457 100644 --- a/orderer/sbft/simplebft/simplebft_bench_test.go +++ b/orderer/sbft/simplebft/simplebft_bench_test.go @@ -26,7 +26,7 @@ func BenchmarkRequestN1(b *testing.B) { logging.SetLevel(logging.WARNING, "sbft") sys := newTestSystem(1) - s, _ := New(0, &Config{N: 1, F: 0, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, sys.NewAdapter(0)) + s, _ := New(0, chainId, &Config{N: 1, F: 0, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, sys.NewAdapter(0)) b.ResetTimer() for i := 0; i < b.N; i++ { s.Request([]byte{byte(i), byte(i >> 8), byte(i >> 16)}) @@ -44,7 +44,7 @@ func BenchmarkRequestN4(b *testing.B) { sys := newTestSystem(N) for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 11, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 11, RequestTimeoutNsec: 20000000000}, a) if err != nil { b.Fatal(err) } @@ -68,7 +68,7 @@ func BenchmarkRequestN80(b *testing.B) { sys := newTestSystem(N) for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: (N - 1) / 3, BatchDurationNsec: 2000000000, BatchSizeBytes: 11, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: (N - 1) / 3, BatchDurationNsec: 2000000000, BatchSizeBytes: 11, RequestTimeoutNsec: 20000000000}, a) if err != nil { b.Fatal(err) } diff --git a/orderer/sbft/simplebft/simplebft_test.go b/orderer/sbft/simplebft/simplebft_test.go index 705bf9926d8..3212a732aaf 100644 --- a/orderer/sbft/simplebft/simplebft_test.go +++ b/orderer/sbft/simplebft/simplebft_test.go @@ -23,10 +23,14 @@ import ( "math" + "fmt" + "strconv" + "github.com/golang/protobuf/proto" "github.com/op/go-logging" ) +const chainId = "id" const lowN uint64 = 4 //keep lowN greater or equal to 4 const highN uint64 = 10 //keep highN greater or equal to 10 @@ -44,7 +48,17 @@ func skipInShortMode(t *testing.T) { } } -func connectAll(sys *testSystem) { +func connectAll(sys *testSystem, chainIds []string) { + for _, chainId := range chainIds { + connectAllForChainId(sys, chainId) + } +} + +func connectAllForDefaultChain(sys *testSystem) { + connectAllForChainId(sys, chainId) +} + +func connectAllForChainId(sys *testSystem, chainId string) { // map iteration is non-deterministic, so use linear iteration instead max := uint64(0) for _, a := range sys.adapters { @@ -65,13 +79,63 @@ func connectAll(sys *testSystem) { continue } if a.id != b.id { - a.receiver.Connection(b.id) + a.receivers[chainId].Connection(b.id) } } } sys.Run() } +func TestMultiChain(t *testing.T) { + skipInShortMode(t) + N := lowN + M := uint64(5) + sys := newTestSystem(N) + chainIds := make([]string, 0, M) + var repls map[string][]*SBFT = map[string][]*SBFT{} + var adapters []*testSystemAdapter + for i := uint64(0); i < N; i++ { + a := sys.NewAdapter(i) + for j := uint64(0); j < M; j++ { + chainId := fmt.Sprintf("%d", j) + s, err := New(i, chainId, &Config{N: N, F: 0, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + if err != nil { + t.Fatal(err) + } + repls[chainId] = append(repls[chainId], s) + if uint64(len(chainIds)) < M { + chainIds = append(chainIds, chainId) + } + } + adapters = append(adapters, a) + } + connectAll(sys, chainIds) + r1 := []byte{1, 2, 3} + for i := uint64(0); i < N; i++ { + for j := uint64(0); j < M; j++ { + if j%uint64(2) == 0 { + chainId := fmt.Sprintf("%d", j) + repls[chainId][i].Request(r1) + } + } + } + sys.Run() + for _, a := range adapters { + for chainId := range a.batches { + // we check that if this is a chain where we sent a req then the req + // was written to the "ledger" + j, _ := strconv.ParseInt(chainId, 10, 64) + if j%2 == 0 && len(a.batches[chainId]) != 1 { + t.Fatalf("expected one batch on chain %s", chainId) + } + // in other cases, we should have at most an empty ledger + if j%2 != 0 && len(a.batches[chainId]) != 0 { + t.Fatalf("expected one batch on chain %s", chainId) + } + } + } +} + func TestSBFT(t *testing.T) { skipInShortMode(t) N := lowN @@ -80,14 +144,14 @@ func TestSBFT(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } repls = append(repls, s) adapters = append(adapters, a) } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() @@ -97,13 +161,13 @@ func TestSBFT(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -114,7 +178,7 @@ func TestQuorumSizes(t *testing.T) { for f := uint64(0); f <= uint64(math.Floor(float64(N-1)/float64(3))); f++ { sys := newTestSystem(N) a := sys.NewAdapter(0) - s, err := New(0, &Config{N: N, F: f, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(0, chainId, &Config{N: N, F: f, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -155,7 +219,7 @@ func TestSBFTDelayed(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -169,21 +233,21 @@ func TestSBFTDelayed(t *testing.T) { adapters[3].arrivals[i] = 200 * time.Millisecond } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} r2 := []byte{3, 1, 2} repls[0].Request(r1) repls[1].Request(r2) sys.Run() for i, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Errorf("expected execution of 2 batches on %d", i) continue } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -197,22 +261,22 @@ func TestN1(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 0, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 0, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } repls = append(repls, s) adapters = append(adapters, a) } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { + if len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } } @@ -226,7 +290,7 @@ func TestMonotonicViews(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -240,11 +304,11 @@ func TestMonotonicViews(t *testing.T) { view := repls[0].view testLog.Notice("TEST: Replica 0 is in view ", view) testLog.Notice("TEST: restarting replica 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } sys.Run() @@ -262,7 +326,7 @@ func TestByzPrimaryN4(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -293,17 +357,17 @@ func TestByzPrimaryN4(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r2}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r2}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed first") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed second") } } @@ -317,7 +381,7 @@ func TestNewPrimaryHandlingViewChange(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 2, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 2, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -348,14 +412,14 @@ func TestNewPrimaryHandlingViewChange(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) < 1 { + if len(a.batches[chainId]) < 1 { t.Fatal("expected execution of at least one batch") } - if a.batches[0].Payloads != nil && !reflect.DeepEqual(adapters[2].batches[0].Payloads, a.batches[0].Payloads) { + if a.batches[chainId][0].Payloads != nil && !reflect.DeepEqual(adapters[2].batches[chainId][0].Payloads, a.batches[chainId][0].Payloads) { t.Error("consensus violated on first batch at replica", a.id) } } @@ -369,7 +433,7 @@ func TestByzPrimaryBullyingSingleReplica(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -400,11 +464,11 @@ func TestByzPrimaryBullyingSingleReplica(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) repls[0].Request(r1) sys.Run() for _, a := range adapters { - if a.id != 1 && len(a.batches) != 1 { + if a.id != 1 && len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch at all except replica 1") } } @@ -418,7 +482,7 @@ func TestViewChange(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -436,15 +500,15 @@ func TestViewChange(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { + if len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } } @@ -458,7 +522,7 @@ func TestMsgReordering(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -489,15 +553,15 @@ func TestMsgReordering(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { + if len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } } @@ -511,7 +575,7 @@ func TestBacklogReordering(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -542,15 +606,15 @@ func TestBacklogReordering(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { + if len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } } @@ -564,7 +628,7 @@ func TestViewChangeWithRetransmission(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -582,15 +646,15 @@ func TestViewChangeWithRetransmission(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { + if len(a.batches[chainId]) != 1 { t.Fatal("expected execution of 1 batch") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } } @@ -604,7 +668,7 @@ func TestViewChangeXset(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -646,7 +710,7 @@ func TestViewChangeXset(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() @@ -661,13 +725,13 @@ func TestViewChangeXset(t *testing.T) { if i == 3 { continue } - if len(a.batches) != 2 { - t.Fatalf("expected execution of 1 batch: %v", a.batches) + if len(a.batches[chainId]) != 2 { + t.Fatalf("expected execution of 1 batch: %v", a.batches[chainId]) } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed first") } - if !reflect.DeepEqual([][]byte{r2}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed second") } } @@ -681,7 +745,7 @@ func TestRestart(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -689,7 +753,7 @@ func TestRestart(t *testing.T) { adapters = append(adapters, a) } - connectAll(sys) + connectAllForDefaultChain(sys) // move to view 1 for _, r := range repls { r.sendViewChange() @@ -701,11 +765,11 @@ func TestRestart(t *testing.T) { sys.Run() testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } @@ -715,13 +779,13 @@ func TestRestart(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatalf("expected execution of 2 batches, %d got %v", a.id, a.batches) } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -735,7 +799,7 @@ func TestAbdicatingPrimary(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -762,7 +826,7 @@ func TestAbdicatingPrimary(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) @@ -773,15 +837,15 @@ func TestAbdicatingPrimary(t *testing.T) { testLog.Notice("TEST: restarting connections from 0") for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } sys.Run() for _, a := range adapters { - if len(a.batches) != 1 { - t.Fatalf("expected execution of 1 batch, %d got %v", a.id, a.batches) + if len(a.batches[chainId]) != 1 { + t.Fatalf("expected execution of 1 batch, %d got %v", a.id, a.batches[chainId]) } } } @@ -794,7 +858,7 @@ func TestRestartAfterPrepare(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -814,11 +878,11 @@ func TestRestartAfterPrepare(t *testing.T) { if p := msg.msg.GetPrepare(); p != nil && p.Seq.Seq == 3 && !restarted { restarted = true testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } } @@ -827,7 +891,7 @@ func TestRestartAfterPrepare(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) // move to view 1 for _, r := range repls { r.sendViewChange() @@ -844,13 +908,13 @@ func TestRestartAfterPrepare(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -864,7 +928,7 @@ func TestRestartAfterCommit(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -884,11 +948,11 @@ func TestRestartAfterCommit(t *testing.T) { if c := msg.msg.GetCommit(); c != nil && c.Seq.Seq == 3 && !restarted { restarted = true testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } } @@ -897,7 +961,7 @@ func TestRestartAfterCommit(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) // move to view 1 for _, r := range repls { r.sendViewChange() @@ -914,13 +978,13 @@ func TestRestartAfterCommit(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -934,7 +998,7 @@ func TestRestartAfterCheckpoint(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -954,11 +1018,11 @@ func TestRestartAfterCheckpoint(t *testing.T) { if c := msg.msg.GetCheckpoint(); c != nil && c.Seq == 3 && !restarted { restarted = true testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } } @@ -967,7 +1031,7 @@ func TestRestartAfterCheckpoint(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) // move to view 1 for _, r := range repls { r.sendViewChange() @@ -984,13 +1048,13 @@ func TestRestartAfterCheckpoint(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -1004,7 +1068,7 @@ func TestErroneousViewChange(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1024,11 +1088,11 @@ func TestErroneousViewChange(t *testing.T) { if c := msg.msg.GetCheckpoint(); c != nil && c.Seq == 3 && !restarted { restarted = true testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } } @@ -1052,7 +1116,7 @@ func TestErroneousViewChange(t *testing.T) { continue } if a.id != b.id { - a.receiver.Connection(b.id) + a.receivers[chainId].Connection(b.id) } } } @@ -1074,13 +1138,13 @@ func TestErroneousViewChange(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -1094,7 +1158,7 @@ func TestRestartMissedViewChange(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1115,7 +1179,7 @@ func TestRestartMissedViewChange(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) @@ -1136,25 +1200,26 @@ func TestRestartMissedViewChange(t *testing.T) { disconnect = false testLog.Notice("restarting 0") - repls[0], _ = New(0, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) + repls[0], _ = New(0, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[0]) for _, a := range adapters { if a.id != 0 { - a.receiver.Connection(0) - adapters[0].receiver.Connection(a.id) + a.receivers[chainId].Connection(0) + adapters[0].receivers[chainId].Connection(a.id) } } r3 := []byte{3, 5, 2} repls[1].Request(r3) + sys.Run() for _, a := range adapters { - if len(a.batches) == 0 { + if len(a.batches[chainId]) == 0 { t.Fatalf("expected execution of some batches on %d", a.id) } - if !reflect.DeepEqual([][]byte{r3}, a.batches[len(a.batches)-1].Payloads) { - t.Errorf("wrong request executed on %d: %v", a.id, a.batches[2]) + if !reflect.DeepEqual([][]byte{r3}, a.batches[chainId][len(a.batches[chainId])-1].Payloads) { + t.Errorf("wrong request executed on %d: %v", a.id, a.batches[chainId][2]) } } } @@ -1167,7 +1232,7 @@ func TestFullBacklog(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1177,9 +1242,9 @@ func TestFullBacklog(t *testing.T) { r1 := []byte{1, 2, 3} - connectAll(sys) + connectAllForDefaultChain(sys) sys.enqueue(200*time.Millisecond, &testTimer{id: 999, tf: func() { - repls[0].sys.Send(&Msg{&Msg_Prepare{&Subject{Seq: &SeqView{Seq: 100}}}}, 1) + repls[0].sys.Send(chainId, &Msg{&Msg_Prepare{&Subject{Seq: &SeqView{Seq: 100}}}}, 1) }}) for i := 0; i < 10; i++ { sys.enqueue(time.Duration(i)*100*time.Millisecond, &testTimer{id: 999, tf: func() { @@ -1191,10 +1256,10 @@ func TestFullBacklog(t *testing.T) { t.Errorf("backlog too long: %d", len(repls[1].replicaState[0].backLog)) } for _, a := range adapters { - if len(a.batches) == 0 { + if len(a.batches[chainId]) == 0 { t.Fatalf("expected execution of batches on %d", a.id) } - bh := a.batches[len(a.batches)-1].DecodeHeader() + bh := a.batches[chainId][len(a.batches[chainId])-1].DecodeHeader() if bh.Seq != 10 { t.Errorf("wrong request executed on %d: %v", a.id, bh) } @@ -1209,7 +1274,7 @@ func TestHelloMsg(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 1, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1237,7 +1302,7 @@ func TestHelloMsg(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() @@ -1245,11 +1310,11 @@ func TestHelloMsg(t *testing.T) { phase = 2 //start delivering msgs to replica 1 testLog.Notice("restarting replica 1") - repls[1], _ = New(1, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[1]) + repls[1], _ = New(1, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, adapters[1]) for _, a := range adapters { if a.id != 1 { - a.receiver.Connection(1) - adapters[1].receiver.Connection(a.id) + a.receivers[chainId].Connection(1) + adapters[1].receivers[chainId].Connection(a.id) } } @@ -1262,7 +1327,7 @@ func TestHelloMsg(t *testing.T) { sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } } @@ -1276,7 +1341,7 @@ func TestViewChangeTimer(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1300,7 +1365,7 @@ func TestViewChangeTimer(t *testing.T) { return e, true } - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) @@ -1338,16 +1403,16 @@ func TestViewChangeTimer(t *testing.T) { repls[2].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { - t.Fatalf("%d: expected execution of 2 batches: %v", a.id, a.batches) + if len(a.batches[chainId]) != 2 { + t.Fatalf("%d: expected execution of 2 batches: %v", a.id, a.batches[chainId]) } if a.id != 3 { - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { - t.Errorf("%d: wrong request executed (1): %v", a.id, a.batches) + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { + t.Errorf("%d: wrong request executed (1): %v", a.id, a.batches[chainId]) } } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { - t.Errorf("%d: wrong request executed (2): %v", a.id, a.batches) + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { + t.Errorf("%d: wrong request executed (2): %v", a.id, a.batches[chainId]) } } } @@ -1360,7 +1425,7 @@ func TestResendViewChange(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 1, BatchDurationNsec: 2000000000, BatchSizeBytes: 10, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1390,7 +1455,7 @@ func TestResendViewChange(t *testing.T) { } sys.Run() - connectAll(sys) + connectAllForDefaultChain(sys) r1 := []byte{1, 2, 3} repls[0].Request(r1) sys.Run() @@ -1400,13 +1465,13 @@ func TestResendViewChange(t *testing.T) { repls[1].Request(r3) sys.Run() for _, a := range adapters { - if len(a.batches) != 2 { + if len(a.batches[chainId]) != 2 { t.Fatal("expected execution of 2 batches") } - if !reflect.DeepEqual([][]byte{r1}, a.batches[0].Payloads) { + if !reflect.DeepEqual([][]byte{r1}, a.batches[chainId][0].Payloads) { t.Error("wrong request executed (1)") } - if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[1].Payloads) { + if !reflect.DeepEqual([][]byte{r2, r3}, a.batches[chainId][1].Payloads) { t.Error("wrong request executed (2)") } } @@ -1421,7 +1486,7 @@ func TestTenReplicasBombedWithRequests(t *testing.T) { var adapters []*testSystemAdapter for i := uint64(0); i < N; i++ { a := sys.NewAdapter(i) - s, err := New(i, &Config{N: N, F: 3, BatchDurationNsec: 2000000000, BatchSizeBytes: 3, RequestTimeoutNsec: 20000000000}, a) + s, err := New(i, chainId, &Config{N: N, F: 3, BatchDurationNsec: 2000000000, BatchSizeBytes: 3, RequestTimeoutNsec: 20000000000}, a) if err != nil { t.Fatal(err) } @@ -1429,7 +1494,7 @@ func TestTenReplicasBombedWithRequests(t *testing.T) { adapters = append(adapters, a) } - connectAll(sys) + connectAllForDefaultChain(sys) for i := 0; i < requestNumber; i++ { r := []byte{byte(i), 2, 3} repls[2].Request(r) @@ -1437,7 +1502,7 @@ func TestTenReplicasBombedWithRequests(t *testing.T) { sys.Run() for _, a := range adapters { i := 0 - for _, b := range a.batches { + for _, b := range a.batches[chainId] { i = i + len(b.Payloads) } if i != requestNumber { diff --git a/orderer/sbft/simplebft/testsys_test.go b/orderer/sbft/simplebft/testsys_test.go index 9bb0bf6fa4c..a1b0ccae62c 100644 --- a/orderer/sbft/simplebft/testsys_test.go +++ b/orderer/sbft/simplebft/testsys_test.go @@ -33,19 +33,24 @@ import ( ) type testSystemAdapter struct { - id uint64 - sys *testSystem - receiver Receiver + id uint64 + sys *testSystem - batches []*Batch - arrivals map[uint64]time.Duration + // chainId to instance mapping + receivers map[string]Receiver + batches map[string][]*Batch persistence map[string][]byte + arrivals map[uint64]time.Duration + key *ecdsa.PrivateKey } -func (t *testSystemAdapter) SetReceiver(recv Receiver) { - if t.receiver != nil { +func (t *testSystemAdapter) AddReceiver(chainId string, recv Receiver) { + if t.receivers == nil { + t.receivers = make(map[string]Receiver) + } + if t.receivers[chainId] != nil { // remove all events for us t.sys.queue.filter(func(e testElem) bool { switch e := e.ev.(type) { @@ -62,7 +67,7 @@ func (t *testSystemAdapter) SetReceiver(recv Receiver) { }) } - t.receiver = recv + t.receivers[chainId] = recv } func (t *testSystemAdapter) getArrival(dest uint64) time.Duration { @@ -81,13 +86,14 @@ func (t *testSystemAdapter) getArrival(dest uint64) time.Duration { return arr } -func (t *testSystemAdapter) Send(msg *Msg, dest uint64) { +func (t *testSystemAdapter) Send(chainId string, msg *Msg, dest uint64) { arr := t.getArrival(dest) ev := &testMsgEvent{ inflight: arr, src: t.id, dst: dest, msg: msg, + chainId: chainId, } // simulate time for marshalling (and unmarshalling) bytes, _ := proto.Marshal(msg) @@ -100,6 +106,7 @@ type testMsgEvent struct { inflight time.Duration src, dst uint64 msg *Msg + chainId string } func (ev *testMsgEvent) Exec(t *testSystem) { @@ -108,7 +115,7 @@ func (ev *testMsgEvent) Exec(t *testSystem) { testLog.Errorf("message to non-existing %s", ev) return } - r.receiver.Receive(ev.msg, ev.src) + r.receivers[ev.chainId].Receive(ev.msg, ev.src) } func (ev *testMsgEvent) String() string { @@ -145,24 +152,32 @@ func (t *testSystemAdapter) Timer(d time.Duration, tf func()) Canceller { return tt } -func (t *testSystemAdapter) Deliver(batch *Batch) { - t.batches = append(t.batches, batch) +func (t *testSystemAdapter) Deliver(chainId string, batch *Batch) { + if t.batches == nil { + t.batches = make(map[string][]*Batch) + } + if t.batches[chainId] == nil { + t.batches[chainId] = make([]*Batch, 0, 1) + } + t.batches[chainId] = append(t.batches[chainId], batch) } -func (t *testSystemAdapter) Persist(key string, data proto.Message) { +func (t *testSystemAdapter) Persist(chainId string, key string, data proto.Message) { + compk := fmt.Sprintf("chain-%s-%s", chainId, key) if data == nil { - delete(t.persistence, key) + delete(t.persistence, compk) } else { bytes, err := proto.Marshal(data) if err != nil { panic(err) } - t.persistence[key] = bytes + t.persistence[compk] = bytes } } -func (t *testSystemAdapter) Restore(key string, out proto.Message) bool { - val, ok := t.persistence[key] +func (t *testSystemAdapter) Restore(chainId string, key string, out proto.Message) bool { + compk := fmt.Sprintf("chain-%s-%s", chainId, key) + val, ok := t.persistence[compk] if !ok { return false } @@ -170,11 +185,11 @@ func (t *testSystemAdapter) Restore(key string, out proto.Message) bool { return (err == nil) } -func (t *testSystemAdapter) LastBatch() *Batch { - if len(t.batches) == 0 { - return t.receiver.(*SBFT).makeBatch(0, nil, nil) +func (t *testSystemAdapter) LastBatch(chainId string) *Batch { + if len(t.batches[chainId]) == 0 { + return t.receivers[chainId].(*SBFT).makeBatch(0, nil, nil) } - return t.batches[len(t.batches)-1] + return t.batches[chainId][len(t.batches[chainId])-1] } func (t *testSystemAdapter) Sign(data []byte) []byte { @@ -207,7 +222,7 @@ func (t *testSystemAdapter) CheckSig(data []byte, src uint64, sig []byte) error return nil } -func (t *testSystemAdapter) Reconnect(replica uint64) { +func (t *testSystemAdapter) Reconnect(chainId string, replica uint64) { testLog.Infof("dropping connection from %d to %d", replica, t.id) t.sys.queue.filter(func(e testElem) bool { switch e := e.ev.(type) { @@ -221,7 +236,7 @@ func (t *testSystemAdapter) Reconnect(replica uint64) { arr := t.sys.adapters[replica].arrivals[t.id] * 10 t.sys.enqueue(arr, &testTimer{id: t.id, tf: func() { testLog.Infof("reconnecting %d to %d", replica, t.id) - t.sys.adapters[replica].receiver.Connection(t.id) + t.sys.adapters[replica].receivers[chainId].Connection(t.id) }}) } diff --git a/orderer/sbft/simplebft/viewchange.go b/orderer/sbft/simplebft/viewchange.go index 249c1164132..be8d03a83a6 100644 --- a/orderer/sbft/simplebft/viewchange.go +++ b/orderer/sbft/simplebft/viewchange.go @@ -38,7 +38,8 @@ func (s *SBFT) sendViewChange() { q = append(q, &s.cur.subject) } - checkpoint := *s.sys.LastBatch() + // TODO fix batch synchronization as we send no payload here + checkpoint := *s.sys.LastBatch(s.chainId) checkpoint.Payloads = nil // don't send the big payload vc := &ViewChange{ @@ -51,7 +52,7 @@ func (s *SBFT) sendViewChange() { s.viewChangeTimer.Cancel() s.cur.timeout.Cancel() - s.sys.Persist(viewchange, svc) + s.sys.Persist(s.chainId, viewchange, svc) s.broadcast(&Msg{&Msg_ViewChange{svc}}) } diff --git a/orderer/sbft/main/testdata/cert1.pem b/orderer/sbft/testdata/cert1.pem similarity index 100% rename from orderer/sbft/main/testdata/cert1.pem rename to orderer/sbft/testdata/cert1.pem diff --git a/orderer/sbft/main/testdata/config.json b/orderer/sbft/testdata/config.json similarity index 100% rename from orderer/sbft/main/testdata/config.json rename to orderer/sbft/testdata/config.json diff --git a/orderer/sbft/main/testdata/key.pem b/orderer/sbft/testdata/key.pem similarity index 100% rename from orderer/sbft/main/testdata/key.pem rename to orderer/sbft/testdata/key.pem diff --git a/orderer/sbft/main/sbft_test.go b/orderer/sbft_test.go similarity index 54% rename from orderer/sbft/main/sbft_test.go rename to orderer/sbft_test.go index cd9f37e2f8d..abc07a568b2 100644 --- a/orderer/sbft/main/sbft_test.go +++ b/orderer/sbft_test.go @@ -23,8 +23,22 @@ import ( "testing" "time" + "net" + + "bytes" + "github.com/golang/protobuf/proto" + "github.com/hyperledger/fabric/common/localmsp" + "github.com/hyperledger/fabric/core/peer/msp" "github.com/hyperledger/fabric/orderer/common/bootstrap/provisional" + "github.com/hyperledger/fabric/orderer/ledger" + "github.com/hyperledger/fabric/orderer/ledger/ram" + "github.com/hyperledger/fabric/orderer/localconfig" + "github.com/hyperledger/fabric/orderer/multichain" + "github.com/hyperledger/fabric/orderer/sbft" + "github.com/hyperledger/fabric/orderer/sbft/backend" + "github.com/hyperledger/fabric/orderer/sbft/crypto" + "github.com/hyperledger/fabric/orderer/sbft/simplebft" cb "github.com/hyperledger/fabric/protos/common" ab "github.com/hyperledger/fabric/protos/orderer" "github.com/hyperledger/fabric/protos/utils" @@ -39,50 +53,87 @@ const sent byte = 1 const neededUpdates = 2 const neededSent = 1 +var testData = []byte{0, 1, 2, 3} + +const sbftName = "sbft" + +type item struct { + itemtype byte + payload []byte +} + func TestSbftPeer(t *testing.T) { + pwd, err := os.Getwd() + if err != nil { + panic(err) + } + t.Parallel() skipInShortMode(t) - tempDir, err := ioutil.TempDir("", "sbft_test") + logging.SetLevel(logging.DEBUG, "") + + // Start SBFT + dataTmpDir, err := ioutil.TempDir("", "sbft_test") if err != nil { panic("Failed to create a temporary directory") } // We only need the path as the directory will be created - // by the peer - TODO: modify sbft to tolerate an existing - // directory - os.RemoveAll(tempDir) + // by the peer + os.RemoveAll(dataTmpDir) defer func() { - os.RemoveAll(tempDir) + os.RemoveAll(dataTmpDir) }() - c := flags{init: "testdata/config.json", - genesisFile: fmt.Sprintf("%s_%s", tempDir, "genesis"), - listenAddr: ":6101", - grpcAddr: ":7101", - certFile: "testdata/cert1.pem", - keyFile: "testdata/key.pem", - dataDir: tempDir} - - logger.Info("Initialization of instance.") - err = initInstance(c) - if err != nil { - t.Errorf("Initialization failed: %s", err) - return + peers := make(map[string][]byte) + peers["6101"], err = crypto.ParseCertPEM("sbft/testdata/cert1.pem") + panicOnError(err) + listenAddr := ":6101" + certFile := "sbft/testdata/cert1.pem" + keyFile := "sbft/testdata/key.pem" + cons := &simplebft.Config{N: 1, F: 0, BatchDurationNsec: 1000, BatchSizeBytes: 1000000000, RequestTimeoutNsec: 1000000000} + c := &sbft.ConsensusConfig{Consensus: cons, Peers: peers} + sc := &backend.StackConfig{listenAddr, certFile, keyFile, dataTmpDir} + sbftConsenter := sbft.New(c, sc) + <-time.After(5 * time.Second) + // End SBFT + + // Start GRPC + logger.Info("Creating a GRPC server.") + conf := config.Load() + conf.Genesis.OrdererType = sbftName + conf.General.LocalMSPDir = pwd + "/../msp/sampleconfig" + lf := newRAMLedgerFactory(conf) + consenters := make(map[string]multichain.Consenter) + consenters[sbftName] = sbftConsenter + + err = mspmgmt.LoadLocalMsp(conf.General.LocalMSPDir) + if err != nil { // Handle errors reading the config file + panic(fmt.Errorf("Failed initializing crypto [%s]", err)) } - logging.SetLevel(logging.DEBUG, "") + signer := localmsp.NewSigner() + manager := multichain.NewManagerImpl(lf, consenters, signer) - logger.Info("Starting an instance in the background.") - go serve(c) - <-time.After(5 * time.Second) + server := NewServer(manager, int(conf.General.QueueSize), int(conf.General.MaxWindowSize)) + grpcServer := grpc.NewServer() + grpcAddr := fmt.Sprintf("%s:%d", conf.General.ListenAddress, conf.General.ListenPort) + lis, err := net.Listen("tcp", grpcAddr) + if err != nil { + panic("Listening on the given port failed.") + } + ab.RegisterAtomicBroadcastServer(grpcServer, server) + go grpcServer.Serve(lis) + // End GRPC + // Start Test Setup logger.Info("Creating an Atomic Broadcast GRPC connection.") timeout := 4 * time.Second - clientconn, err := grpc.Dial(":7101", grpc.WithBlock(), grpc.WithTimeout(timeout), grpc.WithInsecure()) + clientconn, err := grpc.Dial(grpcAddr, grpc.WithBlock(), grpc.WithTimeout(timeout), grpc.WithInsecure()) if err != nil { t.Errorf("Failed to connect to GRPC: %s", err) return } client := ab.NewAtomicBroadcastClient(clientconn) - resultch := make(chan byte) + resultch := make(chan item) errorch := make(chan error) logger.Info("Starting a goroutine waiting for ledger updates.") @@ -90,11 +141,12 @@ func TestSbftPeer(t *testing.T) { logger.Info("Starting a single broadcast sender goroutine.") go broadcastSender(t, resultch, errorch, client) + // End Test Setup checkResults(t, resultch, errorch) } -func checkResults(t *testing.T, resultch chan byte, errorch chan error) { +func checkResults(t *testing.T, resultch chan item, errorch chan error) { l := len(errorch) for i := 0; i < l; i++ { errres := <-errorch @@ -103,12 +155,16 @@ func checkResults(t *testing.T, resultch chan byte, errorch chan error) { updates := 0 sentBroadcast := 0 - for i := 0; i < 3; i++ { + testDataReceived := false + for i := 0; i < neededUpdates+neededSent; i++ { select { case result := <-resultch: - switch result { + switch result.itemtype { case update: updates++ + if bytes.Equal(result.payload, testData) { + testDataReceived = true + } case sent: sentBroadcast++ } @@ -120,12 +176,14 @@ func checkResults(t *testing.T, resultch chan byte, errorch chan error) { t.Errorf("We did not get all the ledger updates.") } else if sentBroadcast != neededSent { t.Errorf("We were unable to send all the broadcasts.") + } else if !testDataReceived { + t.Errorf("We did not receive an update containing the test data sent in a broadcast.") } else { logger.Info("Successfully sent and received everything.") } } -func updateReceiver(t *testing.T, resultch chan byte, errorch chan error, client ab.AtomicBroadcastClient) { +func updateReceiver(t *testing.T, resultch chan item, errorch chan error, client ab.AtomicBroadcastClient) { logger.Info("{Update Receiver} Creating a ledger update delivery stream.") dstream, err := client.Deliver(context.Background()) if err != nil { @@ -152,14 +210,18 @@ func updateReceiver(t *testing.T, resultch chan byte, errorch chan error, client return } logger.Info("{Update Receiver} Listening to ledger updates.") - for i := 0; i < 2; i++ { + for i := 0; i < neededUpdates; { m, inerr := dstream.Recv() logger.Info("{Update Receiver} Got message: ", m, "err:", inerr) if inerr != nil { errorch <- fmt.Errorf("Failed to receive consensus: %s", inerr) return } - b := m.Type.(*ab.DeliverResponse_Block) + b, ok := m.Type.(*ab.DeliverResponse_Block) + if !ok { + logger.Info("{Update Receiver} Received s status message.") + continue + } logger.Info("{Update Receiver} Received a ledger update.") for i, tx := range b.Block.Data.Data { pl := &cb.Payload{} @@ -168,14 +230,15 @@ func updateReceiver(t *testing.T, resultch chan byte, errorch chan error, client merr2 := proto.Unmarshal(e.Payload, pl) if merr1 == nil && merr2 == nil { logger.Infof("{Update Receiver} %d - %v", i+1, pl.Data) + resultch <- item{itemtype: update, payload: pl.Data} } } - resultch <- update + i++ } logger.Info("{Update Receiver} Exiting...") } -func broadcastSender(t *testing.T, resultch chan byte, errorch chan error, client ab.AtomicBroadcastClient) { +func broadcastSender(t *testing.T, resultch chan item, errorch chan error, client ab.AtomicBroadcastClient) { logger.Info("{Broadcast Sender} Waiting before sending.") <-time.After(5 * time.Second) bstream, err := client.Broadcast(context.Background()) @@ -183,8 +246,9 @@ func broadcastSender(t *testing.T, resultch chan byte, errorch chan error, clien errorch <- fmt.Errorf("Failed to get broadcast stream: %s", err) return } - bs := []byte{0, 1, 2, 3} - pl := &cb.Payload{Data: bs} + h := &cb.Header{ChainHeader: &cb.ChainHeader{ChainID: provisional.TestChainID}, SignatureHeader: &cb.SignatureHeader{}} + bs := testData + pl := &cb.Payload{Data: bs, Header: h} mpl, err := proto.Marshal(pl) if err != nil { panic("Failed to marshal payload.") @@ -192,5 +256,19 @@ func broadcastSender(t *testing.T, resultch chan byte, errorch chan error, clien bstream.Send(&cb.Envelope{Payload: mpl}) logger.Infof("{Broadcast Sender} Broadcast sent: %v", bs) logger.Info("{Broadcast Sender} Exiting...") - resultch <- sent + resultch <- item{itemtype: sent, payload: mpl} +} + +func newRAMLedgerFactory(conf *config.TopLevel) ordererledger.Factory { + rlf := ramledger.New(10) + genesisBlock := provisional.New(conf).GenesisBlock() + rl, err := rlf.GetOrCreate(provisional.TestChainID) + if err != nil { + panic(err) + } + err = rl.Append(genesisBlock) + if err != nil { + panic(err) + } + return rlf }