diff --git a/common/cauthdsl/policy_test.go b/common/cauthdsl/policy_test.go index ac8a5059895..e010983cc6a 100644 --- a/common/cauthdsl/policy_test.go +++ b/common/cauthdsl/policy_test.go @@ -24,6 +24,7 @@ import ( cb "github.com/hyperledger/fabric/protos/common" "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/assert" ) var acceptAllPolicy *cb.Policy @@ -56,17 +57,6 @@ func makePolicySource(policyResult bool) *cb.Policy { } } -func addPolicy(manager policies.Proposer, id string, policy *cb.Policy) { - manager.BeginPolicyProposals(id, nil) - _, err := manager.ProposePolicy(id, id, &cb.ConfigPolicy{ - Policy: policy, - }) - if err != nil { - panic(err) - } - manager.CommitProposals(id) -} - func providerMap() map[int32]policies.Provider { r := make(map[int32]policies.Provider) r[int32(cb.Policy_SIGNATURE)] = NewPolicyProvider(&mockDeserializer{}) @@ -75,40 +65,41 @@ func providerMap() map[int32]policies.Provider { func TestAccept(t *testing.T) { policyID := "policyID" - m := policies.NewManagerImpl("test", providerMap()) - addPolicy(m, policyID, acceptAllPolicy) + m, err := policies.NewManagerImpl("test", providerMap(), &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + policyID: &cb.ConfigPolicy{Policy: acceptAllPolicy}, + }, + }) + assert.NoError(t, err) + assert.NotNil(t, m) + policy, ok := m.GetPolicy(policyID) - if !ok { - t.Error("Should have found policy which was just added, but did not") - } - err := policy.Evaluate([]*cb.SignedData{}) - if err != nil { - t.Fatalf("Should not have errored evaluating an acceptAll policy: %s", err) - } + assert.True(t, ok, "Should have found policy which was just added, but did not") + err = policy.Evaluate([]*cb.SignedData{}) + assert.NoError(t, err, "Should not have errored evaluating an acceptAll policy") } func TestReject(t *testing.T) { policyID := "policyID" - m := policies.NewManagerImpl("test", providerMap()) - addPolicy(m, policyID, rejectAllPolicy) + m, err := policies.NewManagerImpl("test", providerMap(), &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + policyID: &cb.ConfigPolicy{Policy: rejectAllPolicy}, + }, + }) + assert.NoError(t, err) + assert.NotNil(t, m) policy, ok := m.GetPolicy(policyID) - if !ok { - t.Error("Should have found policy which was just added, but did not") - } - err := policy.Evaluate([]*cb.SignedData{}) - if err == nil { - t.Fatal("Should have errored evaluating the rejectAll policy") - } + assert.True(t, ok, "Should have found policy which was just added, but did not") + err = policy.Evaluate([]*cb.SignedData{}) + assert.Error(t, err, "Should have errored evaluating an rejectAll policy") } func TestRejectOnUnknown(t *testing.T) { - m := policies.NewManagerImpl("test", providerMap()) + m, err := policies.NewManagerImpl("test", providerMap(), &cb.ConfigGroup{}) + assert.NoError(t, err) + assert.NotNil(t, m) policy, ok := m.GetPolicy("FakePolicyID") - if ok { - t.Error("Should not have found policy which was never added, but did") - } - err := policy.Evaluate([]*cb.SignedData{}) - if err == nil { - t.Fatal("Should have errored evaluating the default policy") - } + assert.False(t, ok, "Should not have found policy which was never added, but did") + err = policy.Evaluate([]*cb.SignedData{}) + assert.Error(t, err, "Should have errored evaluating the default policy") } diff --git a/common/channelconfig/bundle.go b/common/channelconfig/bundle.go index d6f2e2dc446..fa5a788e21e 100644 --- a/common/channelconfig/bundle.go +++ b/common/channelconfig/bundle.go @@ -154,8 +154,7 @@ func NewBundle(channelID string, config *cb.Config) (*Bundle, error) { return nil, errors.Wrap(err, "initializing config values failed") } - policyManager := policies.NewManagerImpl(RootGroupKey, policyProviderMap) - err = InitializePolicyManager(policyManager, config.ChannelGroup) + policyManager, err := policies.NewManagerImpl(RootGroupKey, policyProviderMap, config.ChannelGroup) if err != nil { return nil, errors.Wrap(err, "initializing policymanager failed") } diff --git a/common/channelconfig/util.go b/common/channelconfig/util.go index fcc88398026..4438bce28bd 100644 --- a/common/channelconfig/util.go +++ b/common/channelconfig/util.go @@ -8,44 +8,11 @@ package channelconfig import ( "github.com/hyperledger/fabric/common/config" - "github.com/hyperledger/fabric/common/policies" cb "github.com/hyperledger/fabric/protos/common" "github.com/pkg/errors" ) -// InitializePolicyManager takes a config group and uses it to initialize a PolicyManager -// XXX This goes away by the end of the CR series so no test logic -func InitializePolicyManager(pm policies.Proposer, group *cb.ConfigGroup) error { - subGroups := make([]string, len(group.Groups)) - i := 0 - for subGroup := range group.Groups { - subGroups[i] = subGroup - i++ - } - - subPolicyHandlers, err := pm.BeginPolicyProposals("", subGroups) - if err != nil { - return err - } - - for key, policy := range group.Policies { - _, err := pm.ProposePolicy("", key, policy) - if err != nil { - return err - } - } - - for i := range subGroups { - if err := InitializePolicyManager(subPolicyHandlers[i], group.Groups[subGroups[i]]); err != nil { - return errors.Wrapf(err, "failed initializing subgroup %s", subGroups[i]) - } - } - - pm.CommitProposals("") - return nil -} - // InitializeConfigValues takes a config group and uses it to initialize a config Root // XXX This goes away by the end of the CR series so no test logic func InitializeConfigValues(vp config.ValueProposer, group *cb.ConfigGroup) error { diff --git a/common/config/resources/initializer.go b/common/config/resources/initializer.go index 03d42c2fa0c..06061be3835 100644 --- a/common/config/resources/initializer.go +++ b/common/config/resources/initializer.go @@ -17,47 +17,20 @@ import ( "github.com/hyperledger/fabric/common/policies" "github.com/hyperledger/fabric/msp" cb "github.com/hyperledger/fabric/protos/common" - - "github.com/golang/protobuf/proto" + "github.com/hyperledger/fabric/protos/utils" ) const RootGroupKey = "Resources" -type policyProposerRoot struct { - policyManager *policies.ManagerImpl -} - -// BeginPolicyProposals is used to start a new config proposal -func (p *policyProposerRoot) BeginPolicyProposals(tx interface{}, groups []string) ([]policies.Proposer, error) { - if len(groups) != 1 { - logger.Panicf("Initializer only supports having one root group") - } - return []policies.Proposer{p.policyManager}, nil -} - -func (i *policyProposerRoot) ProposePolicy(tx interface{}, key string, policy *cb.ConfigPolicy) (proto.Message, error) { - return nil, fmt.Errorf("Programming error, this should never be invoked") -} - -// PreCommit is a no-op and returns nil -func (i *policyProposerRoot) PreCommit(tx interface{}) error { - return nil -} - -// RollbackConfig is used to abandon a new config proposal -func (i *policyProposerRoot) RollbackProposals(tx interface{}) {} - -// CommitConfig is used to commit a new config proposal -func (i *policyProposerRoot) CommitProposals(tx interface{}) {} - type Bundle struct { - ppr *policyProposerRoot vpr *valueProposerRoot cm configtxapi.Manager pm policies.Manager + rpm policies.Manager } // New creates a new resources config bundle +// TODO, change interface to take config and not an envelope func New(envConfig *cb.Envelope, mspManager msp.MSPManager, channelPolicyManager policies.Manager) (*Bundle, error) { policyProviderMap := make(map[int32]policies.Provider) for pType := range cb.Policy_PolicyType_name { @@ -72,31 +45,35 @@ func New(envConfig *cb.Envelope, mspManager msp.MSPManager, channelPolicyManager } } - b := &Bundle{ - vpr: newValueProposerRoot(), - ppr: &policyProposerRoot{ - policyManager: policies.NewManagerImpl(RootGroupKey, policyProviderMap), - }, - } - b.pm = &policyRouter{ - channelPolicyManager: channelPolicyManager, - resourcesPolicyManager: b.ppr.policyManager, + payload, err := utils.UnmarshalPayload(envConfig.Payload) + if err != nil { + return nil, err } - var err error - b.cm, err = configtx.NewManagerImpl(envConfig, b) + configEnvelope, err := configtx.UnmarshalConfigEnvelope(payload.Data) if err != nil { return nil, err } - configEnvelope := b.cm.ConfigEnvelope() + if configEnvelope.Config == nil || configEnvelope.Config.ChannelGroup == nil { return nil, fmt.Errorf("config is nil") } - err = newchannelconfig.InitializePolicyManager(b.ppr, &cb.ConfigGroup{ - Groups: map[string]*cb.ConfigGroup{ - RootGroupKey: configEnvelope.Config.ChannelGroup, + + resourcesPolicyManager, err := policies.NewManagerImpl(RootGroupKey, policyProviderMap, configEnvelope.Config.ChannelGroup) + if err != nil { + return nil, err + } + + b := &Bundle{ + vpr: newValueProposerRoot(), + rpm: resourcesPolicyManager, + pm: &policyRouter{ + channelPolicyManager: channelPolicyManager, + resourcesPolicyManager: resourcesPolicyManager, }, - }) + } + + b.cm, err = configtx.NewManagerImpl(envConfig, b) if err != nil { return nil, err } @@ -117,10 +94,6 @@ func (b *Bundle) RootGroupKey() string { return RootGroupKey } -func (b *Bundle) PolicyProposer() policies.Proposer { - return b.ppr -} - func (b *Bundle) ValueProposer() config.ValueProposer { return b.vpr } diff --git a/common/config/resources/initializer_test.go b/common/config/resources/initializer_test.go index d18429ccb65..16305d4efbd 100644 --- a/common/config/resources/initializer_test.go +++ b/common/config/resources/initializer_test.go @@ -69,7 +69,6 @@ func TestBundleGreenPath(t *testing.T) { t.Run("Code coverage nits", func(t *testing.T) { assert.Equal(t, b.RootGroupKey(), RootGroupKey) - assert.NotNil(t, b.PolicyProposer()) assert.NotNil(t, b.ValueProposer()) assert.NotNil(t, b.ConfigtxManager()) assert.NotNil(t, b.PolicyManager()) diff --git a/common/policies/implicitmeta.go b/common/policies/implicitmeta.go index 247d1eccf31..69285e16d0d 100644 --- a/common/policies/implicitmeta.go +++ b/common/policies/implicitmeta.go @@ -25,44 +25,45 @@ import ( ) type implicitMetaPolicy struct { - conf *cb.ImplicitMetaPolicy threshold int subPolicies []Policy } // NewPolicy creates a new policy based on the policy bytes -func newImplicitMetaPolicy(data []byte) (*implicitMetaPolicy, error) { - imp := &cb.ImplicitMetaPolicy{} - if err := proto.Unmarshal(data, imp); err != nil { +func newImplicitMetaPolicy(data []byte, managers map[string]*ManagerImpl) (*implicitMetaPolicy, error) { + definition := &cb.ImplicitMetaPolicy{} + if err := proto.Unmarshal(data, definition); err != nil { return nil, fmt.Errorf("Error unmarshaling to ImplicitMetaPolicy: %s", err) } - return &implicitMetaPolicy{ - conf: imp, - }, nil -} + subPolicies := make([]Policy, len(managers)) -func (imp *implicitMetaPolicy) initialize(config *policyConfig) { - imp.subPolicies = make([]Policy, len(config.managers)) i := 0 - for _, manager := range config.managers { - imp.subPolicies[i], _ = manager.GetPolicy(imp.conf.SubPolicy) + for _, manager := range managers { + subPolicies[i], _ = manager.GetPolicy(definition.SubPolicy) i++ } - switch imp.conf.Rule { + var threshold int + + switch definition.Rule { case cb.ImplicitMetaPolicy_ANY: - imp.threshold = 1 + threshold = 1 case cb.ImplicitMetaPolicy_ALL: - imp.threshold = len(imp.subPolicies) + threshold = len(subPolicies) case cb.ImplicitMetaPolicy_MAJORITY: - imp.threshold = len(imp.subPolicies)/2 + 1 + threshold = len(subPolicies)/2 + 1 } // In the special case that there are no policies, consider 0 to be a majority or any - if len(imp.subPolicies) == 0 { - imp.threshold = 0 + if len(subPolicies) == 0 { + threshold = 0 } + + return &implicitMetaPolicy{ + subPolicies: subPolicies, + threshold: threshold, + }, nil } // Evaluate takes a set of SignedData and evaluates whether this set of signatures satisfies the policy diff --git a/common/policies/implicitmeta_test.go b/common/policies/implicitmeta_test.go index a6c89544772..104f23bad34 100644 --- a/common/policies/implicitmeta_test.go +++ b/common/policies/implicitmeta_test.go @@ -35,7 +35,7 @@ func (rp acceptPolicy) Evaluate(signedData []*cb.SignedData) error { } func TestImplicitMarshalError(t *testing.T) { - _, err := newImplicitMetaPolicy([]byte("GARBAGE")) + _, err := newImplicitMetaPolicy([]byte("GARBAGE"), nil) assert.Error(t, err, "Should have errored unmarshaling garbage") } @@ -50,9 +50,7 @@ func makeManagers(count, passing int) map[string]*ManagerImpl { remaining-- result[fmt.Sprintf("%d", i)] = &ManagerImpl{ - config: &policyConfig{ - policies: policyMap, - }, + policies: policyMap, } } return result @@ -63,15 +61,11 @@ func runPolicyTest(rule cb.ImplicitMetaPolicy_Rule, managerCount int, passingCou imp, err := newImplicitMetaPolicy(utils.MarshalOrPanic(&cb.ImplicitMetaPolicy{ Rule: rule, SubPolicy: TestPolicyName, - })) + }), makeManagers(managerCount, passingCount)) if err != nil { panic(err) } - imp.initialize(&policyConfig{ - managers: makeManagers(managerCount, passingCount), - }) - return imp.Evaluate(nil) } diff --git a/common/policies/policy.go b/common/policies/policy.go index deafc83a990..fd30661c0f0 100644 --- a/common/policies/policy.go +++ b/common/policies/policy.go @@ -9,13 +9,13 @@ package policies import ( "fmt" "strings" - "sync" + "github.com/hyperledger/fabric/common/flogging" cb "github.com/hyperledger/fabric/protos/common" "github.com/golang/protobuf/proto" - "github.com/hyperledger/fabric/common/flogging" logging "github.com/op/go-logging" + "github.com/pkg/errors" ) const ( @@ -67,25 +67,6 @@ type Manager interface { Manager(path []string) (Manager, bool) } -// Proposer is the interface used by the configtx manager for policy management -type Proposer interface { - // BeginPolicyProposals starts a policy update transaction - BeginPolicyProposals(tx interface{}, groups []string) ([]Proposer, error) - - // ProposePolicy createss a pending policy update from a ConfigPolicy and returns the deserialized - // value of the Policy representation - ProposePolicy(tx interface{}, name string, policy *cb.ConfigPolicy) (proto.Message, error) - - // RollbackProposals discards the pending policy updates - RollbackProposals(tx interface{}) - - // CommitProposals commits the pending policy updates - CommitProposals(tx interface{}) - - // PreCommit tests if a commit will apply - PreCommit(tx interface{}) error -} - // Provider provides the backing implementation of a policy type Provider interface { // NewPolicy creates a new policy based on the policy bytes @@ -100,43 +81,75 @@ type ChannelPolicyManagerGetter interface { Manager(channelID string) (Manager, bool) } -type policyConfig struct { - policies map[string]Policy - managers map[string]*ManagerImpl - imps []*implicitMetaPolicy -} - // ManagerImpl is an implementation of Manager and configtx.ConfigHandler // In general, it should only be referenced as an Impl for the configtx.ConfigManager type ManagerImpl struct { - parent *ManagerImpl - basePath string // The group level path - fqPrefix string // If this manager is treated as the root, the fully qualified prefix for policy names - fqPath string // The true absolute path, taking parents into consideration - providers map[int32]Provider - config *policyConfig - pendingConfig map[interface{}]*policyConfig - pendingLock sync.RWMutex + path string // The group level path + policies map[string]Policy + managers map[string]*ManagerImpl } // NewManagerImpl creates a new ManagerImpl with the given CryptoHelper -func NewManagerImpl(basePath string, providers map[int32]Provider) *ManagerImpl { +func NewManagerImpl(path string, providers map[int32]Provider, root *cb.ConfigGroup) (*ManagerImpl, error) { + var err error _, ok := providers[int32(cb.Policy_IMPLICIT_META)] if ok { logger.Panicf("ImplicitMetaPolicy type must be provider by the policy manager") } - return &ManagerImpl{ - basePath: basePath, - fqPrefix: PathSeparator + basePath + PathSeparator, - fqPath: PathSeparator + basePath, // Overridden after construction in the sub-manager case - providers: providers, - config: &policyConfig{ - policies: make(map[string]Policy), - managers: make(map[string]*ManagerImpl), - }, - pendingConfig: make(map[interface{}]*policyConfig), + managers := make(map[string]*ManagerImpl) + + for groupName, group := range root.Groups { + managers[groupName], err = NewManagerImpl(path+PathSeparator+groupName, providers, group) + if err != nil { + return nil, err + } + } + + policies := make(map[string]Policy) + for policyName, configPolicy := range root.Policies { + policy := configPolicy.Policy + if policy == nil { + return nil, fmt.Errorf("policy %s at path %s was nil", policyName, path) + } + + var cPolicy Policy + + if policy.Type == int32(cb.Policy_IMPLICIT_META) { + imp, err := newImplicitMetaPolicy(policy.Value, managers) + if err != nil { + return nil, errors.Wrapf(err, "implicit policy %s at path %s did not compile", policyName, path) + } + cPolicy = imp + } else { + provider, ok := providers[int32(policy.Type)] + if !ok { + return nil, fmt.Errorf("policy %s at path %s has unknown policy type: %v", policyName, path, policy.Type) + } + + var err error + cPolicy, _, err = provider.NewPolicy(policy.Value) + if err != nil { + return nil, errors.Wrapf(err, "policy %s at path %s did not compile", policyName, path) + } + } + + policies[policyName] = cPolicy + + logger.Debugf("Proposed new policy %s for %s", policyName, path) + } + + for groupName, manager := range managers { + for policyName, policy := range manager.policies { + policies[groupName+PathSeparator+policyName] = policy + } } + + return &ManagerImpl{ + path: path, + policies: policies, + managers: managers, + }, nil } type rejectPolicy string @@ -145,33 +158,17 @@ func (rp rejectPolicy) Evaluate(signedData []*cb.SignedData) error { return fmt.Errorf("No such policy type: %s", rp) } -// Basepath returns the basePath the manager was instantiated with -func (pm *ManagerImpl) BasePath() string { - return pm.basePath -} - -func (pm *ManagerImpl) PolicyNames() []string { - policyNames := make([]string, len(pm.config.policies)) - i := 0 - for policyName := range pm.config.policies { - policyNames[i] = policyName - i++ - } - return policyNames -} - // Manager returns the sub-policy manager for a given path and whether it exists func (pm *ManagerImpl) Manager(path []string) (Manager, bool) { - logger.Debugf("Looking for manager at path %v", path) - for name := range pm.config.managers { - logger.Debugf("I have sub-manager: %s", name) + logger.Debugf("Manager %s looking up path %v", pm.path, path) + for manager := range pm.managers { + logger.Debugf("Manager %s has managers %s", pm.path, manager) } - if len(path) == 0 { return pm, true } - m, ok := pm.config.managers[path[0]] + m, ok := pm.managers[path[0]] if !ok { return nil, false } @@ -188,148 +185,27 @@ func (pm *ManagerImpl) GetPolicy(id string) (Policy, bool) { var relpath string if strings.HasPrefix(id, PathSeparator) { - if pm.parent != nil { - return pm.parent.GetPolicy(id) - } - if !strings.HasPrefix(id, pm.fqPrefix) { + if !strings.HasPrefix(id, PathSeparator+pm.path) { if logger.IsEnabledFor(logging.DEBUG) { - logger.Debugf("Requested policy from root manager with wrong basePath: %s, returning rejectAll", id) + logger.Debugf("Requested absolute policy %s from %s, returning rejectAll", id, pm.path) } return rejectPolicy(id), false } - relpath = id[len(pm.fqPrefix):] + // strip off the leading slash, the path, and the trailing slash + relpath = id[1+len(pm.path)+1:] } else { relpath = id } - policy, ok := pm.config.policies[relpath] + policy, ok := pm.policies[relpath] if !ok { if logger.IsEnabledFor(logging.DEBUG) { - logger.Debugf("Returning dummy reject all policy because %s could not be found in /%s/%s", id, pm.basePath, relpath) + logger.Debugf("Returning dummy reject all policy because %s could not be found in %s/%s", id, pm.path, relpath) } return rejectPolicy(relpath), false } if logger.IsEnabledFor(logging.DEBUG) { - logger.Debugf("Returning policy %s from manager %s for evaluation", relpath, pm.fqPath) + logger.Debugf("Returning policy %s from manager %s for evaluation", relpath, pm.path) } return policy, true } - -// BeginPolicies is used to start a new config proposal -func (pm *ManagerImpl) BeginPolicyProposals(tx interface{}, groups []string) ([]Proposer, error) { - pm.pendingLock.Lock() - defer pm.pendingLock.Unlock() - pendingConfig, ok := pm.pendingConfig[tx] - if ok { - logger.Panicf("Serious Programming error: cannot call begin multiply for the same proposal") - } - - pendingConfig = &policyConfig{ - policies: make(map[string]Policy), - managers: make(map[string]*ManagerImpl), - } - pm.pendingConfig[tx] = pendingConfig - - managers := make([]Proposer, len(groups)) - for i, group := range groups { - newManager := NewManagerImpl(group, pm.providers) - newManager.parent = pm - mi := newManager - var fqPath []string - for mi != nil { - fqPath = append([]string{PathSeparator, mi.basePath}, fqPath...) - mi = mi.parent - } - newManager.fqPath = strings.Join(fqPath, "") - pendingConfig.managers[group] = newManager - managers[i] = newManager - } - return managers, nil -} - -// RollbackProposals is used to abandon a new config proposal -func (pm *ManagerImpl) RollbackProposals(tx interface{}) { - pm.pendingLock.Lock() - defer pm.pendingLock.Unlock() - delete(pm.pendingConfig, tx) -} - -// PreCommit is currently a no-op for the policy manager and always returns nil -func (pm *ManagerImpl) PreCommit(tx interface{}) error { - return nil -} - -// CommitProposals is used to commit a new config proposal -func (pm *ManagerImpl) CommitProposals(tx interface{}) { - pm.pendingLock.Lock() - defer pm.pendingLock.Unlock() - pendingConfig, ok := pm.pendingConfig[tx] - if !ok { - logger.Panicf("Programming error, cannot call begin in the middle of a proposal") - } - - if pendingConfig == nil { - logger.Panicf("Programming error, cannot call commit without an existing proposal") - } - - for managerPath, m := range pendingConfig.managers { - for _, policyName := range m.PolicyNames() { - fqKey := managerPath + PathSeparator + policyName - pendingConfig.policies[fqKey], _ = m.GetPolicy(policyName) - logger.Debugf("In commit adding relative sub-policy %s to %s", fqKey, pm.basePath) - } - } - - // Now that all the policies are present, initialize the meta policies - for _, imp := range pendingConfig.imps { - imp.initialize(pendingConfig) - } - - pm.config = pendingConfig - delete(pm.pendingConfig, tx) -} - -// ProposePolicy takes key, path, and ConfigPolicy and registers it in the proposed PolicyManager, or errors -// It also returns the deserialized policy value for tracking and inspection at the invocation side. -func (pm *ManagerImpl) ProposePolicy(tx interface{}, key string, configPolicy *cb.ConfigPolicy) (proto.Message, error) { - pm.pendingLock.RLock() - pendingConfig, ok := pm.pendingConfig[tx] - pm.pendingLock.RUnlock() - if !ok { - logger.Panicf("Serious Programming error: called Propose without Begin") - } - - policy := configPolicy.Policy - if policy == nil { - return nil, fmt.Errorf("Policy cannot be nil") - } - - var cPolicy Policy - var deserialized proto.Message - - if policy.Type == int32(cb.Policy_IMPLICIT_META) { - imp, err := newImplicitMetaPolicy(policy.Value) - if err != nil { - return nil, err - } - pendingConfig.imps = append(pendingConfig.imps, imp) - cPolicy = imp - deserialized = imp.conf - } else { - provider, ok := pm.providers[int32(policy.Type)] - if !ok { - return nil, fmt.Errorf("Unknown policy type: %v", policy.Type) - } - - var err error - cPolicy, deserialized, err = provider.NewPolicy(policy.Value) - if err != nil { - return nil, err - } - } - - pendingConfig.policies[key] = cPolicy - - logger.Debugf("Proposed new policy %s for %s", key, pm.basePath) - return deserialized, nil -} diff --git a/common/policies/policy_test.go b/common/policies/policy_test.go index 8494c46d9e8..770a3160586 100644 --- a/common/policies/policy_test.go +++ b/common/policies/policy_test.go @@ -7,7 +7,6 @@ SPDX-License-Identifier: Apache-2.0 package policies import ( - "fmt" "testing" cb "github.com/hyperledger/fabric/protos/common" @@ -36,19 +35,17 @@ func defaultProviders() map[int32]Provider { } func TestUnnestedManager(t *testing.T) { - m := NewManagerImpl("test", defaultProviders()) - handlers, err := m.BeginPolicyProposals(t, []string{}) - assert.NoError(t, err) - assert.Empty(t, handlers, "Should not have returned additional handlers") - - policyNames := []string{"1", "2", "3"} - - for _, policyName := range policyNames { - _, err := m.ProposePolicy(t, policyName, &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}) - assert.NoError(t, err) + config := &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + "1": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "2": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "3": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + }, } - m.CommitProposals(t) + m, err := NewManagerImpl("test", defaultProviders(), config) + assert.NoError(t, err) + assert.NotNil(t, m) _, ok := m.Manager([]string{"subGroup"}) assert.False(t, ok, "Should not have found a subgroup manager") @@ -57,56 +54,51 @@ func TestUnnestedManager(t *testing.T) { assert.True(t, ok, "Should have found the root manager") assert.Equal(t, m, r) - for _, policyName := range policyNames { + assert.Len(t, m.policies, len(config.Policies)) + + for policyName := range config.Policies { _, ok := m.GetPolicy(policyName) assert.True(t, ok, "Should have found policy %s", policyName) } } func TestNestedManager(t *testing.T) { - m := NewManagerImpl("test", defaultProviders()) - absPrefix := "/test/" - nesting1, err := m.BeginPolicyProposals(t, []string{"nest1"}) - assert.NoError(t, err) - assert.Len(t, nesting1, 1, "Should not have returned exactly one additional manager") - - nesting2, err := nesting1[0].BeginPolicyProposals(t, []string{"nest2a", "nest2b"}) - assert.NoError(t, err) - assert.Len(t, nesting2, 2, "Should not have returned two one additional managers") - - _, err = nesting2[0].BeginPolicyProposals(t, []string{}) - assert.NoError(t, err) - _, err = nesting2[1].BeginPolicyProposals(t, []string{}) - assert.NoError(t, err) - - policyNames := []string{"n0a", "n0b", "n0c"} - for _, policyName := range policyNames { - _, err := m.ProposePolicy(t, policyName, &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}) - assert.NoError(t, err) + config := &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + "n0a": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n0b": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n0c": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + }, + Groups: map[string]*cb.ConfigGroup{ + "nest1": &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + "n1a": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n1b": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n1c": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + }, + Groups: map[string]*cb.ConfigGroup{ + "nest2a": &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + "n2a_1": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n2a_2": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n2a_3": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + }, + }, + "nest2b": &cb.ConfigGroup{ + Policies: map[string]*cb.ConfigPolicy{ + "n2b_1": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n2b_2": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + "n2b_3": &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}, + }, + }, + }, + }, + }, } - n1PolicyNames := []string{"n1a", "n1b", "n1c"} - for _, policyName := range n1PolicyNames { - _, err := nesting1[0].ProposePolicy(t, policyName, &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}) - assert.NoError(t, err) - } - - n2aPolicyNames := []string{"n2a_1", "n2a_2", "n2a_3"} - for _, policyName := range n2aPolicyNames { - _, err := nesting2[0].ProposePolicy(t, policyName, &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}) - assert.NoError(t, err) - } - - n2bPolicyNames := []string{"n2b_1", "n2b_2", "n2b_3"} - for _, policyName := range n2bPolicyNames { - _, err := nesting2[1].ProposePolicy(t, policyName, &cb.ConfigPolicy{Policy: &cb.Policy{Type: mockType}}) - assert.NoError(t, err) - } - - nesting2[0].CommitProposals(t) - nesting2[1].CommitProposals(t) - nesting1[0].CommitProposals(t) - m.CommitProposals(t) + m, err := NewManagerImpl("nest0", defaultProviders(), config) + assert.NoError(t, err) + assert.NotNil(t, m) r, ok := m.Manager([]string{}) assert.True(t, ok, "Should have found the root manager") @@ -126,7 +118,8 @@ func TestNestedManager(t *testing.T) { assert.True(t, ok) assert.Equal(t, n2b, n2bs) - for _, policyName := range policyNames { + absPrefix := PathSeparator + "nest0" + PathSeparator + for policyName := range config.Policies { _, ok := m.GetPolicy(policyName) assert.True(t, ok, "Should have found policy %s", policyName) @@ -135,67 +128,56 @@ func TestNestedManager(t *testing.T) { assert.True(t, ok, "Should have found absolute policy %s", absName) } - for _, policyName := range n1PolicyNames { + for policyName := range config.Groups["nest1"].Policies { _, ok := n1.GetPolicy(policyName) assert.True(t, ok, "Should have found policy %s", policyName) - _, ok = m.GetPolicy(n1.(*ManagerImpl).BasePath() + "/" + policyName) + relPathFromBase := "nest1" + PathSeparator + policyName + _, ok = m.GetPolicy(relPathFromBase) assert.True(t, ok, "Should have found policy %s", policyName) for i, abs := range []Manager{n1, m} { - absName := absPrefix + n1.(*ManagerImpl).BasePath() + "/" + policyName + absName := absPrefix + relPathFromBase _, ok = abs.GetPolicy(absName) assert.True(t, ok, "Should have found absolutely policy for manager %d", i) } } - for _, policyName := range n2aPolicyNames { + for policyName := range config.Groups["nest1"].Groups["nest2a"].Policies { _, ok := n2a.GetPolicy(policyName) assert.True(t, ok, "Should have found policy %s", policyName) - _, ok = n1.GetPolicy(n2a.(*ManagerImpl).BasePath() + "/" + policyName) + relPathFromN1 := "nest2a" + PathSeparator + policyName + _, ok = n1.GetPolicy(relPathFromN1) assert.True(t, ok, "Should have found policy %s", policyName) - _, ok = m.GetPolicy(n1.(*ManagerImpl).BasePath() + "/" + n2a.(*ManagerImpl).BasePath() + "/" + policyName) + relPathFromBase := "nest1" + PathSeparator + relPathFromN1 + _, ok = m.GetPolicy(relPathFromBase) assert.True(t, ok, "Should have found policy %s", policyName) for i, abs := range []Manager{n2a, n1, m} { - absName := absPrefix + n1.(*ManagerImpl).BasePath() + "/" + n2a.(*ManagerImpl).BasePath() + "/" + policyName + absName := absPrefix + relPathFromBase _, ok = abs.GetPolicy(absName) assert.True(t, ok, "Should have found absolutely policy for manager %d", i) } } - for _, policyName := range n2bPolicyNames { + for policyName := range config.Groups["nest1"].Groups["nest2b"].Policies { _, ok := n2b.GetPolicy(policyName) assert.True(t, ok, "Should have found policy %s", policyName) - _, ok = n1.GetPolicy(n2b.(*ManagerImpl).BasePath() + "/" + policyName) + relPathFromN1 := "nest2b" + PathSeparator + policyName + _, ok = n1.GetPolicy(relPathFromN1) assert.True(t, ok, "Should have found policy %s", policyName) - _, ok = m.GetPolicy(n1.(*ManagerImpl).BasePath() + "/" + n2b.(*ManagerImpl).BasePath() + "/" + policyName) + relPathFromBase := "nest1" + PathSeparator + relPathFromN1 + _, ok = m.GetPolicy(relPathFromBase) assert.True(t, ok, "Should have found policy %s", policyName) for i, abs := range []Manager{n2b, n1, m} { - absName := absPrefix + n1.(*ManagerImpl).BasePath() + "/" + n2b.(*ManagerImpl).BasePath() + "/" + policyName + absName := absPrefix + relPathFromBase _, ok = abs.GetPolicy(absName) assert.True(t, ok, "Should have found absolutely policy for manager %d", i) } } } - -func TestFQPathName(t *testing.T) { - fqPath := []string{"base", "nest1", "nest2"} - m := NewManagerImpl(fqPath[0], defaultProviders()) - nesting1, err := m.BeginPolicyProposals(t, []string{fqPath[1]}) - assert.NoError(t, err) - assert.Len(t, nesting1, 1) - - nesting2, err := nesting1[0].BeginPolicyProposals(t, []string{fqPath[2]}) - assert.NoError(t, err) - assert.Len(t, nesting2, 1) - - assert.Equal(t, fmt.Sprintf("/%s/%s/%s", fqPath[0], fqPath[1], fqPath[2]), nesting2[0].(*ManagerImpl).fqPath) - assert.Equal(t, fmt.Sprintf("/%s/%s", fqPath[0], fqPath[1]), nesting1[0].(*ManagerImpl).fqPath) - assert.Equal(t, fmt.Sprintf("/%s", fqPath[0]), m.fqPath) -}