From 2b1cd0ed09baf1ad5ed60b2f6384c41ce6866cfd Mon Sep 17 00:00:00 2001 From: Yacov Manevich Date: Tue, 28 Nov 2023 22:04:59 +0100 Subject: [PATCH 1/4] Raft to BFT migration Signed-off-by: Yacov Manevich --- integration/raft/migration_test.go | 198 +++++++++++++++++- .../common/msgprocessor/maintenancefilter.go | 4 +- orderer/consensus/etcdraft/chain.go | 20 +- orderer/consensus/etcdraft/util.go | 26 +-- 4 files changed, 225 insertions(+), 23 deletions(-) diff --git a/integration/raft/migration_test.go b/integration/raft/migration_test.go index 5573a07724b..c489b38f394 100644 --- a/integration/raft/migration_test.go +++ b/integration/raft/migration_test.go @@ -15,6 +15,10 @@ import ( "syscall" "time" + "github.com/SmartBFT-Go/consensus/pkg/types" + "github.com/hyperledger/fabric-protos-go/orderer/smartbft" + "github.com/hyperledger/fabric/common/policies" + docker "github.com/fsouza/go-dockerclient" "github.com/golang/protobuf/proto" "github.com/hyperledger/fabric-protos-go/common" @@ -39,10 +43,12 @@ var _ = Describe("ConsensusTypeMigration", func() { client *docker.Client network *nwo.Network - o1Proc, o2Proc, o3Proc ifrit.Process + o1Proc, o2Proc, o3Proc, o4Proc ifrit.Process o1Runner *ginkgomon.Runner o2Runner *ginkgomon.Runner + o3Runner *ginkgomon.Runner + o4Runner *ginkgomon.Runner ) BeforeEach(func() { @@ -55,7 +61,7 @@ var _ = Describe("ConsensusTypeMigration", func() { }) AfterEach(func() { - for _, oProc := range []ifrit.Process{o1Proc, o2Proc, o3Proc} { + for _, oProc := range []ifrit.Process{o1Proc, o2Proc, o3Proc, o4Proc} { if oProc != nil { oProc.Signal(syscall.SIGTERM) Eventually(oProc.Wait(), network.EventuallyTimeout).Should(Receive()) @@ -69,6 +75,125 @@ var _ = Describe("ConsensusTypeMigration", func() { _ = os.RemoveAll(testDir) }) + Describe("Raft to BFT migration", func() { + It("migrates from Raft to BFT", func() { + networkConfig := nwo.MultiNodeEtcdRaft() + networkConfig.Orderers = append(networkConfig.Orderers, &nwo.Orderer{Name: "orderer4", Organization: "OrdererOrg"}) + networkConfig.Profiles[0].Orderers = []string{"orderer1", "orderer2", "orderer3", "orderer4"} + + network = nwo.New(networkConfig, testDir, client, StartPort(), components) + + o1, o2, o3, o4 := network.Orderer("orderer1"), network.Orderer("orderer2"), network.Orderer("orderer3"), network.Orderer("orderer4") + network.GenerateConfigTree() + network.Bootstrap() + + runOrderers := func() { + o1Runner = network.OrdererRunner(o1) + o2Runner = network.OrdererRunner(o2) + o3Runner = network.OrdererRunner(o3) + o4Runner = network.OrdererRunner(o4) + + o1Runner.Command.Env = append(o1Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o2Runner.Command.Env = append(o2Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o3Runner.Command.Env = append(o3Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o4Runner.Command.Env = append(o4Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + + o1Proc = ifrit.Invoke(o1Runner) + o2Proc = ifrit.Invoke(o2Runner) + o3Proc = ifrit.Invoke(o3Runner) + o4Proc = ifrit.Invoke(o4Runner) + + Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o2Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o3Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o4Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + } + + runOrderers() + + channelparticipation.JoinOrderersAppChannelCluster(network, "testchannel", o1, o2, o3, o4) + FindLeader([]*ginkgomon.Runner{o1Runner, o2Runner, o3Runner, o4Runner}) + + By("performing operation with orderer1") + env := CreateBroadcastEnvelope(network, o1, "testchannel", []byte("foo")) + resp, err := ordererclient.Broadcast(network, o1, env) + Expect(err).NotTo(HaveOccurred()) + Expect(resp.Status).To(Equal(common.Status_SUCCESS)) + + block := FetchBlock(network, o1, 1, "testchannel") + Expect(block).NotTo(BeNil()) + + By("Change to maintenance mode") + + peer := network.Peer("Org1", "peer0") + orderer := network.Orderer("orderer1") + + addBFTInConfigTree(network, peer, orderer, "testchannel") + + By("Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") + config, updatedConfig := prepareTransition(network, peer, orderer, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) + + bftMetadata := protoutil.MarshalOrPanic(&smartbft.Options{ + RequestBatchMaxCount: types.DefaultConfig.RequestBatchMaxCount, + RequestBatchMaxBytes: types.DefaultConfig.RequestBatchMaxBytes, + RequestBatchMaxInterval: types.DefaultConfig.RequestBatchMaxInterval.String(), + IncomingMessageBufferSize: types.DefaultConfig.IncomingMessageBufferSize, + RequestPoolSize: types.DefaultConfig.RequestPoolSize, + RequestForwardTimeout: types.DefaultConfig.RequestForwardTimeout.String(), + RequestComplainTimeout: types.DefaultConfig.RequestComplainTimeout.String(), + RequestAutoRemoveTimeout: types.DefaultConfig.RequestAutoRemoveTimeout.String(), + ViewChangeResendInterval: types.DefaultConfig.ViewChangeResendInterval.String(), + ViewChangeTimeout: types.DefaultConfig.ViewChangeTimeout.String(), + LeaderHeartbeatTimeout: types.DefaultConfig.LeaderHeartbeatTimeout.String(), + LeaderHeartbeatCount: types.DefaultConfig.LeaderHeartbeatCount, + CollectTimeout: types.DefaultConfig.CollectTimeout.String(), + SyncOnStart: types.DefaultConfig.SyncOnStart, + SpeedUpViewChange: types.DefaultConfig.SpeedUpViewChange, + }) + config, updatedConfig = prepareTransition(network, peer, orderer, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) + + time.Sleep(time.Second) // TODO: check block was committed in all orderers + + for _, oProc := range []ifrit.Process{o1Proc, o2Proc, o3Proc, o4Proc} { + if oProc != nil { + oProc.Signal(syscall.SIGTERM) + Eventually(oProc.Wait(), network.EventuallyTimeout).Should(Receive()) + } + } + + runOrderers() + + // TODO: + By("Waiting for followers to see the leader, again") + Eventually(o2Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) + Eventually(o3Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) + Eventually(o4Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) + + // Exit maintenance mode + + config, updatedConfig = prepareTransition(network, peer, orderer, "testchannel", + "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_NORMAL) + nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) + + // Now, run a transaction to ensure BFT works. + env = CreateBroadcastEnvelope(network, o1, "testchannel", []byte("foo")) + resp, err = ordererclient.Broadcast(network, o1, env) + Expect(err).NotTo(HaveOccurred()) + Expect(resp.Status).To(Equal(common.Status_SUCCESS)) + + time.Sleep(time.Second * 5) + + // TODO: check block was successfully committed in all orderers + }) + }) + // These tests execute the migration config updates on an etcdraft based system, but do not restart the orderer // to a "future-type" consensus-type that currently does not exist. However, these tests try to maintain as much as // possible the testing infrastructure for consensus-type migration that existed when "solo" and "kafka" were still @@ -604,3 +729,72 @@ func createDeliverEnvelope(n *nwo.Network, signer *nwo.SigningIdentity, blkNum u return env } + +func addBFTInConfigTree(network *nwo.Network, peer *nwo.Peer, orderer *nwo.Orderer, channel string) { + config := nwo.GetConfig(network, peer, orderer, channel) + + updatedConfig := proto.Clone(config).(*common.Config) + + orderersVal := &common.Orderers{ + ConsenterMapping: computeConsenterMappings(network), + } + + policies.EncodeBFTBlockVerificationPolicy(orderersVal.ConsenterMapping, updatedConfig.ChannelGroup.Groups["Orderer"]) + + updatedConfig.ChannelGroup.Groups["Orderer"].Values["Orderers"] = &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(orderersVal), + ModPolicy: "/Channel/Orderer/Admins", + } + + nwo.UpdateOrdererConfig(network, orderer, channel, config, updatedConfig, peer, orderer) +} + +func computeConsenterMappings(network *nwo.Network) []*common.Consenter { + var consenters []*common.Consenter + + for i, orderer := range network.Orderers { + ecertPath := network.OrdererCert(orderer) + ecert, err := os.ReadFile(ecertPath) + Expect(err).To(Not(HaveOccurred())) + + tlsDir := network.OrdererLocalTLSDir(orderer) + tlsPath := filepath.Join(tlsDir, "server.crt") + tlsCert, err := os.ReadFile(tlsPath) + Expect(err).To(Not(HaveOccurred())) + + consenters = append(consenters, &common.Consenter{ + ServerTlsCert: tlsCert, + ClientTlsCert: tlsCert, + MspId: network.Organization(orderer.Organization).MSPID, + Host: "127.0.0.1", + Port: uint32(network.PortsByOrdererID[orderer.ID()][nwo.ClusterPort]), + Id: uint32(i + 1), + Identity: ecert, + }) + } + + return consenters +} + +func CreateBroadcastEnvelope(n *nwo.Network, entity interface{}, channel string, data []byte) *common.Envelope { + var signer *nwo.SigningIdentity + switch creator := entity.(type) { + case *nwo.Peer: + signer = n.PeerUserSigner(creator, "Admin") + case *nwo.Orderer: + signer = n.OrdererUserSigner(creator, "Admin") + } + Expect(signer).NotTo(BeNil()) + + env, err := protoutil.CreateSignedEnvelope( + common.HeaderType_ENDORSER_TRANSACTION, + channel, + signer, + &common.Envelope{Payload: data}, + 0, + 0, + ) + Expect(err).NotTo(HaveOccurred()) + + return env +} diff --git a/orderer/common/msgprocessor/maintenancefilter.go b/orderer/common/msgprocessor/maintenancefilter.go index fa068dd703f..23f5eb617ab 100644 --- a/orderer/common/msgprocessor/maintenancefilter.go +++ b/orderer/common/msgprocessor/maintenancefilter.go @@ -46,9 +46,7 @@ func NewMaintenanceFilter(support MaintenanceFilterSupport, bccsp bccsp.BCCSP) * bccsp: bccsp, } mf.permittedTargetConsensusTypes["etcdraft"] = true - // Until we have a BFT consensus type, we use this for integration testing of consensus-type migration. - // Caution: proposing a config block with this type will cause panic. - mf.permittedTargetConsensusTypes["testing-only"] = true + mf.permittedTargetConsensusTypes["BFT"] = true return mf } diff --git a/orderer/consensus/etcdraft/chain.go b/orderer/consensus/etcdraft/chain.go index 371922822e6..5fd1db3c21e 100644 --- a/orderer/consensus/etcdraft/chain.go +++ b/orderer/consensus/etcdraft/chain.go @@ -1115,12 +1115,17 @@ func (c *Chain) commitBlock(block *common.Block) { func (c *Chain) detectConfChange(block *common.Block) *MembershipChanges { // If config is targeting THIS channel, inspect consenter set and // propose raft ConfChange if it adds/removes node. - configMetadata := c.newConfigMetadata(block) + configMetadata, consensusType := c.newConfigMetadata(block) if configMetadata == nil { return nil } + if consensusType.Type != "etcdraft" { + c.logger.Infof("Detected migration to %s", consensusType.Type) + return nil + } + if configMetadata.Options != nil && configMetadata.Options.SnapshotIntervalSize != 0 && configMetadata.Options.SnapshotIntervalSize != c.sizeLimit { @@ -1425,12 +1430,12 @@ func (c *Chain) getInFlightConfChange() *raftpb.ConfChange { } // newMetadata extract config metadata from the configuration block -func (c *Chain) newConfigMetadata(block *common.Block) *etcdraft.ConfigMetadata { - metadata, err := ConsensusMetadataFromConfigBlock(block) +func (c *Chain) newConfigMetadata(block *common.Block) (*etcdraft.ConfigMetadata, *orderer.ConsensusType) { + metadata, consensusType, err := ConsensusMetadataFromConfigBlock(block) if err != nil { c.logger.Panicf("error reading consensus metadata: %s", err) } - return metadata + return metadata, consensusType } // ValidateConsensusMetadata determines the validity of a @@ -1446,6 +1451,11 @@ func (c *Chain) ValidateConsensusMetadata(oldOrdererConfig, newOrdererConfig cha return nil } + if newOrdererConfig.ConsensusType() != "etcdraft" { + // This is a migration, so we don't know how to validate this config change. + return nil + } + if oldOrdererConfig == nil { c.logger.Panic("Programming Error: ValidateConsensusMetadata called with nil old channel config") return nil @@ -1572,7 +1582,7 @@ func (c *Chain) checkForEvictionNCertRotation(env *common.Envelope) bool { return false } - configMeta, err := MetadataFromConfigUpdate(configUpdate) + configMeta, _, err := MetadataFromConfigUpdate(configUpdate) if err != nil || configMeta == nil { c.logger.Warnf("could not read config metadata: %s", err) return false diff --git a/orderer/consensus/etcdraft/util.go b/orderer/consensus/etcdraft/util.go index 53ec773313c..f734fc3c1b4 100644 --- a/orderer/consensus/etcdraft/util.go +++ b/orderer/consensus/etcdraft/util.go @@ -84,22 +84,22 @@ func MetadataHasDuplication(md *etcdraft.ConfigMetadata) error { } // MetadataFromConfigValue reads and translates configuration updates from config value into raft metadata -func MetadataFromConfigValue(configValue *common.ConfigValue) (*etcdraft.ConfigMetadata, error) { +func MetadataFromConfigValue(configValue *common.ConfigValue) (*etcdraft.ConfigMetadata, *orderer.ConsensusType, error) { consensusTypeValue := &orderer.ConsensusType{} if err := proto.Unmarshal(configValue.Value, consensusTypeValue); err != nil { - return nil, errors.Wrap(err, "failed to unmarshal consensusType config update") + return nil, nil, errors.Wrap(err, "failed to unmarshal consensusType config update") } updatedMetadata := &etcdraft.ConfigMetadata{} if err := proto.Unmarshal(consensusTypeValue.Metadata, updatedMetadata); err != nil { - return nil, errors.Wrap(err, "failed to unmarshal updated (new) etcdraft metadata configuration") + return nil, nil, errors.Wrap(err, "failed to unmarshal updated (new) etcdraft metadata configuration") } - return updatedMetadata, nil + return updatedMetadata, consensusTypeValue, nil } // MetadataFromConfigUpdate extracts consensus metadata from config update -func MetadataFromConfigUpdate(update *common.ConfigUpdate) (*etcdraft.ConfigMetadata, error) { +func MetadataFromConfigUpdate(update *common.ConfigUpdate) (*etcdraft.ConfigMetadata, *orderer.ConsensusType, error) { var baseVersion uint64 if update.ReadSet != nil && update.ReadSet.Groups != nil { if ordererConfigGroup, ok := update.ReadSet.Groups["Orderer"]; ok { @@ -115,13 +115,13 @@ func MetadataFromConfigUpdate(update *common.ConfigUpdate) (*etcdraft.ConfigMeta if baseVersion == val.Version { // Only if the version in the write set differs from the read-set // should we consider this to be an update to the consensus type - return nil, nil + return nil, nil, nil } return MetadataFromConfigValue(val) } } } - return nil, nil + return nil, nil, nil } // ConfigChannelHeader expects a config block and returns the header type @@ -168,28 +168,28 @@ func ConfigEnvelopeFromBlock(block *common.Block) (*common.Envelope, error) { } // ConsensusMetadataFromConfigBlock reads consensus metadata updates from the configuration block -func ConsensusMetadataFromConfigBlock(block *common.Block) (*etcdraft.ConfigMetadata, error) { +func ConsensusMetadataFromConfigBlock(block *common.Block) (*etcdraft.ConfigMetadata, *orderer.ConsensusType, error) { if block == nil { - return nil, errors.New("nil block") + return nil, nil, errors.New("nil block") } if !protoutil.IsConfigBlock(block) { - return nil, errors.New("not a config block") + return nil, nil, errors.New("not a config block") } configEnvelope, err := ConfigEnvelopeFromBlock(block) if err != nil { - return nil, errors.Wrap(err, "cannot read config update") + return nil, nil, errors.Wrap(err, "cannot read config update") } payload, err := protoutil.UnmarshalPayload(configEnvelope.Payload) if err != nil { - return nil, errors.Wrap(err, "failed to extract payload from config envelope") + return nil, nil, errors.Wrap(err, "failed to extract payload from config envelope") } // get config update configUpdate, err := configtx.UnmarshalConfigUpdateFromPayload(payload) if err != nil { - return nil, errors.Wrap(err, "could not read config update") + return nil, nil, errors.Wrap(err, "could not read config update") } return MetadataFromConfigUpdate(configUpdate) From 6a21a90d4517055b4486ad8fcd195674fa3af5d5 Mon Sep 17 00:00:00 2001 From: May Rosenbaum Date: Wed, 29 Nov 2023 15:48:56 +0200 Subject: [PATCH 2/4] Raft to BFT migration: functions + tests updates Signed-off-by: May Rosenbaum --- integration/raft/migration_test.go | 487 +++++++----------- .../common/msgprocessor/maintenancefilter.go | 75 ++- .../msgprocessor/maintenancefilter_test.go | 208 ++++++-- orderer/consensus/etcdraft/chain.go | 29 +- orderer/consensus/etcdraft/chain_test.go | 42 ++ orderer/consensus/etcdraft/util.go | 7 + orderer/consensus/etcdraft/util_test.go | 17 + orderer/consensus/etcdraft/validator_test.go | 3 + 8 files changed, 508 insertions(+), 360 deletions(-) diff --git a/integration/raft/migration_test.go b/integration/raft/migration_test.go index c489b38f394..d6cf7b1f275 100644 --- a/integration/raft/migration_test.go +++ b/integration/raft/migration_test.go @@ -23,7 +23,6 @@ import ( "github.com/golang/protobuf/proto" "github.com/hyperledger/fabric-protos-go/common" protosorderer "github.com/hyperledger/fabric-protos-go/orderer" - protosraft "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" "github.com/hyperledger/fabric/integration/channelparticipation" "github.com/hyperledger/fabric/integration/nwo" "github.com/hyperledger/fabric/integration/nwo/commands" @@ -75,6 +74,10 @@ var _ = Describe("ConsensusTypeMigration", func() { _ = os.RemoveAll(testDir) }) + // This test executes the migration on an etcdraft based system. + // The migration includes change to maintenance mode, config updates needed for migration to a bft based system and + // exiting maintenance mode back to the normal state. + // This test restarts the orderers and ensures they operate under the new configuration. Describe("Raft to BFT migration", func() { It("migrates from Raft to BFT", func() { networkConfig := nwo.MultiNodeEtcdRaft() @@ -126,39 +129,25 @@ var _ = Describe("ConsensusTypeMigration", func() { By("Change to maintenance mode") peer := network.Peer("Org1", "peer0") - orderer := network.Orderer("orderer1") - - addBFTInConfigTree(network, peer, orderer, "testchannel") By("Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") - config, updatedConfig := prepareTransition(network, peer, orderer, "testchannel", + config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) - - bftMetadata := protoutil.MarshalOrPanic(&smartbft.Options{ - RequestBatchMaxCount: types.DefaultConfig.RequestBatchMaxCount, - RequestBatchMaxBytes: types.DefaultConfig.RequestBatchMaxBytes, - RequestBatchMaxInterval: types.DefaultConfig.RequestBatchMaxInterval.String(), - IncomingMessageBufferSize: types.DefaultConfig.IncomingMessageBufferSize, - RequestPoolSize: types.DefaultConfig.RequestPoolSize, - RequestForwardTimeout: types.DefaultConfig.RequestForwardTimeout.String(), - RequestComplainTimeout: types.DefaultConfig.RequestComplainTimeout.String(), - RequestAutoRemoveTimeout: types.DefaultConfig.RequestAutoRemoveTimeout.String(), - ViewChangeResendInterval: types.DefaultConfig.ViewChangeResendInterval.String(), - ViewChangeTimeout: types.DefaultConfig.ViewChangeTimeout.String(), - LeaderHeartbeatTimeout: types.DefaultConfig.LeaderHeartbeatTimeout.String(), - LeaderHeartbeatCount: types.DefaultConfig.LeaderHeartbeatCount, - CollectTimeout: types.DefaultConfig.CollectTimeout.String(), - SyncOnStart: types.DefaultConfig.SyncOnStart, - SpeedUpViewChange: types.DefaultConfig.SpeedUpViewChange, - }) - config, updatedConfig = prepareTransition(network, peer, orderer, "testchannel", + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + + By("Config updates: migration from Raft to BFT") + + bftMetadata := protoutil.MarshalOrPanic(prepareBftMetadata()) + + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) + currentBlockNumber := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - time.Sleep(time.Second) // TODO: check block was committed in all orderers + // check block was committed in all orderers + assertBlockReceptionInAllOrderers(network.Orderers[1:], peer, network, "testchannel", currentBlockNumber) for _, oProc := range []ifrit.Process{o1Proc, o2Proc, o3Proc, o4Proc} { if oProc != nil { @@ -169,18 +158,17 @@ var _ = Describe("ConsensusTypeMigration", func() { runOrderers() - // TODO: By("Waiting for followers to see the leader, again") Eventually(o2Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) Eventually(o3Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) Eventually(o4Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) // Exit maintenance mode - - config, updatedConfig = prepareTransition(network, peer, orderer, "testchannel", + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE, "BFT", bftMetadata, protosorderer.ConsensusType_STATE_NORMAL) - nwo.UpdateOrdererConfig(network, orderer, "testchannel", config, updatedConfig, peer, orderer) + currentBlockNumber = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) // Now, run a transaction to ensure BFT works. env = CreateBroadcastEnvelope(network, o1, "testchannel", []byte("foo")) @@ -188,379 +176,247 @@ var _ = Describe("ConsensusTypeMigration", func() { Expect(err).NotTo(HaveOccurred()) Expect(resp.Status).To(Equal(common.Status_SUCCESS)) - time.Sleep(time.Second * 5) - - // TODO: check block was successfully committed in all orderers + // check block was successfully committed in all orderers + assertBlockReceptionInAllOrderers(network.Orderers[1:], peer, network, "testchannel", currentBlockNumber) }) }) - // These tests execute the migration config updates on an etcdraft based system, but do not restart the orderer - // to a "future-type" consensus-type that currently does not exist. However, these tests try to maintain as much as - // possible the testing infrastructure for consensus-type migration that existed when "solo" and "kafka" were still - // supported. When a future type that can be migrated to from raft becomes available, some test-cases within this - // suite will need to be completed and revised. - Describe("Raft to future-type migration", func() { + Describe("Enforcing config updates during Raft to BFT migration", func() { var ( - orderer *nwo.Orderer - peer *nwo.Peer - channel1, channel2 string + o1 *nwo.Orderer + o2 *nwo.Orderer + o3 *nwo.Orderer + o4 *nwo.Orderer + peer *nwo.Peer ) BeforeEach(func() { - network = nwo.New(nwo.MultiChannelEtcdRaft(), testDir, client, StartPort(), components) + networkConfig := nwo.MultiNodeEtcdRaft() + networkConfig.Orderers = append(networkConfig.Orderers, &nwo.Orderer{Name: "orderer4", Organization: "OrdererOrg"}) + networkConfig.Profiles[0].Orderers = []string{"orderer1", "orderer2", "orderer3", "orderer4"} + + network = nwo.New(networkConfig, testDir, client, StartPort(), components) + + o1, o2, o3, o4 = network.Orderer("orderer1"), network.Orderer("orderer2"), network.Orderer("orderer3"), network.Orderer("orderer4") network.GenerateConfigTree() network.Bootstrap() - orderer = network.Orderer("orderer") - peer = network.Peer("Org1", "peer0") + runOrderers := func() { + o1Runner = network.OrdererRunner(o1) + o2Runner = network.OrdererRunner(o2) + o3Runner = network.OrdererRunner(o3) + o4Runner = network.OrdererRunner(o4) - channel1 = "testchannel" - channel2 = "testchannel2" + o1Runner.Command.Env = append(o1Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o2Runner.Command.Env = append(o2Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o3Runner.Command.Env = append(o3Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") + o4Runner.Command.Env = append(o4Runner.Command.Env, "FABRIC_LOGGING_SPEC=orderer.common.cluster=debug:orderer.consensus.smartbft=debug:policies.ImplicitOrderer=debug") - o1Runner = network.OrdererRunner(orderer) - o1Proc = ifrit.Invoke(o1Runner) - Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + o1Proc = ifrit.Invoke(o1Runner) + o2Proc = ifrit.Invoke(o2Runner) + o3Proc = ifrit.Invoke(o3Runner) + o4Proc = ifrit.Invoke(o4Runner) + + Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o2Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o3Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + Eventually(o4Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + } + + runOrderers() - By("Create & join first channel") - channelparticipation.JoinOrdererAppChannel(network, channel1, orderer, o1Runner) + channelparticipation.JoinOrderersAppChannelCluster(network, "testchannel", o1, o2, o3, o4) + FindLeader([]*ginkgomon.Runner{o1Runner, o2Runner, o3Runner, o4Runner}) + + peer = network.Peer("Org1", "peer0") }) // This test executes the "green path" migration config updates on an etcdraft based system, on a standard // channel, and verifies that these config updates have the desired effect. - // // The green path is entering maintenance mode, and then changing the consensus type. // In maintenance mode we check that normal transactions are blocked. - // // We also check that after entering maintenance mode, we can exit it without making any changes - the "abort path". It("executes raft2future green path", func() { // === The abort path ====================================================================================== // === Step 1: Config update on standard channel, MAINTENANCE === By("1) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") - config, updatedConfig := prepareTransition(network, peer, orderer, channel1, + config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("1) Verify: standard channel config changed") - std1EntryBlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) + std1EntryBlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1EntryBlockNum).ToNot(Equal(0)) - config = nwo.GetConfig(network, peer, orderer, channel1) + config = nwo.GetConfig(network, peer, o1, "testchannel") consensusTypeValue := extractOrdererConsensusType(config) validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) By("1) Verify: Normal TX's on standard channel are blocked") - assertTxFailed(network, orderer, channel1) + assertTxFailed(network, o1, "testchannel") // In maintenance mode deliver requests are open to those entities that satisfy the /Channel/Orderer/Readers policy By("1) Verify: delivery request from peer is blocked") - err := checkPeerDeliverRequest(orderer, peer, network, channel1) + err := checkPeerDeliverRequest(o1, peer, network, "testchannel") Expect(err).To(MatchError(errors.New("FORBIDDEN"))) - // === Step 2: Create a new channel - By("2) Create & join second channel") - channelparticipation.JoinOrdererAppChannel(network, channel2, orderer, o1Runner) - assertBlockCreation(network, orderer, peer, channel2, 1) - - By("2) Verify: delivery request from peer is not blocked on new channel") - err = checkPeerDeliverRequest(orderer, peer, network, channel2) - Expect(err).NotTo(HaveOccurred()) - - // === Step 3: config update on standard channel, State=NORMAL, abort === - By("3) Config update on standard channel, State=NORMAL, exit maintenance-mode - abort path") - config, updatedConfig = prepareTransition(network, peer, orderer, channel1, + // === Step 2: config update on standard channel, State=NORMAL, abort === + By("2) Config update on standard channel, State=NORMAL, exit maintenance-mode - abort path") + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("3) Verify: standard channel config changed") - std1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) + By("2) Verify: standard channel config changed") + std1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1BlockNum).To(Equal(std1EntryBlockNum + 1)) - By("3) Verify: standard channel delivery requests from peer unblocked") - err = checkPeerDeliverRequest(orderer, peer, network, channel1) + By("2) Verify: standard channel delivery requests from peer unblocked") + err = checkPeerDeliverRequest(o1, peer, network, "testchannel") Expect(err).NotTo(HaveOccurred()) - By("3) Verify: Normal TX's on standard channel are permitted again") - assertBlockCreation(network, orderer, nil, channel1, 3) + By("2) Verify: Normal TX's on standard channel are permitted again") + assertBlockCreation(network, o1, nil, "testchannel", std1EntryBlockNum+2) // === The green path ====================================================================================== - // === Step 4: Config update on standard channel1, MAINTENANCE, again === - By("4) Config update on standard channel1, State=MAINTENANCE, enter maintenance-mode again") - config, updatedConfig = prepareTransition(network, peer, orderer, channel1, + //=== Step 3: Config update on standard channel, MAINTENANCE, again === + By("3) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode again") + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("4) Verify: standard channel config changed") - std1EntryBlockNum = nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) - Expect(std1EntryBlockNum).ToNot(Equal(0)) + By("3) Verify: standard channel config changed") + std1EntryBlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(std1EntryBlockNum).To(Equal(uint64(4))) - config = nwo.GetConfig(network, peer, orderer, channel1) + config = nwo.GetConfig(network, peer, o1, "testchannel") consensusTypeValue = extractOrdererConsensusType(config) validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) - By("4) Verify: delivery request from peer is blocked") - err = checkPeerDeliverRequest(orderer, peer, network, channel1) + By("3) Verify: delivery request from peer is blocked") + err = checkPeerDeliverRequest(o1, peer, network, "testchannel") Expect(err).To(MatchError(errors.New("FORBIDDEN"))) - By("4) Verify: Normal TX's on standard channel are blocked") - assertTxFailed(network, orderer, channel1) - - // === Step 5: Config update on standard channel2, MAINTENANCE === - By("5) Config update on standard channel2, State=MAINTENANCE, enter maintenance-mode again") - config, updatedConfig = prepareTransition(network, peer, orderer, channel2, - "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, channel2, config, updatedConfig, peer, orderer) + By("3) Verify: Normal TX's on standard channel are blocked") + assertTxFailed(network, o1, "testchannel") - By("5) Verify: standard channel config changed") - std2EntryBlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel2) - Expect(std2EntryBlockNum).ToNot(Equal(0)) + // === Step 4: config update on standard channel, State=MAINTENANCE, type=BFT === + bftMetadata := protoutil.MarshalOrPanic(prepareBftMetadata()) - config = nwo.GetConfig(network, peer, orderer, channel2) - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) + By("4) config update on standard channel, State=MAINTENANCE, type=BFT") + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("5) Verify: delivery request from peer is blocked") - err = checkPeerDeliverRequest(orderer, peer, network, channel2) - Expect(err).To(MatchError(errors.New("FORBIDDEN"))) + By("4) Verify: standard channel config changed") + std1EntryBlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(std1EntryBlockNum).To(Equal(uint64(5))) - By("5) Verify: Normal TX's on standard channel are blocked") - assertTxFailed(network, orderer, channel2) + config = nwo.GetConfig(network, peer, o1, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - // Note: - // The following testing steps should be completed once we have a consensus-type ("future-type") that can be - // migrated to from etcdraft. + By("4) Verify: validate consensus type value in all orderers") + // orderer2 + config = nwo.GetConfig(network, peer, o2, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - // === Step 6: config update on standard channel1, State=MAINTENANCE, type=future-type === + // orderer3 + config = nwo.GetConfig(network, peer, o3, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - // === Step 7: config update on standard channel2, State=MAINTENANCE, type=future-type === + // orderer4 + config = nwo.GetConfig(network, peer, o4, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) }) // This test executes the migration flow and checks that forbidden transitions are rejected. // These transitions are enforced by the maintenance filter: // - Entry to & exit from maintenance mode can only change ConsensusType.State. - // - In maintenance mode one can only change ConsensusType.Type & ConsensusType.Metadata. - // - ConsensusType.Type can only change from "etcdraft" to "future-type" (to be replaced by future consensus - // protocol), and only in maintenance mode. + // - In maintenance mode one can only change ConsensusType.Type & ConsensusType.Metadata & Orderers.ConsenterMapping. + // - ConsensusType.Type can only change from "etcdraft" to "BFT", and only in maintenance mode. It("executes raft2future forbidden transitions", func() { // === Step 1: === By("1) Config update on standard channel, changing both ConsensusType State & Type is forbidden") - assertTransitionFailed(network, peer, orderer, channel1, + assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "testing-only", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "BFT", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) // === Step 2: === By("2) Config update on standard channel, both ConsensusType State & some other value is forbidden") - config, updatedConfig := prepareTransition(network, peer, orderer, channel1, + config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) updateConfigWithBatchTimeout(updatedConfig) - updateOrdererConfigFailed(network, orderer, channel1, config, updatedConfig, peer, orderer) + updateOrdererConfigFailed(network, o1, "testchannel", config, updatedConfig, peer, o1) // === Step 3: === By("3) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") - config, updatedConfig = prepareTransition(network, peer, orderer, channel1, + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("3) Verify: standard channel config changed") - std1StartBlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) + std1StartBlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1StartBlockNum).ToNot(Equal(0)) - config = nwo.GetConfig(network, peer, orderer, channel1) + config = nwo.GetConfig(network, peer, o1, "testchannel") consensusTypeValue := extractOrdererConsensusType(config) validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) // === Step 4: === By("4) Config update on standard channel, change ConsensusType.Type to unsupported type, forbidden") - assertTransitionFailed(network, peer, orderer, channel1, + assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, "hesse", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) // === Step 5: === By("5) Config update on standard channel, change ConsensusType.Type and State, forbidden") - assertTransitionFailed(network, peer, orderer, channel1, + assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "testing-only", nil, protosorderer.ConsensusType_STATE_NORMAL) - - // Note: - // The following testing steps should be completed once we have a consensus-type ("future-type" that can be - // migrated to from etcdraft. + "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL) // === Step 6: Config update on standard channel, changing both ConsensusType.Type and other value is permitted === By("6) changing both ConsensusType.Type and other value is permitted") - // Change consensus type and batch-timeout, for example + // Change consensus type and batch-timeout + bftMetadata := protoutil.MarshalOrPanic(prepareBftMetadata()) + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + updateConfigWithBatchTimeout(updatedConfig) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + + By("6) Verify: standard channel config changed") + std1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(std1BlockNum).To(Equal(std1StartBlockNum + 1)) + config = nwo.GetConfig(network, peer, o1, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) // === Step 7: === By("7) Config update on standard channel, changing value other than ConsensusType.Type is permitted") - config = nwo.GetConfig(network, peer, orderer, channel1) - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) updatedConfig = proto.Clone(config).(*common.Config) updateConfigWithBatchTimeout(updatedConfig) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("7) Verify: standard channel config changed") - std1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) - Expect(std1BlockNum).To(Equal(std1StartBlockNum + 1)) + std1BlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(std1BlockNum).To(Equal(std1StartBlockNum + 2)) // === Step 8: === By("8) Config update on standard channel, both ConsensusType State & some other value is forbidden") - config, updatedConfig = prepareTransition(network, peer, orderer, channel1, - "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL) - updateConfigWithBatchTimeout(updatedConfig) - updateOrdererConfigFailed(network, orderer, channel1, config, updatedConfig, peer, orderer) - }) - - // Note: - // Instead of booting to a future-type which does not exist yet, we change some other config value in - // maintenance mode, reboot, and exit maintenance mode. Once we have a future consensus-type that can be - // migrated to from raft is available, this test should be completed. - It("executes bootstrap to future-type - single node", func() { - // === Step 1: Config update on standard channel, MAINTENANCE === - By("1) Config update on standard channel, State=MAINTENANCE") - config, updatedConfig := prepareTransition(network, peer, orderer, channel1, - "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) - - By("1) Verify: standard channel config changed") - chan1StartBlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) - Expect(chan1StartBlockNum).ToNot(Equal(0)) - - config = nwo.GetConfig(network, peer, orderer, channel1) - consensusTypeValue := extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) - - By("1) Verify: Normal TX's on standard channel are blocked") - assertTxFailed(network, orderer, channel1) - - // In maintenance mode deliver requests are open to those entities that satisfy the /Channel/Orderer/Readers policy - By("1) Verify: delivery request from peer is blocked") - err := checkPeerDeliverRequest(orderer, peer, network, channel1) - Expect(err).To(MatchError(errors.New("FORBIDDEN"))) - - // === Step 2: config update on standard channel, State=MAINTENANCE, type=etcdraft, (in-lieu of future-type) other value changed === - By("2) Config update on standard channel, State=MAINTENANCE, type=etcdraft, (in-lieu of future-type) other value changed") - config = nwo.GetConfig(network, peer, orderer, channel1) - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) - updatedConfig = proto.Clone(config).(*common.Config) + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", + "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL) updateConfigWithBatchTimeout(updatedConfig) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) - - By("2) Verify: standard channel config changed") - chan1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) - Expect(chan1BlockNum).To(Equal(chan1StartBlockNum + 1)) - - // === Step 3: kill === - By("3) killing orderer1") - o1Proc.Signal(syscall.SIGKILL) - Eventually(o1Proc.Wait(), network.EventuallyTimeout).Should(Receive(MatchError("exit status 137"))) - - // === Step 4: restart === - By("4) restarting orderer1") - network.Consensus.Type = "etcdraft" // Note: change to future-type - - o1Runner = network.OrdererRunner(orderer) - o1Proc = ifrit.Invoke(o1Runner) - - Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) - - assertBlockReception( - map[string]int{ - channel1: int(chan1BlockNum), - }, - []*nwo.Orderer{orderer}, - peer, - network, - ) - - Eventually(o1Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Raft leader changed: 0 -> ")) - Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) - - By("5) Release - executing config transaction on standard channel with restarted orderer") - config, updatedConfig = prepareTransition(network, peer, orderer, channel1, - "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL) - nwo.UpdateOrdererConfig(network, orderer, channel1, config, updatedConfig, peer, orderer) - - By("6) Verify: standard channel config changed") - chan1BlockNum = nwo.CurrentConfigBlockNumber(network, peer, orderer, channel1) - Expect(chan1BlockNum).To(Equal(chan1StartBlockNum + 2)) - - By("7) Executing transaction on standard channel with restarted orderer") - assertBlockCreation(network, orderer, peer, channel1, chan1StartBlockNum+3) - assertBlockCreation(network, orderer, nil, channel1, chan1StartBlockNum+4) - - By("8) Create new channel, executing transaction with restarted orderer") - channelparticipation.JoinOrdererAppChannel(network, channel2, orderer, o1Runner) - - assertBlockCreation(network, orderer, peer, channel2, 1) - assertBlockCreation(network, orderer, nil, channel2, 2) - - By("9) Extending the network configuration to add a new orderer") - // Add another orderer - orderer2 := &nwo.Orderer{ - Name: "orderer2", - Organization: "OrdererOrg", - } - ports := nwo.Ports{} - for _, portName := range nwo.OrdererPortNames() { - ports[portName] = network.ReservePort() - } - network.PortsByOrdererID[orderer2.ID()] = ports - network.Orderers = append(network.Orderers, orderer2) - network.GenerateOrdererConfig(orderer2) - extendNetwork(network) - - secondOrdererCertificatePath := filepath.Join(network.OrdererLocalTLSDir(orderer2), "server.crt") - secondOrdererCertificate, err := os.ReadFile(secondOrdererCertificatePath) - Expect(err).NotTo(HaveOccurred()) - - By("10) Adding the second orderer to second channel") - addConsenter(network, peer, orderer, channel2, protosraft.Consenter{ - ServerTlsCert: secondOrdererCertificate, - ClientTlsCert: secondOrdererCertificate, - Host: "127.0.0.1", - Port: uint32(network.OrdererPort(orderer2, nwo.ClusterPort)), - }) - - By("11) Obtaining the last config block from the orderer") - configBlock := nwo.GetConfigBlock(network, peer, orderer, channel2) - err = os.WriteFile(filepath.Join(testDir, "channel2_block.pb"), protoutil.MarshalOrPanic(configBlock), 0o644) - Expect(err).NotTo(HaveOccurred()) - - By("12) Waiting for the existing orderer to relinquish its leadership") - Eventually(o1Runner.Err(), network.EventuallyTimeout).Should(gbytes.Say("1 stepped down to follower since quorum is not active")) - Eventually(o1Runner.Err(), network.EventuallyTimeout).Should(gbytes.Say("No leader is present, cluster size is 2")) - - By("13) Launching the second orderer") - o2Runner = network.OrdererRunner(orderer2) - o2Proc = ifrit.Invoke(o2Runner) - Eventually(o2Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) - - By("14) Joining the second orderer to channel2") - expectedChannelInfo := channelparticipation.ChannelInfo{ - Name: channel2, - URL: fmt.Sprintf("/participation/v1/channels/%s", channel2), - Status: "onboarding", - ConsensusRelation: "consenter", - Height: 0, - } - channelparticipation.Join(network, orderer2, channel2, configBlock, expectedChannelInfo) - Eventually(o2Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Raft leader changed: 0 -> ")) - - assertBlockReception(map[string]int{ - channel2: int(nwo.CurrentConfigBlockNumber(network, peer, orderer, channel2)), - }, []*nwo.Orderer{orderer2}, peer, network) - - By("15) Executing transaction against second orderer on channel2") - assertBlockCreation(network, orderer2, nil, channel2, 3) + updateOrdererConfigFailed(network, o1, "testchannel", config, updatedConfig, peer, o1) }) }) }) @@ -651,6 +507,9 @@ func prepareTransition( consensusTypeValue := extractOrdererConsensusType(current) validateConsensusTypeValue(consensusTypeValue, fromConsensusType, fromMigState) updateConfigWithConsensusType(toConsensusType, toConsensusMetadata, toMigState, updated, consensusTypeValue) + if toConsensusType == "BFT" { + updateBFTOrderersConfig(network, updated) + } return current, updated } @@ -730,23 +589,17 @@ func createDeliverEnvelope(n *nwo.Network, signer *nwo.SigningIdentity, blkNum u return env } -func addBFTInConfigTree(network *nwo.Network, peer *nwo.Peer, orderer *nwo.Orderer, channel string) { - config := nwo.GetConfig(network, peer, orderer, channel) - - updatedConfig := proto.Clone(config).(*common.Config) - +func updateBFTOrderersConfig(network *nwo.Network, config *common.Config) { orderersVal := &common.Orderers{ ConsenterMapping: computeConsenterMappings(network), } - policies.EncodeBFTBlockVerificationPolicy(orderersVal.ConsenterMapping, updatedConfig.ChannelGroup.Groups["Orderer"]) + policies.EncodeBFTBlockVerificationPolicy(orderersVal.ConsenterMapping, config.ChannelGroup.Groups["Orderer"]) - updatedConfig.ChannelGroup.Groups["Orderer"].Values["Orderers"] = &common.ConfigValue{ + config.ChannelGroup.Groups["Orderer"].Values["Orderers"] = &common.ConfigValue{ Value: protoutil.MarshalOrPanic(orderersVal), ModPolicy: "/Channel/Orderer/Admins", } - - nwo.UpdateOrdererConfig(network, orderer, channel, config, updatedConfig, peer, orderer) } func computeConsenterMappings(network *nwo.Network) []*common.Consenter { @@ -798,3 +651,37 @@ func CreateBroadcastEnvelope(n *nwo.Network, entity interface{}, channel string, return env } + +// assertBlockReception asserts that the given orderers have the expected +// newest block number for the specified channels +func assertBlockReceptionInAllOrderers(orderers []*nwo.Orderer, peer *nwo.Peer, network *nwo.Network, channelId string, currentBlockNumber uint64) { + for _, orderer := range orderers { + ccb := func() uint64 { + return nwo.CurrentConfigBlockNumber(network, peer, orderer, channelId) + } + Eventually(ccb, network.EventuallyTimeout).Should(BeNumerically(">", currentBlockNumber)) + + } +} + +// prepareBftMetadata prepare the bft consensusType.Metadata +func prepareBftMetadata() *smartbft.Options { + bftMetadata := &smartbft.Options{ + RequestBatchMaxCount: types.DefaultConfig.RequestBatchMaxCount, + RequestBatchMaxBytes: types.DefaultConfig.RequestBatchMaxBytes, + RequestBatchMaxInterval: types.DefaultConfig.RequestBatchMaxInterval.String(), + IncomingMessageBufferSize: types.DefaultConfig.IncomingMessageBufferSize, + RequestPoolSize: types.DefaultConfig.RequestPoolSize, + RequestForwardTimeout: types.DefaultConfig.RequestForwardTimeout.String(), + RequestComplainTimeout: types.DefaultConfig.RequestComplainTimeout.String(), + RequestAutoRemoveTimeout: types.DefaultConfig.RequestAutoRemoveTimeout.String(), + ViewChangeResendInterval: types.DefaultConfig.ViewChangeResendInterval.String(), + ViewChangeTimeout: types.DefaultConfig.ViewChangeTimeout.String(), + LeaderHeartbeatTimeout: types.DefaultConfig.LeaderHeartbeatTimeout.String(), + LeaderHeartbeatCount: types.DefaultConfig.LeaderHeartbeatCount, + CollectTimeout: types.DefaultConfig.CollectTimeout.String(), + SyncOnStart: types.DefaultConfig.SyncOnStart, + SpeedUpViewChange: types.DefaultConfig.SpeedUpViewChange, + } + return bftMetadata +} diff --git a/orderer/common/msgprocessor/maintenancefilter.go b/orderer/common/msgprocessor/maintenancefilter.go index 23f5eb617ab..6a1b46a2040 100644 --- a/orderer/common/msgprocessor/maintenancefilter.go +++ b/orderer/common/msgprocessor/maintenancefilter.go @@ -8,11 +8,14 @@ package msgprocessor import ( "bytes" + "time" + + "github.com/SmartBFT-Go/consensus/pkg/types" "github.com/golang/protobuf/proto" cb "github.com/hyperledger/fabric-protos-go/common" "github.com/hyperledger/fabric-protos-go/orderer" - protoetcdraft "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" + "github.com/hyperledger/fabric-protos-go/orderer/smartbft" "github.com/hyperledger/fabric/bccsp" "github.com/hyperledger/fabric/common/channelconfig" "github.com/hyperledger/fabric/common/configtx" @@ -45,7 +48,6 @@ func NewMaintenanceFilter(support MaintenanceFilterSupport, bccsp bccsp.BCCSP) * permittedTargetConsensusTypes: make(map[string]bool), bccsp: bccsp, } - mf.permittedTargetConsensusTypes["etcdraft"] = true mf.permittedTargetConsensusTypes["BFT"] = true return mf } @@ -117,7 +119,7 @@ func (mf *MaintenanceFilter) inspect(configEnvelope *cb.ConfigEnvelope, ordererC } // ConsensusType.Type can only change in maintenance-mode, and only within the set of permitted types. - // Note: only solo to etcdraft transitions are supported. + // Note: only etcdraft to BFT transitions are supported. if ordererConfig.ConsensusType() != nextOrdererConfig.ConsensusType() { if ordererConfig.ConsensusState() == orderer.ConsensusType_STATE_NORMAL { return errors.Errorf("attempted to change consensus type from %s to %s, but current config ConsensusType.State is not in maintenance mode", @@ -133,10 +135,15 @@ func (mf *MaintenanceFilter) inspect(configEnvelope *cb.ConfigEnvelope, ordererC ordererConfig.ConsensusType(), nextOrdererConfig.ConsensusType()) } - if nextOrdererConfig.ConsensusType() == "etcdraft" { - updatedMetadata := &protoetcdraft.ConfigMetadata{} + if nextOrdererConfig.ConsensusType() == "BFT" { + updatedMetadata := &smartbft.Options{} if err := proto.Unmarshal(nextOrdererConfig.ConsensusMetadata(), updatedMetadata); err != nil { - return errors.Wrap(err, "failed to unmarshal etcdraft metadata configuration") + return errors.Wrap(err, "failed to unmarshal BFT metadata configuration") + } + + _, err := validateBFTMetadataOptions(1, updatedMetadata) + if updatedMetadata.XXX_unrecognized != nil || err != nil { + return errors.New("invalid BFT metadata configuration") } } @@ -197,3 +204,59 @@ func (mf *MaintenanceFilter) ensureConsensusTypeChangeOnly(configEnvelope *cb.Co return nil } + +func validateBFTMetadataOptions(selfID uint64, options *smartbft.Options) (types.Configuration, error) { + var err error + + config := types.DefaultConfig + config.SelfID = selfID + + if options == nil { + return config, errors.New("config metadata options field is nil") + } + + config.RequestBatchMaxCount = options.RequestBatchMaxCount + config.RequestBatchMaxBytes = options.RequestBatchMaxBytes + if config.RequestBatchMaxInterval, err = time.ParseDuration(options.RequestBatchMaxInterval); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestBatchMaxInterval") + } + config.IncomingMessageBufferSize = options.IncomingMessageBufferSize + config.RequestPoolSize = options.RequestPoolSize + if config.RequestForwardTimeout, err = time.ParseDuration(options.RequestForwardTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestForwardTimeout") + } + if config.RequestComplainTimeout, err = time.ParseDuration(options.RequestComplainTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestComplainTimeout") + } + if config.RequestAutoRemoveTimeout, err = time.ParseDuration(options.RequestAutoRemoveTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestAutoRemoveTimeout") + } + if config.ViewChangeResendInterval, err = time.ParseDuration(options.ViewChangeResendInterval); err != nil { + return config, errors.Wrap(err, "bad config metadata option ViewChangeResendInterval") + } + if config.ViewChangeTimeout, err = time.ParseDuration(options.ViewChangeTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option ViewChangeTimeout") + } + if config.LeaderHeartbeatTimeout, err = time.ParseDuration(options.LeaderHeartbeatTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option LeaderHeartbeatTimeout") + } + config.LeaderHeartbeatCount = options.LeaderHeartbeatCount + if config.CollectTimeout, err = time.ParseDuration(options.CollectTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option CollectTimeout") + } + config.SyncOnStart = options.SyncOnStart + config.SpeedUpViewChange = options.SpeedUpViewChange + + config.LeaderRotation = false + config.DecisionsPerLeader = 0 + + if err = config.Validate(); err != nil { + return config, errors.Wrap(err, "config validation failed") + } + + if options.RequestMaxBytes == 0 { + config.RequestMaxBytes = config.RequestBatchMaxBytes + } + + return config, nil +} diff --git a/orderer/common/msgprocessor/maintenancefilter_test.go b/orderer/common/msgprocessor/maintenancefilter_test.go index 7bd4a6abff8..eace50e2f52 100644 --- a/orderer/common/msgprocessor/maintenancefilter_test.go +++ b/orderer/common/msgprocessor/maintenancefilter_test.go @@ -7,11 +7,18 @@ SPDX-License-Identifier: Apache-2.0 package msgprocessor import ( + "fmt" + "os" + "path" "testing" + "github.com/SmartBFT-Go/consensus/pkg/types" + "github.com/golang/protobuf/proto" + "github.com/hyperledger/fabric-protos-go/orderer/smartbft" + "github.com/hyperledger/fabric/common/crypto/tlsgen" + "github.com/hyperledger/fabric-protos-go/common" "github.com/hyperledger/fabric-protos-go/orderer" - "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" "github.com/hyperledger/fabric/bccsp/sw" "github.com/hyperledger/fabric/common/capabilities" "github.com/hyperledger/fabric/common/channelconfig" @@ -29,7 +36,7 @@ func newMockOrdererConfig(migration bool, state orderer.ConsensusType_State) *mo mockCapabilities := &mocks.OrdererCapabilities{} mockCapabilities.ConsensusTypeMigrationReturns(migration) mockOrderer.CapabilitiesReturns(mockCapabilities) - mockOrderer.ConsensusTypeReturns("solo") + mockOrderer.ConsensusTypeReturns("etcdraft") mockOrderer.ConsensusStateReturns(state) return mockOrderer } @@ -54,7 +61,7 @@ func TestMaintenanceDisabled(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msInactive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + current := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} t.Run("Good", func(t *testing.T) { configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, 3) @@ -63,7 +70,7 @@ func TestMaintenanceDisabled(t *testing.T) { }) t.Run("Block entry to maintenance", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, @@ -71,11 +78,11 @@ func TestMaintenanceDisabled(t *testing.T) { }) t.Run("Block type change", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, - "config transaction inspection failed: next config attempted to change ConsensusType.Type from solo to etcdraft, but capability is disabled") + "config transaction inspection failed: next config attempted to change ConsensusType.Type from etcdraft to BFT, but capability is disabled") }) } @@ -166,29 +173,47 @@ func TestMaintenanceInspectEntry(t *testing.T) { mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) bogusMetadata := []byte{1, 2, 3, 4} - current := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + validMetadata := []byte{} + bftMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} t.Run("Good", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Bad: concurrent change to consensus type & state", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "BFT", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelope(t, current, next) + err := mf.Apply(configTx) + require.EqualError(t, err, + "config transaction inspection failed: attempted to change ConsensusType.Type from etcdraft to BFT, but ConsensusType.State is changing from STATE_NORMAL to STATE_MAINTENANCE") + }) + + t.Run("Bad: concurrent change to metadata & state", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "etcdraft", metadata: bftMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, - "config transaction inspection failed: attempted to change ConsensusType.Type from solo to etcdraft, but ConsensusType.State is changing from STATE_NORMAL to STATE_MAINTENANCE") + "config transaction inspection failed: attempted to change ConsensusType.Metadata, but ConsensusType.State is changing from STATE_NORMAL to STATE_MAINTENANCE") + }) + + t.Run("Bad: concurrent change to state & orderer value", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + err := mf.Apply(configTx) + require.EqualError(t, err, + "config transaction inspection failed: config update contains changes to groups within the Orderer group") }) t.Run("Bad: change consensus type not in maintenance", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, - "config transaction inspection failed: attempted to change consensus type from solo to etcdraft, but current config ConsensusType.State is not in maintenance mode") + "config transaction inspection failed: attempted to change consensus type from etcdraft to BFT, but current config ConsensusType.State is not in maintenance mode") }) } @@ -201,53 +226,62 @@ func TestMaintenanceInspectChange(t *testing.T) { mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) bogusMetadata := []byte{1, 2, 3, 4} - validMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{}) - current := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} + validMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} - t.Run("Good type change", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelope(t, current, next) + t.Run("Good type change with valid BFT metadata and consenter mapping", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good exit, no change", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.NoError(t, err) }) - t.Run("Bad: unsupported consensus type", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "unsupported", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + t.Run("Bad: good type change with invalid BFT metadata", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) + require.Error(t, err) require.EqualError(t, err, - "config transaction inspection failed: attempted to change consensus type from solo to unsupported, transition not supported") + "config transaction inspection failed: invalid BFT metadata configuration") }) - t.Run("Bad: concurrent change to consensus type & state", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + t.Run("Bad: BFT metadata cannot be unmarshalled", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelope(t, current, next) + err := mf.Apply(configTx) + require.Error(t, err) + require.Contains(t, err.Error(), + "config transaction inspection failed: failed to unmarshal BFT metadata configuration") + }) + + t.Run("Bad: unsupported consensus type", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "unsupported", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, - "config transaction inspection failed: attempted to change ConsensusType.Type from solo to etcdraft, but ConsensusType.State is changing from STATE_MAINTENANCE to STATE_NORMAL") + "config transaction inspection failed: attempted to change consensus type from etcdraft to unsupported, transition not supported") }) - t.Run("Bad: etcdraft metadata", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + t.Run("Bad: concurrent change to consensus type & state", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) - require.Error(t, err) - require.Contains(t, err.Error(), - "config transaction inspection failed: failed to unmarshal etcdraft metadata configuration") + require.EqualError(t, err, + "config transaction inspection failed: attempted to change ConsensusType.Type from etcdraft to BFT, but ConsensusType.State is changing from STATE_MAINTENANCE to STATE_NORMAL") }) } func TestMaintenanceInspectExit(t *testing.T) { - validMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{}) + validMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) mockOrderer := newMockOrdererConfig(true, orderer.ConsensusType_STATE_MAINTENANCE) - mockOrderer.ConsensusTypeReturns("etcdraft") + mockOrderer.ConsensusTypeReturns("BFT") mockOrderer.ConsensusMetadataReturns(validMetadata) msActive := &mockSystemChannelFilterSupport{ @@ -257,43 +291,58 @@ func TestMaintenanceInspectExit(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + current := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} t.Run("Good exit", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Bad: concurrent change to consensus type & state", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "solo", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, - "config transaction inspection failed: attempted to change ConsensusType.Type from etcdraft to solo, but ConsensusType.State is changing from STATE_MAINTENANCE to STATE_NORMAL") + "config transaction inspection failed: attempted to change ConsensusType.Type from BFT to etcdraft, but ConsensusType.State is changing from STATE_MAINTENANCE to STATE_NORMAL") + }) + + t.Run("Bad: change consensus type from BFT to Raft", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelope(t, current, next) + err := mf.Apply(configTx) + require.EqualError(t, err, + "config transaction inspection failed: attempted to change consensus type from BFT to etcdraft, transition not supported") }) t.Run("Bad: exit with extra group", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to more than one group") }) t.Run("Bad: exit with extra value", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to values in group Channel") }) t.Run("Bad: exit with extra orderer value", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contain more then just the ConsensusType value in the Orderer group") }) + + t.Run("Bad: exit with extra orderer value required for BFT", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + err := mf.Apply(configTx) + require.EqualError(t, err, "config transaction inspection failed: config update contains changes to groups within the Orderer group") + }) } func TestMaintenanceExtra(t *testing.T) { @@ -304,29 +353,36 @@ func TestMaintenanceExtra(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "solo", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} - validMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{}) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} + validMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) t.Run("Good: with extra group", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good: with extra value", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good: with extra orderer value", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3) err := mf.Apply(configTx) require.NoError(t, err) }) + + t.Run("Good: with extra orderer value required for BFT", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + err := mf.Apply(configTx) + require.NoError(t, err) + }) } func TestMaintenanceMissingConsensusType(t *testing.T) { @@ -337,7 +393,7 @@ func TestMaintenanceMissingConsensusType(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "solo", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} + current := consensusTypeInfo{ordererType: "etcdraft", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} for i := 1; i < 4; i++ { configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, i) err := mf.Apply(configTx) @@ -353,7 +409,7 @@ type consensusTypeInfo struct { func makeConfigEnvelope(t *testing.T, current, next consensusTypeInfo) *common.Envelope { original := makeBaseConfig(t) - updated := makeBaseConfig(t) + updated := proto.Clone(original).(*common.Config) original.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.ConsensusTypeKey] = &common.ConfigValue{ Value: protoutil.MarshalOrPanic( @@ -382,7 +438,7 @@ func makeConfigEnvelope(t *testing.T, current, next consensusTypeInfo) *common.E func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeInfo, extra int) *common.Envelope { original := makeBaseConfig(t) - updated := makeBaseConfig(t) + updated := proto.Clone(original).(*common.Config) original.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.ConsensusTypeKey] = &common.ConfigValue{ Value: protoutil.MarshalOrPanic( @@ -406,7 +462,7 @@ func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeI switch extra { case 1: - updated.ChannelGroup.Groups[channelconfig.ConsortiumsGroupKey] = &common.ConfigGroup{} + updated.ChannelGroup.Groups[channelconfig.ApplicationGroupKey].Policies = nil case 2: updated.ChannelGroup.Values[channelconfig.ConsortiumKey] = &common.ConfigValue{ Value: protoutil.MarshalOrPanic(&common.Consortium{}), @@ -423,6 +479,11 @@ func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeI }), ModPolicy: channelconfig.AdminsPolicyKey, } + case 4: + updated.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.OrderersKey] = &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(&common.Orderers{ConsenterMapping: []*common.Consenter{{Id: 1, Host: "", Port: 0}}}), + ModPolicy: channelconfig.AdminsPolicyKey, + } default: return nil } @@ -455,11 +516,18 @@ func makeConfigTx(original, updated *common.Config, t *testing.T) *common.Envelo } func makeBaseConfig(t *testing.T) *common.Config { - gConf := genesisconfig.Load(genesisconfig.SampleInsecureSoloProfile, configtest.GetDevConfigDir()) + certDir := t.TempDir() + tlsCA, err := tlsgen.NewCA() + require.NoError(t, err) + + gConf := genesisconfig.Load(genesisconfig.SampleAppChannelEtcdRaftProfile, configtest.GetDevConfigDir()) + generateCertificates(t, gConf, tlsCA, certDir) + gConf.Orderer.Capabilities = map[string]bool{ - capabilities.OrdererV1_4_2: true, + capabilities.ChannelV3_0: true, } - gConf.Orderer.OrdererType = "solo" + + gConf.Orderer.OrdererType = "etcdraft" channelGroup, err := encoder.NewChannelGroup(gConf) require.NoError(t, err) original := &common.Config{ @@ -467,3 +535,43 @@ func makeBaseConfig(t *testing.T) *common.Config { } return original } + +func generateCertificates(t *testing.T, confAppRaft *genesisconfig.Profile, tlsCA tlsgen.CA, certDir string) { + for i, c := range confAppRaft.Orderer.EtcdRaft.Consenters { + srvC, err := tlsCA.NewServerCertKeyPair(c.Host) + require.NoError(t, err) + srvP := path.Join(certDir, fmt.Sprintf("server%d.crt", i)) + err = os.WriteFile(srvP, srvC.Cert, 0o644) + require.NoError(t, err) + + clnC, err := tlsCA.NewClientCertKeyPair() + require.NoError(t, err) + clnP := path.Join(certDir, fmt.Sprintf("client%d.crt", i)) + err = os.WriteFile(clnP, clnC.Cert, 0o644) + require.NoError(t, err) + + c.ServerTlsCert = []byte(srvP) + c.ClientTlsCert = []byte(clnP) + } +} + +func createValidBFTMetadata() *smartbft.Options { + bftMetadata := &smartbft.Options{ + RequestBatchMaxCount: types.DefaultConfig.RequestBatchMaxCount, + RequestBatchMaxBytes: types.DefaultConfig.RequestBatchMaxBytes, + RequestBatchMaxInterval: types.DefaultConfig.RequestBatchMaxInterval.String(), + IncomingMessageBufferSize: types.DefaultConfig.IncomingMessageBufferSize, + RequestPoolSize: types.DefaultConfig.RequestPoolSize, + RequestForwardTimeout: types.DefaultConfig.RequestForwardTimeout.String(), + RequestComplainTimeout: types.DefaultConfig.RequestComplainTimeout.String(), + RequestAutoRemoveTimeout: types.DefaultConfig.RequestAutoRemoveTimeout.String(), + ViewChangeResendInterval: types.DefaultConfig.ViewChangeResendInterval.String(), + ViewChangeTimeout: types.DefaultConfig.ViewChangeTimeout.String(), + LeaderHeartbeatTimeout: types.DefaultConfig.LeaderHeartbeatTimeout.String(), + LeaderHeartbeatCount: types.DefaultConfig.LeaderHeartbeatCount, + CollectTimeout: types.DefaultConfig.CollectTimeout.String(), + SyncOnStart: types.DefaultConfig.SyncOnStart, + SpeedUpViewChange: types.DefaultConfig.SpeedUpViewChange, + } + return bftMetadata +} diff --git a/orderer/consensus/etcdraft/chain.go b/orderer/consensus/etcdraft/chain.go index 5fd1db3c21e..3469a1ecdd4 100644 --- a/orderer/consensus/etcdraft/chain.go +++ b/orderer/consensus/etcdraft/chain.go @@ -1115,14 +1115,28 @@ func (c *Chain) commitBlock(block *common.Block) { func (c *Chain) detectConfChange(block *common.Block) *MembershipChanges { // If config is targeting THIS channel, inspect consenter set and // propose raft ConfChange if it adds/removes node. + c.logger.Infof("Detected configuration change") + + // if the consensusType is bft, then configMetadata which represents the raft metadata should be nil configMetadata, consensusType := c.newConfigMetadata(block) + c.logger.Infof("Detected configuration change: consensusType is: %s, configMetadata is: %v", consensusType, configMetadata) - if configMetadata == nil { + if consensusType == nil { + c.logger.Infof("ConsensusType is %v", consensusType) return nil } if consensusType.Type != "etcdraft" { - c.logger.Infof("Detected migration to %s", consensusType.Type) + if consensusType.Type == "BFT" { + c.logger.Infof("Detected migration to %s", consensusType.Type) + return nil + } else { + c.logger.Panicf("illegal consensus type detected: %s", consensusType.Type) + panic("illegal consensus type detected during conf change") + } + } + + if configMetadata == nil { return nil } @@ -1431,6 +1445,7 @@ func (c *Chain) getInFlightConfChange() *raftpb.ConfChange { // newMetadata extract config metadata from the configuration block func (c *Chain) newConfigMetadata(block *common.Block) (*etcdraft.ConfigMetadata, *orderer.ConsensusType) { + c.logger.Infof("Extract config metadata from the configuration block") metadata, consensusType, err := ConsensusMetadataFromConfigBlock(block) if err != nil { c.logger.Panicf("error reading consensus metadata: %s", err) @@ -1452,8 +1467,14 @@ func (c *Chain) ValidateConsensusMetadata(oldOrdererConfig, newOrdererConfig cha } if newOrdererConfig.ConsensusType() != "etcdraft" { - // This is a migration, so we don't know how to validate this config change. - return nil + if newOrdererConfig.ConsensusType() == "BFT" { + // This is a migration, so we don't know how to validate this config change. + return nil + } else { + c.logger.Panicf("illegal consensus type detected during consensus metadata validation: %s", newOrdererConfig.ConsensusType()) + return errors.Errorf("illegal consensus type detected during consensus metadata validation: %s", newOrdererConfig.ConsensusType()) + + } } if oldOrdererConfig == nil { diff --git a/orderer/consensus/etcdraft/chain_test.go b/orderer/consensus/etcdraft/chain_test.go index 88c82a80573..96388d745cd 100644 --- a/orderer/consensus/etcdraft/chain_test.go +++ b/orderer/consensus/etcdraft/chain_test.go @@ -552,11 +552,17 @@ var _ = Describe("Chain", func() { }, "ConsensusType": { Version: 4, + Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", + }), }, } oldValues := map[string]*common.ConfigValue{ "ConsensusType": { Version: 4, + Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", + }), }, } configEnv = newConfigEnv(channelID, @@ -678,6 +684,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(consenterMetadata), }), }, @@ -711,6 +718,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, @@ -726,6 +734,35 @@ var _ = Describe("Chain", func() { }) }) }) + + Context("when a type C config update comes", func() { + Context("change from raft to bft", func() { + // use to prepare the Orderer Values + BeforeEach(func() { + values := map[string]*common.ConfigValue{ + "ConsensusType": { + Version: 1, + Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "BFT", + Metadata: []byte{1, 2}, + }), + }, + } + configEnv = newConfigEnv(channelID, + common.HeaderType_CONFIG, + newConfigUpdateEnv(channelID, nil, values)) + configSeq = 0 + }) // BeforeEach block + + It("should be able to process config update of type C", func() { + err := chain.Configure(configEnv, configSeq) + Expect(err).NotTo(HaveOccurred()) + Expect(fakeFields.fakeConfigProposalsReceived.AddCallCount()).To(Equal(1)) + Expect(fakeFields.fakeConfigProposalsReceived.AddArgsForCall(0)).To(Equal(float64(1))) + Eventually(support.WriteConfigBlockCallCount, LongEventualTimeout).Should(Equal(1)) + }) + }) + }) }) Describe("Crash Fault Tolerance", func() { @@ -1426,6 +1463,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, @@ -1770,6 +1808,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, @@ -1853,6 +1892,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, @@ -1893,6 +1933,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, @@ -1935,6 +1976,7 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 1, Value: marshalOrPanic(&orderer.ConsensusType{ + Type: "etcdraft", Metadata: marshalOrPanic(metadata), }), }, diff --git a/orderer/consensus/etcdraft/util.go b/orderer/consensus/etcdraft/util.go index f734fc3c1b4..022ec8ace30 100644 --- a/orderer/consensus/etcdraft/util.go +++ b/orderer/consensus/etcdraft/util.go @@ -84,12 +84,19 @@ func MetadataHasDuplication(md *etcdraft.ConfigMetadata) error { } // MetadataFromConfigValue reads and translates configuration updates from config value into raft metadata +// In case consensus type is changed to BFT the raft metadata will be nil func MetadataFromConfigValue(configValue *common.ConfigValue) (*etcdraft.ConfigMetadata, *orderer.ConsensusType, error) { consensusTypeValue := &orderer.ConsensusType{} if err := proto.Unmarshal(configValue.Value, consensusTypeValue); err != nil { return nil, nil, errors.Wrap(err, "failed to unmarshal consensusType config update") } + if consensusTypeValue.Type != "etcdraft" { + if consensusTypeValue.Type == "BFT" { + return nil, consensusTypeValue, nil + } + } + updatedMetadata := &etcdraft.ConfigMetadata{} if err := proto.Unmarshal(consensusTypeValue.Metadata, updatedMetadata); err != nil { return nil, nil, errors.Wrap(err, "failed to unmarshal updated (new) etcdraft metadata configuration") diff --git a/orderer/consensus/etcdraft/util_test.go b/orderer/consensus/etcdraft/util_test.go index fbf42a642a1..677915f2f06 100644 --- a/orderer/consensus/etcdraft/util_test.go +++ b/orderer/consensus/etcdraft/util_test.go @@ -17,6 +17,8 @@ import ( "testing" "time" + "github.com/hyperledger/fabric-protos-go/orderer" + "github.com/golang/protobuf/proto" "github.com/hyperledger/fabric-protos-go/common" etcdraftproto "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" @@ -410,3 +412,18 @@ func TestVerifyConfigMetadata(t *testing.T) { require.Nil(t, VerifyConfigMetadata(metadataWithExpiredConsenter, goodVerifyingOpts)) }) } + +func TestMetadataFromConfigValue(t *testing.T) { + configValue := &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(&orderer.ConsensusType{ + Type: "BFT", + Metadata: []byte{1, 2}, + }), + } + + metadata, consensusType, err := MetadataFromConfigValue(configValue) + require.Nil(t, metadata) + require.Nil(t, err) + require.NotNil(t, consensusType) + require.Equal(t, consensusType.Type, "BFT") +} diff --git a/orderer/consensus/etcdraft/validator_test.go b/orderer/consensus/etcdraft/validator_test.go index 1a31f6bc536..c0a6817b9f7 100644 --- a/orderer/consensus/etcdraft/validator_test.go +++ b/orderer/consensus/etcdraft/validator_test.go @@ -113,6 +113,7 @@ var _ = Describe("Metadata Validation", func() { It("fails when old consensus metadata is not well-formed", func() { oldOrdererConf := mockOrderer([]byte("test")) newOrdererConf := mockOrderer([]byte("test")) + newOrdererConf.ConsensusTypeReturns("etcdraft") Expect(func() { chain.ValidateConsensusMetadata(oldOrdererConf, newOrdererConf, false) }).To(Panic()) @@ -122,6 +123,7 @@ var _ = Describe("Metadata Validation", func() { oldBytes, _ := proto.Marshal(&raftprotos.ConfigMetadata{}) oldOrdererConf := mockOrderer(oldBytes) newOrdererConf := mockOrderer([]byte("test")) + newOrdererConf.ConsensusTypeReturns("etcdraft") Expect(chain.ValidateConsensusMetadata(oldOrdererConf, newOrdererConf, false)).NotTo(Succeed()) }) }) @@ -177,6 +179,7 @@ var _ = Describe("Metadata Validation", func() { newOrdererConfig.OrganizationsReturns(map[string]channelconfig.OrdererOrg{ "org1": org1, }) + newOrdererConfig.ConsensusTypeReturns("etcdraft") newChannel = false }) From 7f404afc379ac272bb6a65ae6652c50a4c93c156 Mon Sep 17 00:00:00 2001 From: May Rosenbaum Date: Thu, 18 Jan 2024 03:10:48 +0200 Subject: [PATCH 3/4] Raft to BFT migration: add and edit test cases Signed-off-by: May Rosenbaum --- integration/raft/migration_test.go | 255 ++++++++++++++---- .../common/msgprocessor/maintenancefilter.go | 44 ++- .../msgprocessor/maintenancefilter_test.go | 107 ++++++-- orderer/consensus/etcdraft/chain_test.go | 6 +- orderer/consensus/etcdraft/util.go | 4 +- orderer/consensus/etcdraft/util_test.go | 2 + 6 files changed, 325 insertions(+), 93 deletions(-) diff --git a/integration/raft/migration_test.go b/integration/raft/migration_test.go index d6cf7b1f275..1ea8a1f6ae4 100644 --- a/integration/raft/migration_test.go +++ b/integration/raft/migration_test.go @@ -80,6 +80,8 @@ var _ = Describe("ConsensusTypeMigration", func() { // This test restarts the orderers and ensures they operate under the new configuration. Describe("Raft to BFT migration", func() { It("migrates from Raft to BFT", func() { + // === Step 1: Create and run Raft based system with 4 nodes === + By("1) Starting Raft based system with 4 nodes") networkConfig := nwo.MultiNodeEtcdRaft() networkConfig.Orderers = append(networkConfig.Orderers, &nwo.Orderer{Name: "orderer4", Organization: "OrdererOrg"}) networkConfig.Profiles[0].Orderers = []string{"orderer1", "orderer2", "orderer3", "orderer4"} @@ -117,7 +119,8 @@ var _ = Describe("ConsensusTypeMigration", func() { channelparticipation.JoinOrderersAppChannelCluster(network, "testchannel", o1, o2, o3, o4) FindLeader([]*ginkgomon.Runner{o1Runner, o2Runner, o3Runner, o4Runner}) - By("performing operation with orderer1") + // === Step 2: Create a transaction with orderer1 === + By("2) Performing operation with orderer1") env := CreateBroadcastEnvelope(network, o1, "testchannel", []byte("foo")) resp, err := ordererclient.Broadcast(network, o1, env) Expect(err).NotTo(HaveOccurred()) @@ -126,26 +129,28 @@ var _ = Describe("ConsensusTypeMigration", func() { block := FetchBlock(network, o1, 1, "testchannel") Expect(block).NotTo(BeNil()) - By("Change to maintenance mode") - + // === Step 3: Config update on standard channel, State=MAINTENANCE, enter maintenance-mode === + By("3) Change to maintenance mode") peer := network.Peer("Org1", "peer0") - - By("Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("Config updates: migration from Raft to BFT") + // === Step 4: Config update, State=MAINTENANCE, type=BFT === + By("4) Config updates: migration from Raft to BFT") bftMetadata := protoutil.MarshalOrPanic(prepareBftMetadata()) config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) currentBlockNumber := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + // === Step 5: Check block reception in all orderers and restart all orderers === + By("5) Checking all orderers received the last block and restarting all orderers") + // check block was committed in all orderers assertBlockReceptionInAllOrderers(network.Orderers[1:], peer, network, "testchannel", currentBlockNumber) @@ -158,19 +163,22 @@ var _ = Describe("ConsensusTypeMigration", func() { runOrderers() - By("Waiting for followers to see the leader, again") + // === Step 6: Waiting for followers to see the leader, again === + By("6) Waiting for followers to see the leader, again") Eventually(o2Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) Eventually(o3Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) Eventually(o4Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Message from 1 channel=testchannel")) - // Exit maintenance mode + // === Step 7: Config update on standard channel, State=NORMAL, exit maintenance-mode === + By("7) Exit maintenance mode") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", bftMetadata, protosorderer.ConsensusType_STATE_NORMAL) + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_NORMAL, 1) currentBlockNumber = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - // Now, run a transaction to ensure BFT works. + // === Step 8: Run a transaction to ensure BFT works === + By("8) Running a transaction with orderer1 to ensure BFT works and check that the tx was committed in all orderers") env = CreateBroadcastEnvelope(network, o1, "testchannel", []byte("foo")) resp, err = ordererclient.Broadcast(network, o1, env) Expect(err).NotTo(HaveOccurred()) @@ -236,14 +244,14 @@ var _ = Describe("ConsensusTypeMigration", func() { // The green path is entering maintenance mode, and then changing the consensus type. // In maintenance mode we check that normal transactions are blocked. // We also check that after entering maintenance mode, we can exit it without making any changes - the "abort path". - It("executes raft2future green path", func() { + It("executes raft2fbft green path with extra checks", func() { // === The abort path ====================================================================================== // === Step 1: Config update on standard channel, MAINTENANCE === By("1) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("1) Verify: standard channel config changed") @@ -265,7 +273,7 @@ var _ = Describe("ConsensusTypeMigration", func() { By("2) Config update on standard channel, State=NORMAL, exit maintenance-mode - abort path") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL) + "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL, 0) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("2) Verify: standard channel config changed") @@ -285,7 +293,7 @@ var _ = Describe("ConsensusTypeMigration", func() { By("3) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode again") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("3) Verify: standard channel config changed") @@ -309,32 +317,19 @@ var _ = Describe("ConsensusTypeMigration", func() { By("4) config update on standard channel, State=MAINTENANCE, type=BFT") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("4) Verify: standard channel config changed") std1EntryBlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1EntryBlockNum).To(Equal(uint64(5))) - config = nwo.GetConfig(network, peer, o1, "testchannel") - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - - By("4) Verify: validate consensus type value in all orderers") - // orderer2 - config = nwo.GetConfig(network, peer, o2, "testchannel") - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - - // orderer3 - config = nwo.GetConfig(network, peer, o3, "testchannel") - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - - // orderer4 - config = nwo.GetConfig(network, peer, o4, "testchannel") - consensusTypeValue = extractOrdererConsensusType(config) - validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) + By("4) Verify: validate consensus type value in all orderers o1, o2, o3, o4") + for _, o := range []*nwo.Orderer{o1, o2, o3, o4} { + config = nwo.GetConfig(network, peer, o, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) + } }) // This test executes the migration flow and checks that forbidden transitions are rejected. @@ -342,18 +337,18 @@ var _ = Describe("ConsensusTypeMigration", func() { // - Entry to & exit from maintenance mode can only change ConsensusType.State. // - In maintenance mode one can only change ConsensusType.Type & ConsensusType.Metadata & Orderers.ConsenterMapping. // - ConsensusType.Type can only change from "etcdraft" to "BFT", and only in maintenance mode. - It("executes raft2future forbidden transitions", func() { + It("executes raft2bft forbidden transitions", func() { // === Step 1: === By("1) Config update on standard channel, changing both ConsensusType State & Type is forbidden") assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "BFT", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "BFT", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) // === Step 2: === By("2) Config update on standard channel, both ConsensusType State & some other value is forbidden") config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) updateConfigWithBatchTimeout(updatedConfig) updateOrdererConfigFailed(network, o1, "testchannel", config, updatedConfig, peer, o1) @@ -361,7 +356,7 @@ var _ = Describe("ConsensusTypeMigration", func() { By("3) Config update on standard channel, State=MAINTENANCE, enter maintenance-mode") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, - "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) By("3) Verify: standard channel config changed") @@ -375,49 +370,152 @@ var _ = Describe("ConsensusTypeMigration", func() { By("4) Config update on standard channel, change ConsensusType.Type to unsupported type, forbidden") assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "hesse", nil, protosorderer.ConsensusType_STATE_MAINTENANCE) + "hesse", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) // === Step 5: === By("5) Config update on standard channel, change ConsensusType.Type and State, forbidden") assertTransitionFailed(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL) + "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL, 1) - // === Step 6: Config update on standard channel, changing both ConsensusType.Type and other value is permitted === - By("6) changing both ConsensusType.Type and other value is permitted") - // Change consensus type and batch-timeout + // === Step 6: Config update on standard channel, changing ConsensusType.Type with invalid bft metadata === + By("6) changing ConsensusType.Type with invalid BFT metadata") + invalidBftMetadata := protoutil.MarshalOrPanic(prepareInvalidBftMetadata()) + assertTransitionFailed(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", invalidBftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) + + By("6) changing ConsensusType.Type with missing BFT metadata") + assertTransitionFailed(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) + + By("6) changing ConsensusType.Type with missing bft consenters mapping") bftMetadata := protoutil.MarshalOrPanic(prepareBftMetadata()) + assertTransitionFailed(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) + + By("6) changing ConsensusType.Type with corrupt bft consenters mapping") + assertTransitionFailed(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 2) + + // === Step 7: Config update on standard channel, changing both ConsensusType.Type and other value is permitted === + By("7) changing both ConsensusType.Type and other value is permitted") + // Change consensus type and batch-timeout config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE) + "BFT", bftMetadata, protosorderer.ConsensusType_STATE_MAINTENANCE, 1) updateConfigWithBatchTimeout(updatedConfig) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("6) Verify: standard channel config changed") + By("7) Verify: standard channel config changed") std1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1BlockNum).To(Equal(std1StartBlockNum + 1)) config = nwo.GetConfig(network, peer, o1, "testchannel") consensusTypeValue = extractOrdererConsensusType(config) validateConsensusTypeValue(consensusTypeValue, "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE) - // === Step 7: === - By("7) Config update on standard channel, changing value other than ConsensusType.Type is permitted") + // === Step 8: === + By("8) Config update on standard channel, changing value other than ConsensusType.Type is permitted") updatedConfig = proto.Clone(config).(*common.Config) updateConfigWithBatchTimeout(updatedConfig) nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) - By("7) Verify: standard channel config changed") + By("8) Verify: standard channel config changed") std1BlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") Expect(std1BlockNum).To(Equal(std1StartBlockNum + 2)) - // === Step 8: === - By("8) Config update on standard channel, both ConsensusType State & some other value is forbidden") + // === Step 9: === + By("9) Config update on standard channel, both ConsensusType State & some other value is forbidden") config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", "BFT", protosorderer.ConsensusType_STATE_MAINTENANCE, - "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL) + "BFT", nil, protosorderer.ConsensusType_STATE_NORMAL, 1) updateConfigWithBatchTimeout(updatedConfig) updateOrdererConfigFailed(network, o1, "testchannel", config, updatedConfig, peer, o1) }) + + // Note: + // This test aims to check some other config value in maintenance mode, reboot, and exit maintenance mode. + // The config value is unrelated to consensus type migration. + It("Config value change unrelated to consensus type migration", func() { + // === Step 1: Config update on standard channel, MAINTENANCE === + By("1) Config update on standard channel, State=MAINTENANCE") + config, updatedConfig := prepareTransition(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_NORMAL, + "etcdraft", nil, protosorderer.ConsensusType_STATE_MAINTENANCE, 0) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + + By("1) Verify: standard channel config changed") + chan1StartBlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(chan1StartBlockNum).ToNot(Equal(0)) + + config = nwo.GetConfig(network, peer, o1, "testchannel") + consensusTypeValue := extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) + + By("1) Verify: Normal TX's on standard channel are blocked") + assertTxFailed(network, o1, "testchannel") + + // In maintenance mode deliver requests are open to those entities that satisfy the /Channel/Orderer/Readers policy + By("1) Verify: delivery request from peer is blocked") + err := checkPeerDeliverRequest(o1, peer, network, "testchannel") + Expect(err).To(MatchError(errors.New("FORBIDDEN"))) + + // === Step 2: config update on standard channel, State=MAINTENANCE, type=etcdraft === + By("2) Config update on standard channel, State=MAINTENANCE, type=etcdraft") + config = nwo.GetConfig(network, peer, o1, "testchannel") + consensusTypeValue = extractOrdererConsensusType(config) + validateConsensusTypeValue(consensusTypeValue, "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE) + updatedConfig = proto.Clone(config).(*common.Config) + updateConfigWithBatchTimeout(updatedConfig) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + + By("2) Verify: standard channel config changed") + chan1BlockNum := nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(chan1BlockNum).To(Equal(chan1StartBlockNum + 1)) + + // === Step 3: kill === + By("3) killing orderer1") + o1Proc.Signal(syscall.SIGKILL) + Eventually(o1Proc.Wait(), network.EventuallyTimeout).Should(Receive(MatchError("exit status 137"))) + + // === Step 4: restart === + By("4) restarting orderer1") + network.Consensus.Type = "etcdraft" + + o1Runner = network.OrdererRunner(o1) + o1Proc = ifrit.Invoke(o1Runner) + + Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + + assertBlockReception( + map[string]int{ + "testchannel": int(chan1BlockNum), + }, + []*nwo.Orderer{o1}, + peer, + network, + ) + + Eventually(o1Runner.Err(), network.EventuallyTimeout, time.Second).Should(gbytes.Say("Raft leader changed: 0 -> ")) + Eventually(o1Proc.Ready(), network.EventuallyTimeout).Should(BeClosed()) + + By("5) Release - executing config transaction on standard channel with restarted orderer") + config, updatedConfig = prepareTransition(network, peer, o1, "testchannel", + "etcdraft", protosorderer.ConsensusType_STATE_MAINTENANCE, + "etcdraft", nil, protosorderer.ConsensusType_STATE_NORMAL, 0) + nwo.UpdateOrdererConfig(network, o1, "testchannel", config, updatedConfig, peer, o1) + + By("6) Verify: standard channel config changed") + chan1BlockNum = nwo.CurrentConfigBlockNumber(network, peer, o1, "testchannel") + Expect(chan1BlockNum).To(Equal(chan1StartBlockNum + 2)) + + By("7) Executing transaction on standard channel with restarted orderer") + assertBlockCreation(network, o1, peer, "testchannel", chan1StartBlockNum+3) + assertBlockCreation(network, o1, nil, "testchannel", chan1StartBlockNum+4) + }) }) }) @@ -500,7 +598,7 @@ func updateOrdererConfigFailed(n *nwo.Network, orderer *nwo.Orderer, channel str func prepareTransition( network *nwo.Network, peer *nwo.Peer, orderer *nwo.Orderer, channel string, // Auxiliary fromConsensusType string, fromMigState protosorderer.ConsensusType_State, // From - toConsensusType string, toConsensusMetadata []byte, toMigState protosorderer.ConsensusType_State, // To + toConsensusType string, toConsensusMetadata []byte, toMigState protosorderer.ConsensusType_State, toConsenterMapping int, // To ) (current, updated *common.Config) { current = nwo.GetConfig(network, peer, orderer, channel) updated = proto.Clone(current).(*common.Config) @@ -508,7 +606,13 @@ func prepareTransition( validateConsensusTypeValue(consensusTypeValue, fromConsensusType, fromMigState) updateConfigWithConsensusType(toConsensusType, toConsensusMetadata, toMigState, updated, consensusTypeValue) if toConsensusType == "BFT" { - updateBFTOrderersConfig(network, updated) + // 1: updating valid consenters mapping + // 2: updating invalid consenters mapping + if toConsenterMapping == 1 { + updateBFTOrderersConfig(network, updated) + } else if toConsenterMapping == 2 { + updateInvalidBFTOrderersConfig(network, updated) + } } return current, updated } @@ -516,12 +620,12 @@ func prepareTransition( func assertTransitionFailed( network *nwo.Network, peer *nwo.Peer, orderer *nwo.Orderer, channel string, // Auxiliary fromConsensusType string, fromMigState protosorderer.ConsensusType_State, // From - toConsensusType string, toConsensusMetadata []byte, toMigState protosorderer.ConsensusType_State, // To + toConsensusType string, toConsensusMetadata []byte, toMigState protosorderer.ConsensusType_State, toConsenterMapping int, // To ) { current, updated := prepareTransition( network, peer, orderer, channel, fromConsensusType, fromMigState, - toConsensusType, toConsensusMetadata, toMigState) + toConsensusType, toConsensusMetadata, toMigState, toConsenterMapping) updateOrdererConfigFailed(network, orderer, channel, current, updated, peer, orderer) } @@ -602,6 +706,21 @@ func updateBFTOrderersConfig(network *nwo.Network, config *common.Config) { } } +func updateInvalidBFTOrderersConfig(network *nwo.Network, config *common.Config) { + orderersVal := &common.Orderers{ + ConsenterMapping: computeConsenterMappings(network), + } + + orderersVal.ConsenterMapping[0].Port = orderersVal.ConsenterMapping[0].Port + 1 + + policies.EncodeBFTBlockVerificationPolicy(orderersVal.ConsenterMapping, config.ChannelGroup.Groups["Orderer"]) + + config.ChannelGroup.Groups["Orderer"].Values["Orderers"] = &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(orderersVal), + ModPolicy: "/Channel/Orderer/Admins", + } +} + func computeConsenterMappings(network *nwo.Network) []*common.Consenter { var consenters []*common.Consenter @@ -685,3 +804,25 @@ func prepareBftMetadata() *smartbft.Options { } return bftMetadata } + +// prepareBftMetadata prepare the bft consensusType.Metadata +func prepareInvalidBftMetadata() *smartbft.Options { + bftMetadata := &smartbft.Options{ + RequestBatchMaxCount: 0, + RequestBatchMaxBytes: types.DefaultConfig.RequestBatchMaxBytes, + RequestBatchMaxInterval: types.DefaultConfig.RequestBatchMaxInterval.String(), + IncomingMessageBufferSize: types.DefaultConfig.IncomingMessageBufferSize, + RequestPoolSize: types.DefaultConfig.RequestPoolSize, + RequestForwardTimeout: types.DefaultConfig.RequestForwardTimeout.String(), + RequestComplainTimeout: types.DefaultConfig.RequestComplainTimeout.String(), + RequestAutoRemoveTimeout: types.DefaultConfig.RequestAutoRemoveTimeout.String(), + ViewChangeResendInterval: types.DefaultConfig.ViewChangeResendInterval.String(), + ViewChangeTimeout: types.DefaultConfig.ViewChangeTimeout.String(), + LeaderHeartbeatTimeout: types.DefaultConfig.LeaderHeartbeatTimeout.String(), + LeaderHeartbeatCount: types.DefaultConfig.LeaderHeartbeatCount, + CollectTimeout: types.DefaultConfig.CollectTimeout.String(), + SyncOnStart: types.DefaultConfig.SyncOnStart, + SpeedUpViewChange: types.DefaultConfig.SpeedUpViewChange, + } + return bftMetadata +} diff --git a/orderer/common/msgprocessor/maintenancefilter.go b/orderer/common/msgprocessor/maintenancefilter.go index 6a1b46a2040..3da2957ada7 100644 --- a/orderer/common/msgprocessor/maintenancefilter.go +++ b/orderer/common/msgprocessor/maintenancefilter.go @@ -11,10 +11,10 @@ import ( "time" "github.com/SmartBFT-Go/consensus/pkg/types" - "github.com/golang/protobuf/proto" cb "github.com/hyperledger/fabric-protos-go/common" "github.com/hyperledger/fabric-protos-go/orderer" + "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" "github.com/hyperledger/fabric-protos-go/orderer/smartbft" "github.com/hyperledger/fabric/bccsp" "github.com/hyperledger/fabric/common/channelconfig" @@ -142,9 +142,14 @@ func (mf *MaintenanceFilter) inspect(configEnvelope *cb.ConfigEnvelope, ordererC } _, err := validateBFTMetadataOptions(1, updatedMetadata) - if updatedMetadata.XXX_unrecognized != nil || err != nil { + if err != nil { return errors.New("invalid BFT metadata configuration") } + + err = validateBFTConsenterMapping(ordererConfig, nextOrdererConfig) + if err != nil { + return errors.Wrap(err, "invalid BFT consenter mapping configuration") + } } logger.Infof("[channel: %s] consensus-type migration: about to change from %s to %s", @@ -260,3 +265,38 @@ func validateBFTMetadataOptions(selfID uint64, options *smartbft.Options) (types return config, nil } + +func validateBFTConsenterMapping(currentOrdererConfig channelconfig.Orderer, nextOrdererConfig channelconfig.Orderer) error { + // extract raft consenters from consensusTypeValue.metadata + raftMetadata := &etcdraft.ConfigMetadata{} + proto.Unmarshal(currentOrdererConfig.ConsensusMetadata(), raftMetadata) + raftConsenters := raftMetadata.GetConsenters() + + // extract bft consenters + bftConsenters := nextOrdererConfig.Consenters() + + if len(bftConsenters) == 0 { + return errors.Errorf("Invalid new config: bft consenters are missing") + } + + if len(raftConsenters) != len(bftConsenters) { + return errors.Errorf("Invalid new config: the number of bft consenters: %d is not equal to the number of raft consenters: %d", len(bftConsenters), len(raftConsenters)) + } + + for _, raftConsenter := range raftConsenters { + flag := false + for _, bftConsenter := range bftConsenters { + if raftConsenter.Port == bftConsenter.Port && raftConsenter.Host == bftConsenter.Host && + bytes.Equal(raftConsenter.ServerTlsCert, bftConsenter.ServerTlsCert) && + bytes.Equal(raftConsenter.ClientTlsCert, bftConsenter.ClientTlsCert) { + flag = true + break + } + } + if !flag { + return errors.Errorf("No suitable BFT consenter for Raft consenter: %v", raftConsenter) + } + } + + return nil +} diff --git a/orderer/common/msgprocessor/maintenancefilter_test.go b/orderer/common/msgprocessor/maintenancefilter_test.go index eace50e2f52..ccd6cbceb96 100644 --- a/orderer/common/msgprocessor/maintenancefilter_test.go +++ b/orderer/common/msgprocessor/maintenancefilter_test.go @@ -12,6 +12,8 @@ import ( "path" "testing" + "github.com/hyperledger/fabric-protos-go/orderer/etcdraft" + "github.com/SmartBFT-Go/consensus/pkg/types" "github.com/golang/protobuf/proto" "github.com/hyperledger/fabric-protos-go/orderer/smartbft" @@ -38,6 +40,12 @@ func newMockOrdererConfig(migration bool, state orderer.ConsensusType_State) *mo mockOrderer.CapabilitiesReturns(mockCapabilities) mockOrderer.ConsensusTypeReturns("etcdraft") mockOrderer.ConsensusStateReturns(state) + mockOrderer.ConsensusMetadataReturns(protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{ + Consenters: []*etcdraft.Consenter{ + {Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, + {Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}, + }, + })) return mockOrderer } @@ -61,10 +69,11 @@ func TestMaintenanceDisabled(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msInactive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + raftMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{Consenters: []*etcdraft.Consenter{{Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: raftMetadata, state: orderer.ConsensusType_STATE_NORMAL} t.Run("Good", func(t *testing.T) { - configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, 3) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, 3, 0) err := mf.Apply(configTx) require.NoError(t, err) }) @@ -173,12 +182,12 @@ func TestMaintenanceInspectEntry(t *testing.T) { mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) bogusMetadata := []byte{1, 2, 3, 4} - validMetadata := []byte{} + validRaftMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{Consenters: []*etcdraft.Consenter{{Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}) bftMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) - current := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + current := consensusTypeInfo{ordererType: "etcdraft", metadata: validRaftMetadata, state: orderer.ConsensusType_STATE_NORMAL} t.Run("Good", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: validRaftMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.NoError(t, err) @@ -201,8 +210,8 @@ func TestMaintenanceInspectEntry(t *testing.T) { }) t.Run("Bad: concurrent change to state & orderer value", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + next := consensusTypeInfo{ordererType: "etcdraft", metadata: validRaftMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 1) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to groups within the Orderer group") @@ -226,18 +235,19 @@ func TestMaintenanceInspectChange(t *testing.T) { mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) bogusMetadata := []byte{1, 2, 3, 4} - validMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) - current := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_MAINTENANCE} + validBFTMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) + raftMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{Consenters: []*etcdraft.Consenter{{Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: raftMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - t.Run("Good type change with valid BFT metadata and consenter mapping", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + t.Run("Good type change with valid BFT metadata and suitable consenter mapping", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validBFTMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 1) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good exit, no change", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "etcdraft", metadata: []byte{}, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "etcdraft", metadata: raftMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.NoError(t, err) @@ -261,6 +271,33 @@ func TestMaintenanceInspectChange(t *testing.T) { "config transaction inspection failed: failed to unmarshal BFT metadata configuration") }) + t.Run("Bad: good type change with valid BFT metadata but missing consenters mapping", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validBFTMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelope(t, current, next) + err := mf.Apply(configTx) + require.Error(t, err) + require.EqualError(t, err, + "config transaction inspection failed: invalid BFT consenter mapping configuration: Invalid new config: bft consenters are missing") + }) + + t.Run("Bad: good type change with valid BFT metadata but corrupt consenters mapping", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validBFTMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 2) + err := mf.Apply(configTx) + require.Error(t, err) + require.EqualError(t, err, + "config transaction inspection failed: invalid BFT consenter mapping configuration: No suitable BFT consenter for Raft consenter: host:\"127.0.0.1\" port:4000 client_tls_cert:\"\\001\\002\\003\" server_tls_cert:\"\\004\\005\\006\" ") + }) + + t.Run("Bad: good type change with valid BFT metadata but missing consenters", func(t *testing.T) { + next := consensusTypeInfo{ordererType: "BFT", metadata: validBFTMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 3) + err := mf.Apply(configTx) + require.Error(t, err) + require.EqualError(t, err, + "config transaction inspection failed: invalid BFT consenter mapping configuration: Invalid new config: the number of bft consenters: 1 is not equal to the number of raft consenters: 2") + }) + t.Run("Bad: unsupported consensus type", func(t *testing.T) { next := consensusTypeInfo{ordererType: "unsupported", metadata: bogusMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} configTx := makeConfigEnvelope(t, current, next) @@ -270,7 +307,7 @@ func TestMaintenanceInspectChange(t *testing.T) { }) t.Run("Bad: concurrent change to consensus type & state", func(t *testing.T) { - next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} + next := consensusTypeInfo{ordererType: "BFT", metadata: validBFTMetadata, state: orderer.ConsensusType_STATE_NORMAL} configTx := makeConfigEnvelope(t, current, next) err := mf.Apply(configTx) require.EqualError(t, err, @@ -318,28 +355,28 @@ func TestMaintenanceInspectExit(t *testing.T) { t.Run("Bad: exit with extra group", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1, 1) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to more than one group") }) t.Run("Bad: exit with extra value", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2, 1) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to values in group Channel") }) t.Run("Bad: exit with extra orderer value", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3, 0) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contain more then just the ConsensusType value in the Orderer group") }) t.Run("Bad: exit with extra orderer value required for BFT", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_NORMAL} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 1) err := mf.Apply(configTx) require.EqualError(t, err, "config transaction inspection failed: config update contains changes to groups within the Orderer group") }) @@ -353,33 +390,34 @@ func TestMaintenanceExtra(t *testing.T) { require.NoError(t, err) mf := NewMaintenanceFilter(msActive, cryptoProvider) require.NotNil(t, mf) - current := consensusTypeInfo{ordererType: "etcdraft", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} + raftMetadata := protoutil.MarshalOrPanic(&etcdraft.ConfigMetadata{Consenters: []*etcdraft.Consenter{{Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}) + current := consensusTypeInfo{ordererType: "etcdraft", metadata: raftMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} validMetadata := protoutil.MarshalOrPanic(createValidBFTMetadata()) t.Run("Good: with extra group", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 1, 1) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good: with extra value", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 2, 1) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good: with extra orderer value", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 3, 1) err := mf.Apply(configTx) require.NoError(t, err) }) t.Run("Good: with extra orderer value required for BFT", func(t *testing.T) { next := consensusTypeInfo{ordererType: "BFT", metadata: validMetadata, state: orderer.ConsensusType_STATE_MAINTENANCE} - configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 4) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, next, 0, 1) err := mf.Apply(configTx) require.NoError(t, err) }) @@ -395,7 +433,7 @@ func TestMaintenanceMissingConsensusType(t *testing.T) { require.NotNil(t, mf) current := consensusTypeInfo{ordererType: "etcdraft", metadata: nil, state: orderer.ConsensusType_STATE_MAINTENANCE} for i := 1; i < 4; i++ { - configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, i) + configTx := makeConfigEnvelopeWithExtraStuff(t, current, current, i, 1) err := mf.Apply(configTx) require.NoError(t, err) } @@ -436,7 +474,7 @@ func makeConfigEnvelope(t *testing.T, current, next consensusTypeInfo) *common.E return configTx } -func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeInfo, extra int) *common.Envelope { +func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeInfo, extra int, addBFTConsenterMapping int) *common.Envelope { original := makeBaseConfig(t) updated := proto.Clone(original).(*common.Config) @@ -479,13 +517,24 @@ func makeConfigEnvelopeWithExtraStuff(t *testing.T, current, next consensusTypeI }), ModPolicy: channelconfig.AdminsPolicyKey, } - case 4: + } + + switch addBFTConsenterMapping { + case 1: + updated.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.OrderersKey] = &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(&common.Orderers{ConsenterMapping: []*common.Consenter{{Id: 1, Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Id: 2, Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}), + ModPolicy: channelconfig.AdminsPolicyKey, + } + case 2: + updated.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.OrderersKey] = &common.ConfigValue{ + Value: protoutil.MarshalOrPanic(&common.Orderers{ConsenterMapping: []*common.Consenter{{Id: 1, Host: "127.0.0.1", Port: 4005, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}, {Id: 2, Host: "127.0.0.1", Port: 4001, ClientTlsCert: []byte{7, 8, 9}, ServerTlsCert: []byte{1, 2, 3}}}}), + ModPolicy: channelconfig.AdminsPolicyKey, + } + case 3: updated.ChannelGroup.Groups[channelconfig.OrdererGroupKey].Values[channelconfig.OrderersKey] = &common.ConfigValue{ - Value: protoutil.MarshalOrPanic(&common.Orderers{ConsenterMapping: []*common.Consenter{{Id: 1, Host: "", Port: 0}}}), + Value: protoutil.MarshalOrPanic(&common.Orderers{ConsenterMapping: []*common.Consenter{{Id: 1, Host: "127.0.0.1", Port: 4000, ClientTlsCert: []byte{1, 2, 3}, ServerTlsCert: []byte{4, 5, 6}}}}), ModPolicy: channelconfig.AdminsPolicyKey, } - default: - return nil } configTx := makeConfigTx(original, updated, t) diff --git a/orderer/consensus/etcdraft/chain_test.go b/orderer/consensus/etcdraft/chain_test.go index 96388d745cd..09b446ec417 100644 --- a/orderer/consensus/etcdraft/chain_test.go +++ b/orderer/consensus/etcdraft/chain_test.go @@ -553,7 +553,8 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 4, Value: marshalOrPanic(&orderer.ConsensusType{ - Type: "etcdraft", + Type: "etcdraft", + Metadata: []byte{1, 2, 3}, }), }, } @@ -561,7 +562,8 @@ var _ = Describe("Chain", func() { "ConsensusType": { Version: 4, Value: marshalOrPanic(&orderer.ConsensusType{ - Type: "etcdraft", + Type: "etcdraft", + Metadata: []byte{1, 2, 3}, }), }, } diff --git a/orderer/consensus/etcdraft/util.go b/orderer/consensus/etcdraft/util.go index 022ec8ace30..b158a2d0cba 100644 --- a/orderer/consensus/etcdraft/util.go +++ b/orderer/consensus/etcdraft/util.go @@ -92,9 +92,7 @@ func MetadataFromConfigValue(configValue *common.ConfigValue) (*etcdraft.ConfigM } if consensusTypeValue.Type != "etcdraft" { - if consensusTypeValue.Type == "BFT" { - return nil, consensusTypeValue, nil - } + return nil, consensusTypeValue, nil } updatedMetadata := &etcdraft.ConfigMetadata{} diff --git a/orderer/consensus/etcdraft/util_test.go b/orderer/consensus/etcdraft/util_test.go index 677915f2f06..c3dc659de07 100644 --- a/orderer/consensus/etcdraft/util_test.go +++ b/orderer/consensus/etcdraft/util_test.go @@ -413,6 +413,8 @@ func TestVerifyConfigMetadata(t *testing.T) { }) } +// This test checks that MetadataFromConfigValue, which reads and translates configuration updates from config value, +// returns an empty raft metadata when the consensus type is BFT. func TestMetadataFromConfigValue(t *testing.T) { configValue := &common.ConfigValue{ Value: protoutil.MarshalOrPanic(&orderer.ConsensusType{ From 903f2ead762affe0ccb62b04e934ca097d8b870d Mon Sep 17 00:00:00 2001 From: May Rosenbaum Date: Sun, 21 Jan 2024 16:50:21 +0200 Subject: [PATCH 4/4] break import cycle to prevent code duplication Signed-off-by: May Rosenbaum --- .../common/msgprocessor/maintenancefilter.go | 62 +--------------- orderer/consensus/smartbft/consenter.go | 4 +- orderer/consensus/smartbft/util.go | 60 +--------------- orderer/consensus/smartbft/util/util.go | 71 +++++++++++++++++++ 4 files changed, 79 insertions(+), 118 deletions(-) create mode 100644 orderer/consensus/smartbft/util/util.go diff --git a/orderer/common/msgprocessor/maintenancefilter.go b/orderer/common/msgprocessor/maintenancefilter.go index 3da2957ada7..25566eb8c2b 100644 --- a/orderer/common/msgprocessor/maintenancefilter.go +++ b/orderer/common/msgprocessor/maintenancefilter.go @@ -8,9 +8,9 @@ package msgprocessor import ( "bytes" - "time" - "github.com/SmartBFT-Go/consensus/pkg/types" + "github.com/hyperledger/fabric/orderer/consensus/smartbft/util" + "github.com/golang/protobuf/proto" cb "github.com/hyperledger/fabric-protos-go/common" "github.com/hyperledger/fabric-protos-go/orderer" @@ -141,7 +141,7 @@ func (mf *MaintenanceFilter) inspect(configEnvelope *cb.ConfigEnvelope, ordererC return errors.Wrap(err, "failed to unmarshal BFT metadata configuration") } - _, err := validateBFTMetadataOptions(1, updatedMetadata) + _, err := util.ConfigFromMetadataOptions(1, updatedMetadata) if err != nil { return errors.New("invalid BFT metadata configuration") } @@ -210,62 +210,6 @@ func (mf *MaintenanceFilter) ensureConsensusTypeChangeOnly(configEnvelope *cb.Co return nil } -func validateBFTMetadataOptions(selfID uint64, options *smartbft.Options) (types.Configuration, error) { - var err error - - config := types.DefaultConfig - config.SelfID = selfID - - if options == nil { - return config, errors.New("config metadata options field is nil") - } - - config.RequestBatchMaxCount = options.RequestBatchMaxCount - config.RequestBatchMaxBytes = options.RequestBatchMaxBytes - if config.RequestBatchMaxInterval, err = time.ParseDuration(options.RequestBatchMaxInterval); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestBatchMaxInterval") - } - config.IncomingMessageBufferSize = options.IncomingMessageBufferSize - config.RequestPoolSize = options.RequestPoolSize - if config.RequestForwardTimeout, err = time.ParseDuration(options.RequestForwardTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestForwardTimeout") - } - if config.RequestComplainTimeout, err = time.ParseDuration(options.RequestComplainTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestComplainTimeout") - } - if config.RequestAutoRemoveTimeout, err = time.ParseDuration(options.RequestAutoRemoveTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestAutoRemoveTimeout") - } - if config.ViewChangeResendInterval, err = time.ParseDuration(options.ViewChangeResendInterval); err != nil { - return config, errors.Wrap(err, "bad config metadata option ViewChangeResendInterval") - } - if config.ViewChangeTimeout, err = time.ParseDuration(options.ViewChangeTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option ViewChangeTimeout") - } - if config.LeaderHeartbeatTimeout, err = time.ParseDuration(options.LeaderHeartbeatTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option LeaderHeartbeatTimeout") - } - config.LeaderHeartbeatCount = options.LeaderHeartbeatCount - if config.CollectTimeout, err = time.ParseDuration(options.CollectTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option CollectTimeout") - } - config.SyncOnStart = options.SyncOnStart - config.SpeedUpViewChange = options.SpeedUpViewChange - - config.LeaderRotation = false - config.DecisionsPerLeader = 0 - - if err = config.Validate(); err != nil { - return config, errors.Wrap(err, "config validation failed") - } - - if options.RequestMaxBytes == 0 { - config.RequestMaxBytes = config.RequestBatchMaxBytes - } - - return config, nil -} - func validateBFTConsenterMapping(currentOrdererConfig channelconfig.Orderer, nextOrdererConfig channelconfig.Orderer) error { // extract raft consenters from consensusTypeValue.metadata raftMetadata := &etcdraft.ConfigMetadata{} diff --git a/orderer/consensus/smartbft/consenter.go b/orderer/consensus/smartbft/consenter.go index 0b8bc12763c..ed0a5468f01 100644 --- a/orderer/consensus/smartbft/consenter.go +++ b/orderer/consensus/smartbft/consenter.go @@ -15,6 +15,8 @@ import ( "path" "reflect" + "github.com/hyperledger/fabric/orderer/consensus/smartbft/util" + "github.com/SmartBFT-Go/consensus/pkg/api" "github.com/SmartBFT-Go/consensus/pkg/wal" "github.com/golang/protobuf/proto" @@ -188,7 +190,7 @@ func (c *Consenter) HandleChain(support consensus.ConsenterSupport, metadata *cb c.Logger.Panicf("Failed initializing block puller") } - config, err := configFromMetadataOptions((uint64)(selfID), configOptions) + config, err := util.ConfigFromMetadataOptions((uint64)(selfID), configOptions) if err != nil { return nil, errors.Wrap(err, "failed parsing smartbft configuration") } diff --git a/orderer/consensus/smartbft/util.go b/orderer/consensus/smartbft/util.go index e2bf9779c69..c7720902572 100644 --- a/orderer/consensus/smartbft/util.go +++ b/orderer/consensus/smartbft/util.go @@ -14,7 +14,6 @@ import ( "encoding/pem" "fmt" "sort" - "time" "github.com/SmartBFT-Go/consensus/pkg/types" "github.com/SmartBFT-Go/consensus/smartbftprotos" @@ -30,6 +29,7 @@ import ( "github.com/hyperledger/fabric/orderer/common/localconfig" "github.com/hyperledger/fabric/orderer/consensus" "github.com/hyperledger/fabric/orderer/consensus/etcdraft" + "github.com/hyperledger/fabric/orderer/consensus/smartbft/util" "github.com/hyperledger/fabric/protoutil" "github.com/pkg/errors" ) @@ -119,7 +119,7 @@ func configBlockToBFTConfig(selfID uint64, block *cb.Block, bccsp bccsp.BCCSP) ( return types.Configuration{}, err } - return configFromMetadataOptions(selfID, consensusConfigOptions) + return util.ConfigFromMetadataOptions(selfID, consensusConfigOptions) } //go:generate counterfeiter -o mocks/mock_blockpuller.go . BlockPuller @@ -189,62 +189,6 @@ func getViewMetadataFromBlock(block *cb.Block) (*smartbftprotos.ViewMetadata, er return &viewMetadata, nil } -func configFromMetadataOptions(selfID uint64, options *smartbft.Options) (types.Configuration, error) { - var err error - - config := types.DefaultConfig - config.SelfID = selfID - - if options == nil { - return config, errors.New("config metadata options field is nil") - } - - config.RequestBatchMaxCount = options.RequestBatchMaxCount - config.RequestBatchMaxBytes = options.RequestBatchMaxBytes - if config.RequestBatchMaxInterval, err = time.ParseDuration(options.RequestBatchMaxInterval); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestBatchMaxInterval") - } - config.IncomingMessageBufferSize = options.IncomingMessageBufferSize - config.RequestPoolSize = options.RequestPoolSize - if config.RequestForwardTimeout, err = time.ParseDuration(options.RequestForwardTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestForwardTimeout") - } - if config.RequestComplainTimeout, err = time.ParseDuration(options.RequestComplainTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestComplainTimeout") - } - if config.RequestAutoRemoveTimeout, err = time.ParseDuration(options.RequestAutoRemoveTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option RequestAutoRemoveTimeout") - } - if config.ViewChangeResendInterval, err = time.ParseDuration(options.ViewChangeResendInterval); err != nil { - return config, errors.Wrap(err, "bad config metadata option ViewChangeResendInterval") - } - if config.ViewChangeTimeout, err = time.ParseDuration(options.ViewChangeTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option ViewChangeTimeout") - } - if config.LeaderHeartbeatTimeout, err = time.ParseDuration(options.LeaderHeartbeatTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option LeaderHeartbeatTimeout") - } - config.LeaderHeartbeatCount = options.LeaderHeartbeatCount - if config.CollectTimeout, err = time.ParseDuration(options.CollectTimeout); err != nil { - return config, errors.Wrap(err, "bad config metadata option CollectTimeout") - } - config.SyncOnStart = options.SyncOnStart - config.SpeedUpViewChange = options.SpeedUpViewChange - - config.LeaderRotation = false - config.DecisionsPerLeader = 0 - - if err = config.Validate(); err != nil { - return config, errors.Wrap(err, "config validation failed") - } - - if options.RequestMaxBytes == 0 { - config.RequestMaxBytes = config.RequestBatchMaxBytes - } - - return config, nil -} - type request struct { sigHdr *cb.SignatureHeader envelope *cb.Envelope diff --git a/orderer/consensus/smartbft/util/util.go b/orderer/consensus/smartbft/util/util.go new file mode 100644 index 00000000000..9e18a2b4713 --- /dev/null +++ b/orderer/consensus/smartbft/util/util.go @@ -0,0 +1,71 @@ +/* +Copyright IBM Corp. All Rights Reserved. + +SPDX-License-Identifier: Apache-2.0 +*/ + +package util + +import ( + "time" + + "github.com/SmartBFT-Go/consensus/pkg/types" + "github.com/hyperledger/fabric-protos-go/orderer/smartbft" + "github.com/pkg/errors" +) + +func ConfigFromMetadataOptions(selfID uint64, options *smartbft.Options) (types.Configuration, error) { + var err error + + config := types.DefaultConfig + config.SelfID = selfID + + if options == nil { + return config, errors.New("config metadata options field is nil") + } + + config.RequestBatchMaxCount = options.RequestBatchMaxCount + config.RequestBatchMaxBytes = options.RequestBatchMaxBytes + if config.RequestBatchMaxInterval, err = time.ParseDuration(options.RequestBatchMaxInterval); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestBatchMaxInterval") + } + config.IncomingMessageBufferSize = options.IncomingMessageBufferSize + config.RequestPoolSize = options.RequestPoolSize + if config.RequestForwardTimeout, err = time.ParseDuration(options.RequestForwardTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestForwardTimeout") + } + if config.RequestComplainTimeout, err = time.ParseDuration(options.RequestComplainTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestComplainTimeout") + } + if config.RequestAutoRemoveTimeout, err = time.ParseDuration(options.RequestAutoRemoveTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option RequestAutoRemoveTimeout") + } + if config.ViewChangeResendInterval, err = time.ParseDuration(options.ViewChangeResendInterval); err != nil { + return config, errors.Wrap(err, "bad config metadata option ViewChangeResendInterval") + } + if config.ViewChangeTimeout, err = time.ParseDuration(options.ViewChangeTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option ViewChangeTimeout") + } + if config.LeaderHeartbeatTimeout, err = time.ParseDuration(options.LeaderHeartbeatTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option LeaderHeartbeatTimeout") + } + config.LeaderHeartbeatCount = options.LeaderHeartbeatCount + if config.CollectTimeout, err = time.ParseDuration(options.CollectTimeout); err != nil { + return config, errors.Wrap(err, "bad config metadata option CollectTimeout") + } + config.SyncOnStart = options.SyncOnStart + config.SpeedUpViewChange = options.SpeedUpViewChange + + config.LeaderRotation = false + config.DecisionsPerLeader = 0 + + if err = config.Validate(); err != nil { + return config, errors.Wrap(err, "config validation failed") + } + + if options.RequestMaxBytes == 0 { + config.RequestMaxBytes = config.RequestBatchMaxBytes + } + + return config, nil +}