Updates to allow for Namespacing ACL resources in Consul Enterp… (#6675)

Main Changes:

• method signature updates everywhere to account for passing around enterprise meta.
• populate the EnterpriseAuthorizerContext for all ACL related authorizations.
• ACL resource listings now operate like the catalog or kv listings in that the returned entries are filtered down to what the token is allowed to see. With Namespaces its no longer all or nothing.
• Modified the acl.Policy parsing to abstract away basic decoding so that enterprise can do it slightly differently. Also updated method signatures so that when parsing a policy it can take extra ent metadata to use during rules validation and policy creation.

Secondary Changes:

• Moved protobuf encoding functions out of the agentpb package to eliminate circular dependencies.
• Added custom JSON unmarshalers for a few ACL resource types (to support snake case and to get rid of mapstructure)
• AuthMethod validator cache is now an interface as these will be cached per-namespace for Consul Enterprise.
• Added checks for policy/role link existence at the RPC API so we don’t push the request through raft to have it fail internally.
• Forward ACL token delete request to the primary datacenter when the secondary DC doesn’t have the token.
• Added a bunch of ACL test helpers for inserting ACL resource test data.
This commit is contained in:
Matt Keeler 2019-10-24 14:38:09 -04:00 committed by GitHub
parent 749915ce0f
commit e4ea9b0a96
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 1896 additions and 1104 deletions

View File

@ -288,11 +288,10 @@ func (pr *PolicyRules) Validate(conf *EnterpriseACLConfig) error {
return nil
}
func parseCurrent(rules string, conf *EnterpriseACLConfig) (*Policy, error) {
p := &Policy{}
if err := hcl.Decode(p, rules); err != nil {
return nil, fmt.Errorf("Failed to parse ACL rules: %v", err)
func parseCurrent(rules string, conf *EnterpriseACLConfig, meta *EnterprisePolicyMeta) (*Policy, error) {
p, err := decodeRules(rules, conf, meta)
if err != nil {
return nil, err
}
if err := p.PolicyRules.Validate(conf); err != nil {
@ -300,7 +299,7 @@ func parseCurrent(rules string, conf *EnterpriseACLConfig) (*Policy, error) {
}
if err := p.EnterprisePolicyRules.Validate(conf); err != nil {
return nil, fmt.Errorf("Invalidate enterprise rules: %v", err)
return nil, err
}
return p, nil
@ -423,7 +422,7 @@ func parseLegacy(rules string, conf *EnterpriseACLConfig) (*Policy, error) {
// NewPolicyFromSource is used to parse the specified ACL rules into an
// intermediary set of policies, before being compiled into
// the ACL
func NewPolicyFromSource(id string, revision uint64, rules string, syntax SyntaxVersion, conf *EnterpriseACLConfig) (*Policy, error) {
func NewPolicyFromSource(id string, revision uint64, rules string, syntax SyntaxVersion, conf *EnterpriseACLConfig, meta *EnterprisePolicyMeta) (*Policy, error) {
if rules == "" {
// Hot path for empty source
return &Policy{ID: id, Revision: revision}, nil
@ -435,7 +434,7 @@ func NewPolicyFromSource(id string, revision uint64, rules string, syntax Syntax
case SyntaxLegacy:
policy, err = parseLegacy(rules, conf)
case SyntaxCurrent:
policy, err = parseCurrent(rules, conf)
policy, err = parseCurrent(rules, conf, meta)
default:
return nil, fmt.Errorf("Invalid rules version: %d", syntax)
}

View File

@ -2,6 +2,15 @@
package acl
import (
"fmt"
"github.com/hashicorp/hcl"
)
// EnterprisePolicyMeta stub
type EnterprisePolicyMeta struct{}
// EnterpriseRule stub
type EnterpriseRule struct{}
@ -17,3 +26,13 @@ func (r *EnterprisePolicyRules) Validate(*EnterpriseACLConfig) error {
// nothing to validate
return nil
}
func decodeRules(rules string, _ *EnterpriseACLConfig, _ *EnterprisePolicyMeta) (*Policy, error) {
p := &Policy{}
if err := hcl.Decode(p, rules); err != nil {
return nil, fmt.Errorf("Failed to parse ACL rules: %v", err)
}
return p, nil
}

View File

@ -544,7 +544,7 @@ func TestPolicySourceParse(t *testing.T) {
for _, tc := range cases {
t.Run(tc.Name, func(t *testing.T) {
req := require.New(t)
actual, err := NewPolicyFromSource("", 0, tc.Rules, tc.Syntax, nil)
actual, err := NewPolicyFromSource("", 0, tc.Rules, tc.Syntax, nil, nil)
if tc.Err != "" {
errStartsWith(t, err, tc.Err)
} else {

View File

@ -187,6 +187,7 @@ func (s *HTTPServer) ACLPolicyList(resp http.ResponseWriter, req *http.Request)
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
@ -244,6 +245,8 @@ func (s *HTTPServer) ACLPolicyRead(resp http.ResponseWriter, req *http.Request,
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
@ -326,6 +329,7 @@ func (s *HTTPServer) aclPolicyWriteInternal(resp http.ResponseWriter, req *http.
Datacenter: s.agent.config.Datacenter,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.Policy.EnterpriseMeta)
if err := decodeBody(req, &args.Policy, fixTimeAndHashFields); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("Policy decoding failed: %v", err)}
@ -359,6 +363,7 @@ func (s *HTTPServer) ACLPolicyDelete(resp http.ResponseWriter, req *http.Request
PolicyID: policyID,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.EnterpriseMeta)
var ignored string
if err := s.agent.RPC("ACL.PolicyDelete", args, &ignored); err != nil {
@ -381,6 +386,8 @@ func (s *HTTPServer) ACLTokenList(resp http.ResponseWriter, req *http.Request) (
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
@ -483,6 +490,8 @@ func (s *HTTPServer) ACLTokenGet(resp http.ResponseWriter, req *http.Request, to
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
@ -510,6 +519,7 @@ func (s *HTTPServer) aclTokenSetInternal(resp http.ResponseWriter, req *http.Req
Create: create,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.ACLToken.EnterpriseMeta)
if err := decodeBody(req, &args.ACLToken, fixTimeAndHashFields); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("Token decoding failed: %v", err)}
@ -537,6 +547,7 @@ func (s *HTTPServer) ACLTokenDelete(resp http.ResponseWriter, req *http.Request,
TokenID: tokenID,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.EnterpriseMeta)
var ignored string
if err := s.agent.RPC("ACL.TokenDelete", args, &ignored); err != nil {
@ -555,6 +566,8 @@ func (s *HTTPServer) ACLTokenClone(resp http.ResponseWriter, req *http.Request,
Create: true,
}
s.parseEntMeta(req, &args.ACLToken.EnterpriseMeta)
if err := decodeBody(req, &args.ACLToken, fixTimeAndHashFields); err != nil && err.Error() != "EOF" {
return nil, BadRequestError{Reason: fmt.Sprintf("Token decoding failed: %v", err)}
}
@ -580,6 +593,7 @@ func (s *HTTPServer) ACLRoleList(resp http.ResponseWriter, req *http.Request) (i
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
@ -656,6 +670,7 @@ func (s *HTTPServer) ACLRoleRead(resp http.ResponseWriter, req *http.Request, ro
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
@ -688,6 +703,7 @@ func (s *HTTPServer) ACLRoleWrite(resp http.ResponseWriter, req *http.Request, r
Datacenter: s.agent.config.Datacenter,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.Role.EnterpriseMeta)
if err := decodeBody(req, &args.Role, fixTimeAndHashFields); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("Role decoding failed: %v", err)}
@ -713,6 +729,7 @@ func (s *HTTPServer) ACLRoleDelete(resp http.ResponseWriter, req *http.Request,
RoleID: roleID,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.EnterpriseMeta)
var ignored string
if err := s.agent.RPC("ACL.RoleDelete", args, &ignored); err != nil {
@ -732,6 +749,8 @@ func (s *HTTPServer) ACLBindingRuleList(resp http.ResponseWriter, req *http.Requ
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
@ -790,6 +809,8 @@ func (s *HTTPServer) ACLBindingRuleRead(resp http.ResponseWriter, req *http.Requ
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
@ -821,6 +842,7 @@ func (s *HTTPServer) ACLBindingRuleWrite(resp http.ResponseWriter, req *http.Req
Datacenter: s.agent.config.Datacenter,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.BindingRule.EnterpriseMeta)
if err := decodeBody(req, &args.BindingRule, fixTimeAndHashFields); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("BindingRule decoding failed: %v", err)}
@ -846,6 +868,7 @@ func (s *HTTPServer) ACLBindingRuleDelete(resp http.ResponseWriter, req *http.Re
BindingRuleID: bindingRuleID,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.EnterpriseMeta)
var ignored bool
if err := s.agent.RPC("ACL.BindingRuleDelete", args, &ignored); err != nil {
@ -864,6 +887,7 @@ func (s *HTTPServer) ACLAuthMethodList(resp http.ResponseWriter, req *http.Reque
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
@ -920,6 +944,7 @@ func (s *HTTPServer) ACLAuthMethodRead(resp http.ResponseWriter, req *http.Reque
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
s.parseEntMeta(req, &args.EnterpriseMeta)
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
@ -953,6 +978,7 @@ func (s *HTTPServer) ACLAuthMethodWrite(resp http.ResponseWriter, req *http.Requ
Datacenter: s.agent.config.Datacenter,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.AuthMethod.EnterpriseMeta)
if err := decodeBody(req, &args.AuthMethod, fixTimeAndHashFields); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("AuthMethod decoding failed: %v", err)}
@ -981,6 +1007,7 @@ func (s *HTTPServer) ACLAuthMethodDelete(resp http.ResponseWriter, req *http.Req
AuthMethodName: methodName,
}
s.parseToken(req, &args.Token)
s.parseEntMeta(req, &args.EnterpriseMeta)
var ignored bool
if err := s.agent.RPC("ACL.AuthMethodDelete", args, &ignored); err != nil {
@ -997,8 +1024,10 @@ func (s *HTTPServer) ACLLogin(resp http.ResponseWriter, req *http.Request) (inte
args := &structs.ACLLoginRequest{
Datacenter: s.agent.config.Datacenter,
Auth: &structs.ACLLoginParams{},
}
s.parseDC(req, &args.Datacenter)
s.parseEntMeta(req, &args.Auth.EnterpriseMeta)
if err := decodeBody(req, &args.Auth, nil); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("Failed to decode request body:: %v", err)}

View File

@ -1,35 +0,0 @@
package agentpb
import (
"fmt"
"github.com/hashicorp/consul/agent/structs"
)
type ProtoMarshaller interface {
Size() int
MarshalTo([]byte) (int, error)
Unmarshal([]byte) error
ProtoMessage()
}
func EncodeInterface(t structs.MessageType, message interface{}) ([]byte, error) {
if marshaller, ok := message.(ProtoMarshaller); ok {
return Encode(t, marshaller)
}
return nil, fmt.Errorf("message does not implement the ProtoMarshaller interface: %T", message)
}
func Encode(t structs.MessageType, message ProtoMarshaller) ([]byte, error) {
data := make([]byte, message.Size()+1)
data[0] = uint8(t)
if _, err := message.MarshalTo(data[1:]); err != nil {
return nil, err
}
return data, nil
}
func Decode(buf []byte, out ProtoMarshaller) error {
// Note that this assumes the leading byte indicating the type has already been stripped off
return out.Unmarshal(buf)
}

View File

@ -608,7 +608,7 @@ func (r *ACLResolver) resolvePoliciesForIdentity(identity structs.ACLIdentity) (
serviceIdentities = dedupeServiceIdentities(serviceIdentities)
// Generate synthetic policies for all service identities in effect.
syntheticPolicies := r.synthesizePoliciesForServiceIdentities(serviceIdentities)
syntheticPolicies := r.synthesizePoliciesForServiceIdentities(serviceIdentities, identity.EnterpriseMetadata())
// For the new ACLs policy replication is mandatory for correct operation on servers. Therefore
// we only attempt to resolve policies locally
@ -622,14 +622,14 @@ func (r *ACLResolver) resolvePoliciesForIdentity(identity structs.ACLIdentity) (
return filtered, nil
}
func (r *ACLResolver) synthesizePoliciesForServiceIdentities(serviceIdentities []*structs.ACLServiceIdentity) []*structs.ACLPolicy {
func (r *ACLResolver) synthesizePoliciesForServiceIdentities(serviceIdentities []*structs.ACLServiceIdentity, entMeta *structs.EnterpriseMeta) []*structs.ACLPolicy {
if len(serviceIdentities) == 0 {
return nil
}
syntheticPolicies := make([]*structs.ACLPolicy, 0, len(serviceIdentities))
for _, s := range serviceIdentities {
syntheticPolicies = append(syntheticPolicies, s.SyntheticPolicy())
syntheticPolicies = append(syntheticPolicies, s.SyntheticPolicy(entMeta))
}
return syntheticPolicies
@ -1355,26 +1355,166 @@ func (f *aclFilter) filterPreparedQueries(queries *structs.PreparedQueries) {
*queries = ret
}
func (f *aclFilter) redactTokenSecret(token **structs.ACLToken) {
// TODO (namespaces) update to call with an actual ent authz context once acls support it
if token == nil || *token == nil || f == nil || f.authorizer.ACLWrite(nil) == acl.Allow {
func (f *aclFilter) filterToken(token **structs.ACLToken) {
var entCtx acl.EnterpriseAuthorizerContext
if token == nil || *token == nil || f == nil {
return
}
clone := *(*token)
clone.SecretID = redactedToken
*token = &clone
(*token).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
// no permissions to read
*token = nil
} else if f.authorizer.ACLWrite(&entCtx) != acl.Allow {
// no write permissions - redact secret
clone := *(*token)
clone.SecretID = redactedToken
*token = &clone
}
}
func (f *aclFilter) redactTokenSecrets(tokens *structs.ACLTokens) {
func (f *aclFilter) filterTokens(tokens *structs.ACLTokens) {
ret := make(structs.ACLTokens, 0, len(*tokens))
for _, token := range *tokens {
final := token
f.redactTokenSecret(&final)
ret = append(ret, final)
f.filterToken(&final)
if final != nil {
ret = append(ret, final)
}
}
*tokens = ret
}
func (f *aclFilter) filterTokenStub(token **structs.ACLTokenListStub) {
var entCtx acl.EnterpriseAuthorizerContext
if token == nil || *token == nil || f == nil {
return
}
(*token).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
*token = nil
}
}
func (f *aclFilter) filterTokenStubs(tokens *[]*structs.ACLTokenListStub) {
ret := make(structs.ACLTokenListStubs, 0, len(*tokens))
for _, token := range *tokens {
final := token
f.filterTokenStub(&final)
if final != nil {
ret = append(ret, final)
}
}
*tokens = ret
}
func (f *aclFilter) filterPolicy(policy **structs.ACLPolicy) {
var entCtx acl.EnterpriseAuthorizerContext
if policy == nil || *policy == nil || f == nil {
return
}
(*policy).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
// no permissions to read
*policy = nil
}
}
func (f *aclFilter) filterPolicies(policies *structs.ACLPolicies) {
ret := make(structs.ACLPolicies, 0, len(*policies))
for _, policy := range *policies {
final := policy
f.filterPolicy(&final)
if final != nil {
ret = append(ret, final)
}
}
*policies = ret
}
func (f *aclFilter) filterRole(role **structs.ACLRole) {
var entCtx acl.EnterpriseAuthorizerContext
if role == nil || *role == nil || f == nil {
return
}
(*role).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
// no permissions to read
*role = nil
}
}
func (f *aclFilter) filterRoles(roles *structs.ACLRoles) {
ret := make(structs.ACLRoles, 0, len(*roles))
for _, role := range *roles {
final := role
f.filterRole(&final)
if final != nil {
ret = append(ret, final)
}
}
*roles = ret
}
func (f *aclFilter) filterBindingRule(rule **structs.ACLBindingRule) {
var entCtx acl.EnterpriseAuthorizerContext
if rule == nil || *rule == nil || f == nil {
return
}
(*rule).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
// no permissions to read
*rule = nil
}
}
func (f *aclFilter) filterBindingRules(rules *structs.ACLBindingRules) {
ret := make(structs.ACLBindingRules, 0, len(*rules))
for _, rule := range *rules {
final := rule
f.filterBindingRule(&final)
if final != nil {
ret = append(ret, final)
}
}
*rules = ret
}
func (f *aclFilter) filterAuthMethod(method **structs.ACLAuthMethod) {
var entCtx acl.EnterpriseAuthorizerContext
if method == nil || *method == nil || f == nil {
return
}
(*method).FillAuthzContext(&entCtx)
if f.authorizer.ACLRead(&entCtx) != acl.Allow {
// no permissions to read
*method = nil
}
}
func (f *aclFilter) filterAuthMethods(methods *structs.ACLAuthMethods) {
ret := make(structs.ACLAuthMethods, 0, len(*methods))
for _, method := range *methods {
final := method
f.filterAuthMethod(&final)
if final != nil {
ret = append(ret, final)
}
}
*methods = ret
}
func (r *ACLResolver) filterACLWithAuthorizer(authorizer acl.Authorizer, subj interface{}) error {
if authorizer == nil {
return nil
@ -1423,13 +1563,36 @@ func (r *ACLResolver) filterACLWithAuthorizer(authorizer acl.Authorizer, subj in
filt.redactPreparedQueryTokens(v)
case *structs.ACLTokens:
filt.redactTokenSecrets(v)
filt.filterTokens(v)
case **structs.ACLToken:
filt.redactTokenSecret(v)
filt.filterToken(v)
case *[]*structs.ACLTokenListStub:
filt.filterTokenStubs(v)
case **structs.ACLTokenListStub:
filt.filterTokenStub(v)
case *structs.ACLPolicies:
filt.filterPolicies(v)
case **structs.ACLPolicy:
filt.filterPolicy(v)
case *structs.ACLRoles:
filt.filterRoles(v)
case **structs.ACLRole:
filt.filterRole(v)
case *structs.ACLBindingRules:
filt.filterBindingRules(v)
case **structs.ACLBindingRule:
filt.filterBindingRule(v)
case *structs.ACLAuthMethods:
filt.filterAuthMethods(v)
case **structs.ACLAuthMethod:
filt.filterAuthMethod(v)
default:
panic(fmt.Errorf("Unhandled type passed to ACL filter: %#v", subj))
panic(fmt.Errorf("Unhandled type passed to ACL filter: %T %#v", subj, subj))
}
return nil

View File

@ -21,7 +21,7 @@ type authMethodValidatorEntry struct {
// then the cached version is returned, otherwise a new validator is created
// and cached.
func (s *Server) loadAuthMethodValidator(idx uint64, method *structs.ACLAuthMethod) (authmethod.Validator, error) {
if prevIdx, v, ok := s.getCachedAuthMethodValidator(method.Name); ok && idx <= prevIdx {
if prevIdx, v, ok := s.aclAuthMethodValidators.GetValidator(method); ok && idx <= prevIdx {
return v, nil
}
@ -30,61 +30,11 @@ func (s *Server) loadAuthMethodValidator(idx uint64, method *structs.ACLAuthMeth
return nil, fmt.Errorf("auth method validator for %q could not be initialized: %v", method.Name, err)
}
v = s.getOrReplaceAuthMethodValidator(method.Name, idx, v)
v = s.aclAuthMethodValidators.PutValidatorIfNewer(method, v, idx)
return v, nil
}
// getCachedAuthMethodValidator returns an AuthMethodValidator for
// the given name exclusively from the cache. If one is not found in the cache
// nil is returned.
func (s *Server) getCachedAuthMethodValidator(name string) (uint64, authmethod.Validator, bool) {
s.aclAuthMethodValidatorLock.RLock()
defer s.aclAuthMethodValidatorLock.RUnlock()
if s.aclAuthMethodValidators != nil {
v, ok := s.aclAuthMethodValidators[name]
if ok {
return v.ModifyIndex, v.Validator, true
}
}
return 0, nil, false
}
// getOrReplaceAuthMethodValidator updates the cached validator with the
// provided one UNLESS it has been updated by another goroutine in which case
// the updated one is returned.
func (s *Server) getOrReplaceAuthMethodValidator(name string, idx uint64, v authmethod.Validator) authmethod.Validator {
s.aclAuthMethodValidatorLock.Lock()
defer s.aclAuthMethodValidatorLock.Unlock()
if s.aclAuthMethodValidators == nil {
s.aclAuthMethodValidators = make(map[string]*authMethodValidatorEntry)
}
prev, ok := s.aclAuthMethodValidators[name]
if ok {
if prev.ModifyIndex >= idx {
return prev.Validator
}
}
s.logger.Printf("[DEBUG] acl: updating cached auth method validator for %q", name)
s.aclAuthMethodValidators[name] = &authMethodValidatorEntry{
Validator: v,
ModifyIndex: idx,
}
return v
}
// purgeAuthMethodValidators resets the cache of validators.
func (s *Server) purgeAuthMethodValidators() {
s.aclAuthMethodValidatorLock.Lock()
s.aclAuthMethodValidators = make(map[string]*authMethodValidatorEntry)
s.aclAuthMethodValidatorLock.Unlock()
}
// evaluateRoleBindings evaluates all current binding rules associated with the
// given auth method against the verified data returned from the authentication
// process.
@ -93,9 +43,10 @@ func (s *Server) purgeAuthMethodValidators() {
func (s *Server) evaluateRoleBindings(
validator authmethod.Validator,
verifiedFields map[string]string,
entMeta *structs.EnterpriseMeta,
) ([]*structs.ACLServiceIdentity, []structs.ACLTokenRoleLink, error) {
// Only fetch rules that are relevant for this method.
_, rules, err := s.fsm.State().ACLBindingRuleList(nil, validator.Name())
_, rules, err := s.fsm.State().ACLBindingRuleList(nil, validator.Name(), entMeta)
if err != nil {
return nil, nil, err
} else if len(rules) == 0 {
@ -136,7 +87,7 @@ func (s *Server) evaluateRoleBindings(
})
case structs.BindingRuleBindTypeRole:
_, role, err := s.fsm.State().ACLRoleGetByName(nil, bindName)
_, role, err := s.fsm.State().ACLRoleGetByName(nil, bindName, &rule.EnterpriseMeta)
if err != nil {
return nil, nil, err
}

View File

@ -166,6 +166,8 @@ func (a *ACL) BootstrapTokens(args *structs.DCSpecificRequest, reply *structs.AC
ResetIndex: specifiedIndex,
}
req.Token.EnterpriseMeta.InitDefault()
req.Token.SetHash(true)
resp, err := a.srv.raftApply(structs.ACLBootstrapRequestType, &req)
@ -177,7 +179,7 @@ func (a *ACL) BootstrapTokens(args *structs.DCSpecificRequest, reply *structs.AC
return err
}
if _, token, err := state.ACLTokenGetByAccessor(nil, accessor); err == nil {
if _, token, err := state.ACLTokenGetByAccessor(nil, accessor, structs.DefaultEnterpriseMeta()); err == nil {
*reply = *token
}
@ -201,15 +203,18 @@ func (a *ACL) TokenRead(args *structs.ACLTokenGetRequest, reply *structs.ACLToke
}
var rule acl.Authorizer
if args.TokenIDType == structs.ACLTokenAccessor {
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
var err error
// Only ACLRead privileges are required to list tokens
// However if you do not have ACLWrite as well the token
// secrets will be redacted
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err = a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil || rule.ACLRead(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
}
@ -221,16 +226,18 @@ func (a *ACL) TokenRead(args *structs.ACLTokenGetRequest, reply *structs.ACLToke
var err error
if args.TokenIDType == structs.ACLTokenAccessor {
index, token, err = state.ACLTokenGetByAccessor(ws, args.TokenID)
index, token, err = state.ACLTokenGetByAccessor(ws, args.TokenID, &args.EnterpriseMeta)
if token != nil {
a.srv.filterACLWithAuthorizer(rule, &token)
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
if rule.ACLWrite(nil) != acl.Allow {
// token secret was redacted
if token.SecretID == redactedToken {
reply.Redacted = true
}
}
} else {
index, token, err = state.ACLTokenGetBySecret(ws, args.TokenID)
index, token, err = state.ACLTokenGetBySecret(ws, args.TokenID, nil)
// no extra validation is needed here. If you have the secret ID you can read it.
}
if token != nil && token.IsExpired(time.Now()) {
@ -263,14 +270,16 @@ func (a *ACL) TokenClone(args *structs.ACLTokenSetRequest, reply *structs.ACLTok
defer metrics.MeasureSince([]string{"acl", "token", "clone"}, time.Now())
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.ACLToken.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
_, token, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, args.ACLToken.AccessorID)
_, token, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, args.ACLToken.AccessorID, &args.ACLToken.EnterpriseMeta)
if err != nil {
return err
} else if token == nil || token.IsExpired(time.Now()) {
@ -297,6 +306,7 @@ func (a *ACL) TokenClone(args *structs.ACLTokenSetRequest, reply *structs.ACLTok
Local: token.Local,
Description: token.Description,
ExpirationTime: token.ExpirationTime,
EnterpriseMeta: args.ACLToken.EnterpriseMeta,
},
WriteRequest: args.WriteRequest,
}
@ -327,10 +337,11 @@ func (a *ACL) TokenSet(args *structs.ACLTokenSetRequest, reply *structs.ACLToken
defer metrics.MeasureSince([]string{"acl", "token", "upsert"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.ACLToken.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -354,13 +365,13 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
var err error
if token.AccessorID != "" {
_, accessorMatch, err = state.ACLTokenGetByAccessor(nil, token.AccessorID)
_, accessorMatch, err = state.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
if err != nil {
return fmt.Errorf("Failed acl token lookup by accessor: %v", err)
}
}
if token.SecretID != "" {
_, secretMatch, err = state.ACLTokenGetBySecret(nil, token.SecretID)
_, secretMatch, err = state.ACLTokenGetBySecret(nil, token.SecretID, nil)
if err != nil {
return fmt.Errorf("Failed acl token lookup by secret: %v", err)
}
@ -379,7 +390,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
return fmt.Errorf("Invalid Token: AccessorID is not a valid UUID")
} else if accessorMatch != nil {
return fmt.Errorf("Invalid Token: AccessorID is already in use")
} else if _, match, err := state.ACLTokenGetBySecret(nil, token.AccessorID); err != nil || match != nil {
} else if _, match, err := state.ACLTokenGetBySecret(nil, token.AccessorID, nil); err != nil || match != nil {
if err != nil {
return fmt.Errorf("Failed to lookup the acl token: %v", err)
}
@ -388,7 +399,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
return fmt.Errorf("Invalid Token: UUIDs with the prefix %q are reserved", structs.ACLReservedPrefix)
}
// Generate the AccessorID if not specified
// Generate the SecretID if not specified
if token.SecretID == "" {
token.SecretID, err = lib.GenerateUUID(a.srv.checkTokenUUID)
if err != nil {
@ -398,7 +409,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
return fmt.Errorf("Invalid Token: SecretID is not a valid UUID")
} else if secretMatch != nil {
return fmt.Errorf("Invalid Token: SecretID is already in use")
} else if _, match, err := state.ACLTokenGetByAccessor(nil, token.SecretID); err != nil || match != nil {
} else if _, match, err := state.ACLTokenGetByAccessor(nil, token.SecretID, nil); err != nil || match != nil {
if err != nil {
return fmt.Errorf("Failed to lookup the acl token: %v", err)
}
@ -509,7 +520,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
// Validate all the policy names and convert them to policy IDs
for _, link := range token.Policies {
if link.ID == "" {
_, policy, err := state.ACLPolicyGetByName(nil, link.Name)
_, policy, err := state.ACLPolicyGetByName(nil, link.Name, &token.EnterpriseMeta)
if err != nil {
return fmt.Errorf("Error looking up policy for name %q: %v", link.Name, err)
}
@ -517,6 +528,15 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
return fmt.Errorf("No such ACL policy with name %q", link.Name)
}
link.ID = policy.ID
} else {
_, policy, err := state.ACLPolicyGetByID(nil, link.ID, &token.EnterpriseMeta)
if err != nil {
return fmt.Errorf("Error looking up policy for id %q: %v", link.ID, err)
}
if policy == nil {
return fmt.Errorf("No such ACL policy with ID %q", link.ID)
}
}
// Do not store the policy name within raft/memdb as the policy could be renamed in the future.
@ -536,7 +556,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
// Validate all the role names and convert them to role IDs.
for _, link := range token.Roles {
if link.ID == "" {
_, role, err := state.ACLRoleGetByName(nil, link.Name)
_, role, err := state.ACLRoleGetByName(nil, link.Name, &token.EnterpriseMeta)
if err != nil {
return fmt.Errorf("Error looking up role for name %q: %v", link.Name, err)
}
@ -544,6 +564,15 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
return fmt.Errorf("No such ACL role with name %q", link.Name)
}
link.ID = role.ID
} else {
_, role, err := state.ACLRoleGetByID(nil, link.ID, &token.EnterpriseMeta)
if err != nil {
return fmt.Errorf("Error looking up role for id %q: %v", link.ID, err)
}
if role == nil {
return fmt.Errorf("No such ACL role with ID %q", link.ID)
}
}
// Do not store the role name within raft/memdb as the role could be renamed in the future.
@ -580,6 +609,12 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
token.SetHash(true)
// validate the enterprise meta
err = state.ACLTokenUpsertValidateEnterprise(token, accessorMatch)
if err != nil {
return err
}
req := &structs.ACLTokenBatchSetRequest{
Tokens: structs.ACLTokens{token},
CAS: false,
@ -606,7 +641,7 @@ func (a *ACL) tokenSetInternal(args *structs.ACLTokenSetRequest, reply *structs.
}
// Don't check expiration times here as it doesn't really matter.
if _, updatedToken, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, token.AccessorID); err == nil && updatedToken != nil {
if _, updatedToken, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, token.AccessorID, nil); err == nil && updatedToken != nil {
*reply = *updatedToken
} else {
return fmt.Errorf("Failed to retrieve the token after insertion")
@ -687,10 +722,12 @@ func (a *ACL) TokenDelete(args *structs.ACLTokenDeleteRequest, reply *string) er
defer metrics.MeasureSince([]string{"acl", "token", "delete"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -703,7 +740,7 @@ func (a *ACL) TokenDelete(args *structs.ACLTokenDeleteRequest, reply *string) er
}
// grab the token here so we can invalidate our cache later on
_, token, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, args.TokenID)
_, token, err := a.srv.fsm.State().ACLTokenGetByAccessor(nil, args.TokenID, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -715,10 +752,18 @@ func (a *ACL) TokenDelete(args *structs.ACLTokenDeleteRequest, reply *string) er
// No need to check expiration time because it's being deleted.
// token found in secondary DC but its not local so it must be deleted in the primary
if !a.srv.InACLDatacenter() && !token.Local {
args.Datacenter = a.srv.config.ACLDatacenter
return a.srv.forwardDC("ACL.TokenDelete", a.srv.config.ACLDatacenter, args, reply)
}
} else if !a.srv.InACLDatacenter() {
// token not found in secondary DC - attempt to delete within the primary
args.Datacenter = a.srv.config.ACLDatacenter
return a.srv.forwardDC("ACL.TokenDelete", a.srv.config.ACLDatacenter, args, reply)
} else {
// in Primary Datacenter but the token does not exist - return early as there is nothing to do.
return nil
}
req := &structs.ACLTokenBatchDeleteRequest{
@ -731,15 +776,13 @@ func (a *ACL) TokenDelete(args *structs.ACLTokenDeleteRequest, reply *string) er
}
// Purge the identity from the cache to prevent using the previous definition of the identity
if token != nil {
a.srv.acls.cache.RemoveIdentity(token.SecretID)
}
a.srv.acls.cache.RemoveIdentity(token.SecretID)
if respErr, ok := resp.(error); ok {
return respErr
}
if reply != nil && token != nil {
if reply != nil {
*reply = token.AccessorID
}
@ -765,16 +808,15 @@ func (a *ACL) TokenList(args *structs.ACLTokenListRequest, reply *structs.ACLTok
}
rule, err := a.srv.ResolveToken(args.Token)
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, tokens, err := state.ACLTokenList(ws, args.IncludeLocal, args.IncludeGlobal, args.Policy, args.Role, args.AuthMethod)
index, tokens, err := state.ACLTokenList(ws, args.IncludeLocal, args.IncludeGlobal, args.Policy, args.Role, args.AuthMethod, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -788,6 +830,12 @@ func (a *ACL) TokenList(args *structs.ACLTokenListRequest, reply *structs.ACLTok
}
stubs = append(stubs, token.Stub())
}
// filter down to just the tokens that the requester has permissions to read
if err := a.srv.filterACLWithAuthorizer(rule, &stubs); err != nil {
return err
}
reply.Index, reply.Tokens = index, stubs
return nil
})
@ -807,10 +855,9 @@ func (a *ACL) TokenBatchRead(args *structs.ACLTokenBatchGetRequest, reply *struc
}
rule, err := a.srv.ResolveToken(args.Token)
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
@ -823,11 +870,27 @@ func (a *ACL) TokenBatchRead(args *structs.ACLTokenBatchGetRequest, reply *struc
// This RPC is used for replication, so don't filter out expired tokens here.
a.srv.filterACLWithAuthorizer(rule, &tokens)
// Filter the tokens down to just what we have permission to see - also redact
// secrets based on allowed permissions. We could just call filterACLWithAuthorizer
// on the whole token list but then it would require another pass through the token
// list to determine if any secrets were redacted. Its a small amount of code to
// process the loop so it was duplicated here and we instead call the filter func
// with just a single token.
ret := make(structs.ACLTokens, 0, len(tokens))
for _, token := range tokens {
final := token
a.srv.filterACLWithAuthorizer(rule, &final)
if final != nil {
ret = append(ret, final)
if final.SecretID == redactedToken {
reply.Redacted = true
}
} else {
reply.Removed = true
}
}
reply.Index, reply.Tokens = index, tokens
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
reply.Redacted = rule.ACLWrite(nil) != acl.Allow
reply.Index, reply.Tokens = index, ret
return nil
})
}
@ -841,15 +904,18 @@ func (a *ACL) PolicyRead(args *structs.ACLPolicyGetRequest, reply *structs.ACLPo
return err
}
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil || rule.ACLRead(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, policy, err := state.ACLPolicyGetByID(ws, args.PolicyID)
index, policy, err := state.ACLPolicyGetByID(ws, args.PolicyID, &args.EnterpriseMeta)
if err != nil {
return err
@ -869,9 +935,10 @@ func (a *ACL) PolicyBatchRead(args *structs.ACLPolicyBatchGetRequest, reply *str
return err
}
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
@ -882,6 +949,8 @@ func (a *ACL) PolicyBatchRead(args *structs.ACLPolicyBatchGetRequest, reply *str
return err
}
a.srv.filterACLWithAuthorizer(rule, &policies)
reply.Index, reply.Policies = index, policies
return nil
})
@ -903,10 +972,12 @@ func (a *ACL) PolicySet(args *structs.ACLPolicySetRequest, reply *structs.ACLPol
defer metrics.MeasureSince([]string{"acl", "policy", "upsert"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.Policy.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -935,12 +1006,12 @@ func (a *ACL) PolicySet(args *structs.ACLPolicySetRequest, reply *structs.ACLPol
return fmt.Errorf("Policy ID invalid UUID")
}
_, idMatch, err = state.ACLPolicyGetByID(nil, policy.ID)
_, idMatch, err = state.ACLPolicyGetByID(nil, policy.ID, nil)
if err != nil {
return fmt.Errorf("acl policy lookup by id failed: %v", err)
}
}
_, nameMatch, err = state.ACLPolicyGetByName(nil, policy.Name)
_, nameMatch, err = state.ACLPolicyGetByName(nil, policy.Name, &policy.EnterpriseMeta)
if err != nil {
return fmt.Errorf("acl policy lookup by name failed: %v", err)
}
@ -980,7 +1051,13 @@ func (a *ACL) PolicySet(args *structs.ACLPolicySetRequest, reply *structs.ACLPol
}
// validate the rules
_, err = acl.NewPolicyFromSource("", 0, policy.Rules, policy.Syntax, a.srv.enterpriseACLConfig)
_, err = acl.NewPolicyFromSource("", 0, policy.Rules, policy.Syntax, a.srv.enterpriseACLConfig, policy.EnterprisePolicyMeta())
if err != nil {
return err
}
// validate the enterprise meta
err = state.ACLPolicyUpsertValidateEnterprise(policy, idMatch)
if err != nil {
return err
}
@ -1004,7 +1081,7 @@ func (a *ACL) PolicySet(args *structs.ACLPolicySetRequest, reply *structs.ACLPol
return respErr
}
if _, policy, err := a.srv.fsm.State().ACLPolicyGetByID(nil, policy.ID); err == nil && policy != nil {
if _, policy, err := a.srv.fsm.State().ACLPolicyGetByID(nil, policy.ID, &policy.EnterpriseMeta); err == nil && policy != nil {
*reply = *policy
}
@ -1027,14 +1104,16 @@ func (a *ACL) PolicyDelete(args *structs.ACLPolicyDeleteRequest, reply *string)
defer metrics.MeasureSince([]string{"acl", "policy", "delete"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
_, policy, err := a.srv.fsm.State().ACLPolicyGetByID(nil, args.PolicyID)
_, policy, err := a.srv.fsm.State().ACLPolicyGetByID(nil, args.PolicyID, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -1078,20 +1157,23 @@ func (a *ACL) PolicyList(args *structs.ACLPolicyListRequest, reply *structs.ACLP
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, policies, err := state.ACLPolicyList(ws)
index, policies, err := state.ACLPolicyList(ws, &args.EnterpriseMeta)
if err != nil {
return err
}
// filter down to just what the requester has permissions to see
a.srv.filterACLWithAuthorizer(rule, &policies)
var stubs structs.ACLPolicyListStubs
for _, policy := range policies {
stubs = append(stubs, policy.Stub())
@ -1224,10 +1306,12 @@ func (a *ACL) RoleRead(args *structs.ACLRoleGetRequest, reply *structs.ACLRoleRe
return err
}
// TODO (namespaces) update to create and use actual enterprise authorizer context
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil || rule.ACLRead(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -1239,9 +1323,9 @@ func (a *ACL) RoleRead(args *structs.ACLRoleGetRequest, reply *structs.ACLRoleRe
err error
)
if args.RoleID != "" {
index, role, err = state.ACLRoleGetByID(ws, args.RoleID)
index, role, err = state.ACLRoleGetByID(ws, args.RoleID, &args.EnterpriseMeta)
} else {
index, role, err = state.ACLRoleGetByName(ws, args.RoleName)
index, role, err = state.ACLRoleGetByName(ws, args.RoleName, &args.EnterpriseMeta)
}
if err != nil {
@ -1262,10 +1346,10 @@ func (a *ACL) RoleBatchRead(args *structs.ACLRoleBatchGetRequest, reply *structs
return err
}
// TODO (namespaces) update to create and use actual enterprise authorizer context
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
@ -1276,6 +1360,8 @@ func (a *ACL) RoleBatchRead(args *structs.ACLRoleBatchGetRequest, reply *structs
return err
}
a.srv.filterACLWithAuthorizer(rule, &roles)
reply.Index, reply.Roles = index, roles
return nil
})
@ -1297,10 +1383,12 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
defer metrics.MeasureSince([]string{"acl", "role", "upsert"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.Role.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -1330,7 +1418,7 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
}
// validate the name is unique
if _, existing, err := state.ACLRoleGetByName(nil, role.Name); err != nil {
if _, existing, err := state.ACLRoleGetByName(nil, role.Name, &role.EnterpriseMeta); err != nil {
return fmt.Errorf("acl role lookup by name failed: %v", err)
} else if existing != nil {
return fmt.Errorf("Invalid Role: A Role with Name %q already exists", role.Name)
@ -1341,7 +1429,7 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
}
// Verify the role exists
_, existing, err := state.ACLRoleGetByID(nil, role.ID)
_, existing, err := state.ACLRoleGetByID(nil, role.ID, nil)
if err != nil {
return fmt.Errorf("acl role lookup failed: %v", err)
} else if existing == nil {
@ -1349,7 +1437,7 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
}
if existing.Name != role.Name {
if _, nameMatch, err := state.ACLRoleGetByName(nil, role.Name); err != nil {
if _, nameMatch, err := state.ACLRoleGetByName(nil, role.Name, &role.EnterpriseMeta); err != nil {
return fmt.Errorf("acl role lookup by name failed: %v", err)
} else if nameMatch != nil {
return fmt.Errorf("Invalid Role: A role with name %q already exists", role.Name)
@ -1363,7 +1451,7 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
// Validate all the policy names and convert them to policy IDs
for _, link := range role.Policies {
if link.ID == "" {
_, policy, err := state.ACLPolicyGetByName(nil, link.Name)
_, policy, err := state.ACLPolicyGetByName(nil, link.Name, &role.EnterpriseMeta)
if err != nil {
return fmt.Errorf("Error looking up policy for name %q: %v", link.Name, err)
}
@ -1413,7 +1501,7 @@ func (a *ACL) RoleSet(args *structs.ACLRoleSetRequest, reply *structs.ACLRole) e
return respErr
}
if _, role, err := a.srv.fsm.State().ACLRoleGetByID(nil, role.ID); err == nil && role != nil {
if _, role, err := a.srv.fsm.State().ACLRoleGetByID(nil, role.ID, &role.EnterpriseMeta); err == nil && role != nil {
*reply = *role
}
@ -1436,14 +1524,16 @@ func (a *ACL) RoleDelete(args *structs.ACLRoleDeleteRequest, reply *string) erro
defer metrics.MeasureSince([]string{"acl", "role", "delete"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
_, role, err := a.srv.fsm.State().ACLRoleGetByID(nil, args.RoleID)
_, role, err := a.srv.fsm.State().ACLRoleGetByID(nil, args.RoleID, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -1483,20 +1573,22 @@ func (a *ACL) RoleList(args *structs.ACLRoleListRequest, reply *structs.ACLRoleL
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, roles, err := state.ACLRoleList(ws, args.Policy)
index, roles, err := state.ACLRoleList(ws, args.Policy, &args.EnterpriseMeta)
if err != nil {
return err
}
a.srv.filterACLWithAuthorizer(rule, &roles)
reply.Index, reply.Roles = index, roles
return nil
})
@ -1560,16 +1652,19 @@ func (a *ACL) BindingRuleRead(args *structs.ACLBindingRuleGetRequest, reply *str
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil || rule.ACLRead(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, rule, err := state.ACLBindingRuleGetByID(ws, args.BindingRuleID)
index, rule, err := state.ACLBindingRuleGetByID(ws, args.BindingRuleID, &args.EnterpriseMeta)
if err != nil {
return err
@ -1595,14 +1690,17 @@ func (a *ACL) BindingRuleSet(args *structs.ACLBindingRuleSetRequest, reply *stru
defer metrics.MeasureSince([]string{"acl", "bindingrule", "upsert"}, time.Now())
var entCtx acl.EnterpriseAuthorizerContext
args.BindingRule.FillAuthzContext(&entCtx)
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
var existing *structs.ACLBindingRule
rule := &args.BindingRule
state := a.srv.fsm.State()
@ -1620,7 +1718,8 @@ func (a *ACL) BindingRuleSet(args *structs.ACLBindingRuleSetRequest, reply *stru
}
// Verify the role exists
_, existing, err := state.ACLBindingRuleGetByID(nil, rule.ID)
var err error
_, existing, err = state.ACLBindingRuleGetByID(nil, rule.ID, nil)
if err != nil {
return fmt.Errorf("acl binding rule lookup failed: %v", err)
} else if existing == nil {
@ -1638,7 +1737,12 @@ func (a *ACL) BindingRuleSet(args *structs.ACLBindingRuleSetRequest, reply *stru
return fmt.Errorf("Invalid Binding Rule: no AuthMethod is set")
}
methodIdx, method, err := state.ACLAuthMethodGetByName(nil, rule.AuthMethod)
// this is done early here to produce better errors
if err := state.ACLBindingRuleUpsertValidateEnterprise(rule, existing); err != nil {
return err
}
methodIdx, method, err := state.ACLAuthMethodGetByName(nil, rule.AuthMethod, &args.BindingRule.EnterpriseMeta)
if err != nil {
return fmt.Errorf("acl auth method lookup failed: %v", err)
} else if method == nil {
@ -1691,7 +1795,7 @@ func (a *ACL) BindingRuleSet(args *structs.ACLBindingRuleSetRequest, reply *stru
return respErr
}
if _, rule, err := a.srv.fsm.State().ACLBindingRuleGetByID(nil, rule.ID); err == nil && rule != nil {
if _, rule, err := a.srv.fsm.State().ACLBindingRuleGetByID(nil, rule.ID, &rule.EnterpriseMeta); err == nil && rule != nil {
*reply = *rule
}
@ -1713,15 +1817,17 @@ func (a *ACL) BindingRuleDelete(args *structs.ACLBindingRuleDeleteRequest, reply
defer metrics.MeasureSince([]string{"acl", "bindingrule", "delete"}, time.Now())
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
_, rule, err := a.srv.fsm.State().ACLBindingRuleGetByID(nil, args.BindingRuleID)
_, rule, err := a.srv.fsm.State().ACLBindingRuleGetByID(nil, args.BindingRuleID, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -1761,20 +1867,22 @@ func (a *ACL) BindingRuleList(args *structs.ACLBindingRuleListRequest, reply *st
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, rules, err := state.ACLBindingRuleList(ws, args.AuthMethod)
index, rules, err := state.ACLBindingRuleList(ws, args.AuthMethod, &args.EnterpriseMeta)
if err != nil {
return err
}
a.srv.filterACLWithAuthorizer(rule, &rules)
reply.Index, reply.BindingRules = index, rules
return nil
})
@ -1793,16 +1901,18 @@ func (a *ACL) AuthMethodRead(args *structs.ACLAuthMethodGetRequest, reply *struc
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil || rule.ACLRead(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, method, err := state.ACLAuthMethodGetByName(ws, args.AuthMethodName)
index, method, err := state.ACLAuthMethodGetByName(ws, args.AuthMethodName, &args.EnterpriseMeta)
if err != nil {
return err
@ -1829,10 +1939,12 @@ func (a *ACL) AuthMethodSet(args *structs.ACLAuthMethodSetRequest, reply *struct
defer metrics.MeasureSince([]string{"acl", "authmethod", "upsert"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.AuthMethod.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
@ -1848,7 +1960,7 @@ func (a *ACL) AuthMethodSet(args *structs.ACLAuthMethodSetRequest, reply *struct
}
// Check to see if the method exists first.
_, existing, err := state.ACLAuthMethodGetByName(nil, method.Name)
_, existing, err := state.ACLAuthMethodGetByName(nil, method.Name, &method.EnterpriseMeta)
if err != nil {
return fmt.Errorf("acl auth method lookup failed: %v", err)
}
@ -1871,6 +1983,10 @@ func (a *ACL) AuthMethodSet(args *structs.ACLAuthMethodSetRequest, reply *struct
return fmt.Errorf("Invalid Auth Method: %v", err)
}
if err := a.srv.fsm.State().ACLAuthMethodUpsertValidateEnterprise(method, existing); err != nil {
return err
}
req := &structs.ACLAuthMethodBatchSetRequest{
AuthMethods: structs.ACLAuthMethods{method},
}
@ -1884,7 +2000,7 @@ func (a *ACL) AuthMethodSet(args *structs.ACLAuthMethodSetRequest, reply *struct
return respErr
}
if _, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, method.Name); err == nil && method != nil {
if _, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, method.Name, &method.EnterpriseMeta); err == nil && method != nil {
*reply = *method
}
@ -1907,14 +2023,16 @@ func (a *ACL) AuthMethodDelete(args *structs.ACLAuthMethodDeleteRequest, reply *
defer metrics.MeasureSince([]string{"acl", "authmethod", "delete"}, time.Now())
// Verify token is permitted to modify ACLs
// TODO (namespaces) update to call ACLWrite with an authz context once ACLs support it
var entCtx acl.EnterpriseAuthorizerContext
args.FillAuthzContext(&entCtx)
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
return err
} else if rule == nil || rule.ACLWrite(nil) != acl.Allow {
} else if rule == nil || rule.ACLWrite(&entCtx) != acl.Allow {
return acl.ErrPermissionDenied
}
_, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, args.AuthMethodName)
_, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, args.AuthMethodName, &args.EnterpriseMeta)
if err != nil {
return err
}
@ -1925,6 +2043,7 @@ func (a *ACL) AuthMethodDelete(args *structs.ACLAuthMethodDeleteRequest, reply *
req := structs.ACLAuthMethodBatchDeleteRequest{
AuthMethodNames: []string{args.AuthMethodName},
EnterpriseMeta: args.EnterpriseMeta,
}
resp, err := a.srv.raftApply(structs.ACLAuthMethodDeleteRequestType, &req)
@ -1954,20 +2073,22 @@ func (a *ACL) AuthMethodList(args *structs.ACLAuthMethodListRequest, reply *stru
return err
}
// TODO (namespaces) update to call ACLRead with an authz context once ACLs support it
if rule, err := a.srv.ResolveToken(args.Token); err != nil {
rule, err := a.srv.ResolveToken(args.Token)
if err != nil {
return err
} else if rule == nil || rule.ACLRead(nil) != acl.Allow {
} else if rule == nil {
return acl.ErrPermissionDenied
}
return a.srv.blockingQuery(&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, methods, err := state.ACLAuthMethodList(ws)
index, methods, err := state.ACLAuthMethodList(ws, &args.EnterpriseMeta)
if err != nil {
return err
}
a.srv.filterACLWithAuthorizer(rule, &methods)
var stubs structs.ACLAuthMethodListStubs
for _, method := range methods {
stubs = append(stubs, method.Stub())
@ -2000,7 +2121,7 @@ func (a *ACL) Login(args *structs.ACLLoginRequest, reply *structs.ACLToken) erro
auth := args.Auth
// 1. take args.Data.AuthMethod to get an AuthMethod Validator
idx, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, auth.AuthMethod)
idx, method, err := a.srv.fsm.State().ACLAuthMethodGetByName(nil, auth.AuthMethod, &auth.EnterpriseMeta)
if err != nil {
return err
} else if method == nil {
@ -2019,7 +2140,7 @@ func (a *ACL) Login(args *structs.ACLLoginRequest, reply *structs.ACLToken) erro
}
// 3. send map through role bindings
serviceIdentities, roleLinks, err := a.srv.evaluateRoleBindings(validator, verifiedFields)
serviceIdentities, roleLinks, err := a.srv.evaluateRoleBindings(validator, verifiedFields, &auth.EnterpriseMeta)
if err != nil {
return err
}
@ -2048,6 +2169,7 @@ func (a *ACL) Login(args *structs.ACLLoginRequest, reply *structs.ACLToken) erro
AuthMethod: auth.AuthMethod,
ServiceIdentities: serviceIdentities,
Roles: roleLinks,
EnterpriseMeta: auth.EnterpriseMeta,
},
WriteRequest: args.WriteRequest,
}
@ -2097,7 +2219,7 @@ func (a *ACL) Logout(args *structs.ACLLogoutRequest, reply *bool) error {
defer metrics.MeasureSince([]string{"acl", "logout"}, time.Now())
_, token, err := a.srv.fsm.State().ACLTokenGetBySecret(nil, args.Token)
_, token, err := a.srv.fsm.State().ACLTokenGetBySecret(nil, args.Token, nil)
if err != nil {
return err

View File

@ -94,7 +94,7 @@ func aclApplyInternal(srv *Server, args *structs.ACLRequest, reply *string) erro
}
// No need to check expiration times as those did not exist in legacy tokens.
_, existing, _ := srv.fsm.State().ACLTokenGetBySecret(nil, args.ACL.ID)
_, existing, _ := srv.fsm.State().ACLTokenGetBySecret(nil, args.ACL.ID, nil)
if existing != nil && existing.UsesNonLegacyFields() {
return fmt.Errorf("Cannot use legacy endpoint to modify a non-legacy token")
}
@ -114,7 +114,7 @@ func aclApplyInternal(srv *Server, args *structs.ACLRequest, reply *string) erro
}
// Validate the rules compile
_, err := acl.NewPolicyFromSource("", 0, args.ACL.Rules, acl.SyntaxLegacy, srv.enterpriseACLConfig)
_, err := acl.NewPolicyFromSource("", 0, args.ACL.Rules, acl.SyntaxLegacy, srv.enterpriseACLConfig, nil)
if err != nil {
return fmt.Errorf("ACL rule compilation failed: %v", err)
}
@ -211,7 +211,7 @@ func (a *ACL) Get(args *structs.ACLSpecificRequest,
return a.srv.blockingQuery(&args.QueryOptions,
&reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, token, err := state.ACLTokenGetBySecret(ws, args.ACL)
index, token, err := state.ACLTokenGetBySecret(ws, args.ACL, nil)
if err != nil {
return err
}
@ -262,7 +262,7 @@ func (a *ACL) List(args *structs.DCSpecificRequest,
return a.srv.blockingQuery(&args.QueryOptions,
&reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, tokens, err := state.ACLTokenList(ws, false, true, "", "", "")
index, tokens, err := state.ACLTokenList(ws, false, true, "", "", "", nil)
if err != nil {
return err
}

View File

@ -144,7 +144,7 @@ func TestACLEndpoint_Apply(t *testing.T) {
// Verify
state := s1.fsm.State()
_, s, err := state.ACLTokenGetBySecret(nil, out)
_, s, err := state.ACLTokenGetBySecret(nil, out, nil)
if err != nil {
t.Fatalf("err: %v", err)
}
@ -166,7 +166,7 @@ func TestACLEndpoint_Apply(t *testing.T) {
}
// Verify
_, s, err = state.ACLTokenGetBySecret(nil, id)
_, s, err = state.ACLTokenGetBySecret(nil, id, nil)
if err != nil {
t.Fatalf("err: %v", err)
}
@ -289,7 +289,7 @@ func TestACLEndpoint_Apply_CustomID(t *testing.T) {
// Verify
state := s1.fsm.State()
_, s, err := state.ACLTokenGetBySecret(nil, out)
_, s, err := state.ACLTokenGetBySecret(nil, out, nil)
if err != nil {
t.Fatalf("err: %v", err)
}
@ -733,7 +733,7 @@ func TestACLEndpoint_TokenRead(t *testing.T) {
err := acl.TokenRead(&req, &resp)
require.Nil(t, resp.Token)
require.EqualError(t, err, "failed acl token lookup: failed acl token lookup: index error: UUID must be 36 characters")
require.EqualError(t, err, "failed acl token lookup: index error: UUID must be 36 characters")
})
}
@ -5358,12 +5358,7 @@ func deleteTestPolicy(codec rpc.ClientCodec, masterToken string, datacenter stri
return err
}
// upsertTestPolicy creates a policy for testing purposes
func upsertTestPolicy(codec rpc.ClientCodec, masterToken string, datacenter string) (*structs.ACLPolicy, error) {
return upsertTestPolicyWithRules(codec, masterToken, datacenter, "")
}
func upsertTestPolicyWithRules(codec rpc.ClientCodec, masterToken string, datacenter string, rules string) (*structs.ACLPolicy, error) {
func upsertTestCustomizedPolicy(codec rpc.ClientCodec, masterToken string, datacenter string, policyModificationFn func(policy *structs.ACLPolicy)) (*structs.ACLPolicy, error) {
// Make sure test policies can't collide
policyUnq, err := uuid.GenerateUUID()
if err != nil {
@ -5373,12 +5368,15 @@ func upsertTestPolicyWithRules(codec rpc.ClientCodec, masterToken string, datace
arg := structs.ACLPolicySetRequest{
Datacenter: datacenter,
Policy: structs.ACLPolicy{
Name: fmt.Sprintf("test-policy-%s", policyUnq),
Rules: rules,
Name: fmt.Sprintf("test-policy-%s", policyUnq),
},
WriteRequest: structs.WriteRequest{Token: masterToken},
}
if policyModificationFn != nil {
policyModificationFn(&arg.Policy)
}
var out structs.ACLPolicy
err = msgpackrpc.CallWithCodec(codec, "ACL.PolicySet", &arg, &out)
@ -5394,6 +5392,17 @@ func upsertTestPolicyWithRules(codec rpc.ClientCodec, masterToken string, datace
return &out, nil
}
// upsertTestPolicy creates a policy for testing purposes
func upsertTestPolicy(codec rpc.ClientCodec, masterToken string, datacenter string) (*structs.ACLPolicy, error) {
return upsertTestPolicyWithRules(codec, masterToken, datacenter, "")
}
func upsertTestPolicyWithRules(codec rpc.ClientCodec, masterToken string, datacenter string, rules string) (*structs.ACLPolicy, error) {
return upsertTestCustomizedPolicy(codec, masterToken, datacenter, func(policy *structs.ACLPolicy) {
policy.Rules = rules
})
}
// retrieveTestPolicy returns a policy for testing purposes
func retrieveTestPolicy(codec rpc.ClientCodec, masterToken string, datacenter string, id string) (*structs.ACLPolicyResponse, error) {
arg := structs.ACLPolicyGetRequest{
@ -5439,6 +5448,10 @@ func deleteTestRoleByName(codec rpc.ClientCodec, masterToken string, datacenter
// upsertTestRole creates a role for testing purposes
func upsertTestRole(codec rpc.ClientCodec, masterToken string, datacenter string) (*structs.ACLRole, error) {
return upsertTestCustomizedRole(codec, masterToken, datacenter, nil)
}
func upsertTestCustomizedRole(codec rpc.ClientCodec, masterToken string, datacenter string, modify func(role *structs.ACLRole)) (*structs.ACLRole, error) {
// Make sure test roles can't collide
roleUnq, err := uuid.GenerateUUID()
if err != nil {
@ -5453,6 +5466,10 @@ func upsertTestRole(codec rpc.ClientCodec, masterToken string, datacenter string
WriteRequest: structs.WriteRequest{Token: masterToken},
}
if modify != nil {
modify(&arg.Role)
}
var out structs.ACLRole
err = msgpackrpc.CallWithCodec(codec, "ACL.RoleSet", &arg, &out)
@ -5518,6 +5535,17 @@ func deleteTestAuthMethod(codec rpc.ClientCodec, masterToken string, datacenter
func upsertTestAuthMethod(
codec rpc.ClientCodec, masterToken string, datacenter string,
sessionID string,
) (*structs.ACLAuthMethod, error) {
return upsertTestCustomizedAuthMethod(codec, masterToken, datacenter, func(method *structs.ACLAuthMethod) {
method.Config = map[string]interface{}{
"SessionID": sessionID,
}
})
}
func upsertTestCustomizedAuthMethod(
codec rpc.ClientCodec, masterToken string, datacenter string,
modify func(method *structs.ACLAuthMethod),
) (*structs.ACLAuthMethod, error) {
name, err := uuid.GenerateUUID()
if err != nil {
@ -5529,13 +5557,14 @@ func upsertTestAuthMethod(
AuthMethod: structs.ACLAuthMethod{
Name: "test-method-" + name,
Type: "testing",
Config: map[string]interface{}{
"SessionID": sessionID,
},
},
WriteRequest: structs.WriteRequest{Token: masterToken},
}
if modify != nil {
modify(&req.AuthMethod)
}
var out structs.ACLAuthMethod
err = msgpackrpc.CallWithCodec(codec, "ACL.AuthMethodSet", &req, &out)
@ -5625,17 +5654,25 @@ func upsertTestBindingRule(
bindType string,
bindName string,
) (*structs.ACLBindingRule, error) {
return upsertTestCustomizedBindingRule(codec, masterToken, datacenter, func(rule *structs.ACLBindingRule) {
rule.AuthMethod = methodName
rule.BindType = bindType
rule.BindName = bindName
rule.Selector = selector
})
}
func upsertTestCustomizedBindingRule(codec rpc.ClientCodec, masterToken string, datacenter string, modify func(rule *structs.ACLBindingRule)) (*structs.ACLBindingRule, error) {
req := structs.ACLBindingRuleSetRequest{
Datacenter: datacenter,
BindingRule: structs.ACLBindingRule{
AuthMethod: methodName,
BindType: bindType,
BindName: bindName,
Selector: selector,
},
Datacenter: datacenter,
BindingRule: structs.ACLBindingRule{},
WriteRequest: structs.WriteRequest{Token: masterToken},
}
if modify != nil {
modify(&req.BindingRule)
}
var out structs.ACLBindingRule
err := msgpackrpc.CallWithCodec(codec, "ACL.BindingRuleSet", &req, &out)

View File

@ -138,7 +138,7 @@ func reconcileLegacyACLs(local, remote structs.ACLs, lastRemoteIndex uint64) str
// FetchLocalACLs returns the ACLs in the local state store.
func (s *Server) fetchLocalLegacyACLs() (structs.ACLs, error) {
_, local, err := s.fsm.State().ACLTokenList(nil, false, true, "", "", "")
_, local, err := s.fsm.State().ACLTokenList(nil, false, true, "", "", "", nil)
if err != nil {
return nil, err
}

View File

@ -396,11 +396,11 @@ func TestACLReplication_LegacyTokens(t *testing.T) {
}
checkSame := func() error {
index, remote, err := s1.fsm.State().ACLTokenList(nil, true, true, "", "", "")
index, remote, err := s1.fsm.State().ACLTokenList(nil, true, true, "", "", "", nil)
if err != nil {
return err
}
_, local, err := s2.fsm.State().ACLTokenList(nil, true, true, "", "", "")
_, local, err := s2.fsm.State().ACLTokenList(nil, true, true, "", "", "", nil)
if err != nil {
return err
}

View File

@ -351,9 +351,9 @@ func TestACLReplication_Tokens(t *testing.T) {
checkSame := func(t *retry.R) {
// only account for global tokens - local tokens shouldn't be replicated
index, remote, err := s1.fsm.State().ACLTokenList(nil, false, true, "", "", "")
index, remote, err := s1.fsm.State().ACLTokenList(nil, false, true, "", "", "", nil)
require.NoError(t, err)
_, local, err := s2.fsm.State().ACLTokenList(nil, false, true, "", "", "")
_, local, err := s2.fsm.State().ACLTokenList(nil, false, true, "", "", "", nil)
require.NoError(t, err)
require.Len(t, local, len(remote))
@ -378,7 +378,7 @@ func TestACLReplication_Tokens(t *testing.T) {
// Wait for s2 global-management policy
retry.Run(t, func(r *retry.R) {
_, policy, err := s2.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID)
_, policy, err := s2.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID, nil)
require.NoError(r, err)
require.NotNil(r, policy)
})
@ -451,7 +451,7 @@ func TestACLReplication_Tokens(t *testing.T) {
})
// verify dc2 local tokens didn't get blown away
_, local, err := s2.fsm.State().ACLTokenList(nil, true, false, "", "", "")
_, local, err := s2.fsm.State().ACLTokenList(nil, true, false, "", "", "", nil)
require.NoError(t, err)
require.Len(t, local, 50)
@ -529,9 +529,9 @@ func TestACLReplication_Policies(t *testing.T) {
checkSame := func(t *retry.R) {
// only account for global tokens - local tokens shouldn't be replicated
index, remote, err := s1.fsm.State().ACLPolicyList(nil)
index, remote, err := s1.fsm.State().ACLPolicyList(nil, nil)
require.NoError(t, err)
_, local, err := s2.fsm.State().ACLPolicyList(nil)
_, local, err := s2.fsm.State().ACLPolicyList(nil, nil)
require.NoError(t, err)
require.Len(t, local, len(remote))
@ -787,10 +787,10 @@ func TestACLReplication_AllTypes(t *testing.T) {
checkSameTokens := func(t *retry.R) {
// only account for global tokens - local tokens shouldn't be replicated
index, remote, err := s1.fsm.State().ACLTokenList(nil, false, true, "", "", "")
index, remote, err := s1.fsm.State().ACLTokenList(nil, false, true, "", "", "", nil)
require.NoError(t, err)
// Query for all of them, so that we can prove that no globals snuck in.
_, local, err := s2.fsm.State().ACLTokenList(nil, true, true, "", "", "")
_, local, err := s2.fsm.State().ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
require.Len(t, remote, len(local))
@ -809,9 +809,9 @@ func TestACLReplication_AllTypes(t *testing.T) {
require.Equal(t, status.SourceDatacenter, "dc1")
}
checkSamePolicies := func(t *retry.R) {
index, remote, err := s1.fsm.State().ACLPolicyList(nil)
index, remote, err := s1.fsm.State().ACLPolicyList(nil, nil)
require.NoError(t, err)
_, local, err := s2.fsm.State().ACLPolicyList(nil)
_, local, err := s2.fsm.State().ACLPolicyList(nil, nil)
require.NoError(t, err)
require.Len(t, remote, len(local))
@ -830,9 +830,9 @@ func TestACLReplication_AllTypes(t *testing.T) {
require.Equal(t, status.SourceDatacenter, "dc1")
}
checkSameRoles := func(t *retry.R) {
index, remote, err := s1.fsm.State().ACLRoleList(nil, "")
index, remote, err := s1.fsm.State().ACLRoleList(nil, "", nil)
require.NoError(t, err)
_, local, err := s2.fsm.State().ACLRoleList(nil, "")
_, local, err := s2.fsm.State().ACLRoleList(nil, "", nil)
require.NoError(t, err)
require.Len(t, remote, len(local))

View File

@ -34,7 +34,7 @@ func (r *aclTokenReplicator) FetchRemote(srv *Server, lastRemoteIndex uint64) (i
func (r *aclTokenReplicator) FetchLocal(srv *Server) (int, uint64, error) {
r.local = nil
idx, local, err := srv.fsm.State().ACLTokenList(nil, false, true, "", "", "")
idx, local, err := srv.fsm.State().ACLTokenList(nil, false, true, "", "", "", structs.ReplicationEnterpriseMeta())
if err != nil {
return 0, 0, err
}
@ -155,7 +155,7 @@ func (r *aclPolicyReplicator) FetchRemote(srv *Server, lastRemoteIndex uint64) (
func (r *aclPolicyReplicator) FetchLocal(srv *Server) (int, uint64, error) {
r.local = nil
idx, local, err := srv.fsm.State().ACLPolicyList(nil)
idx, local, err := srv.fsm.State().ACLPolicyList(nil, structs.ReplicationEnterpriseMeta())
if err != nil {
return 0, 0, err
}
@ -265,7 +265,7 @@ func (r *aclRoleReplicator) FetchRemote(srv *Server, lastRemoteIndex uint64) (in
func (r *aclRoleReplicator) FetchLocal(srv *Server) (int, uint64, error) {
r.local = nil
idx, local, err := srv.fsm.State().ACLRoleList(nil, "")
idx, local, err := srv.fsm.State().ACLRoleList(nil, "", nil)
if err != nil {
return 0, 0, err
}

View File

@ -36,13 +36,13 @@ func (s *Server) checkTokenUUID(id string) (bool, error) {
// a token that hasn't been reaped yet, then we won't be able to insert the
// new token due to a collision.
if _, token, err := state.ACLTokenGetByAccessor(nil, id); err != nil {
if _, token, err := state.ACLTokenGetByAccessor(nil, id, nil); err != nil {
return false, err
} else if token != nil {
return false, nil
}
if _, token, err := state.ACLTokenGetBySecret(nil, id); err != nil {
if _, token, err := state.ACLTokenGetBySecret(nil, id, nil); err != nil {
return false, err
} else if token != nil {
return false, nil
@ -53,7 +53,7 @@ func (s *Server) checkTokenUUID(id string) (bool, error) {
func (s *Server) checkPolicyUUID(id string) (bool, error) {
state := s.fsm.State()
if _, policy, err := state.ACLPolicyGetByID(nil, id); err != nil {
if _, policy, err := state.ACLPolicyGetByID(nil, id, nil); err != nil {
return false, err
} else if policy != nil {
return false, nil
@ -64,7 +64,7 @@ func (s *Server) checkPolicyUUID(id string) (bool, error) {
func (s *Server) checkRoleUUID(id string) (bool, error) {
state := s.fsm.State()
if _, role, err := state.ACLRoleGetByID(nil, id); err != nil {
if _, role, err := state.ACLRoleGetByID(nil, id, nil); err != nil {
return false, err
} else if role != nil {
return false, nil
@ -75,7 +75,7 @@ func (s *Server) checkRoleUUID(id string) (bool, error) {
func (s *Server) checkBindingRuleUUID(id string) (bool, error) {
state := s.fsm.State()
if _, rule, err := state.ACLBindingRuleGetByID(nil, id); err != nil {
if _, rule, err := state.ACLBindingRuleGetByID(nil, id, nil); err != nil {
return false, err
} else if rule != nil {
return false, nil
@ -171,7 +171,7 @@ func (s *Server) ResolveIdentityFromToken(token string) (bool, structs.ACLIdenti
return false, nil, nil
}
index, aclToken, err := s.fsm.State().ACLTokenGetBySecret(nil, token)
index, aclToken, err := s.fsm.State().ACLTokenGetBySecret(nil, token, nil)
if err != nil {
return true, nil, err
} else if aclToken != nil && !aclToken.IsExpired(time.Now()) {
@ -182,7 +182,7 @@ func (s *Server) ResolveIdentityFromToken(token string) (bool, structs.ACLIdenti
}
func (s *Server) ResolvePolicyFromID(policyID string) (bool, *structs.ACLPolicy, error) {
index, policy, err := s.fsm.State().ACLPolicyGetByID(nil, policyID)
index, policy, err := s.fsm.State().ACLPolicyGetByID(nil, policyID, nil)
if err != nil {
return true, nil, err
} else if policy != nil {
@ -196,7 +196,7 @@ func (s *Server) ResolvePolicyFromID(policyID string) (bool, *structs.ACLPolicy,
}
func (s *Server) ResolveRoleFromID(roleID string) (bool, *structs.ACLRole, error) {
index, role, err := s.fsm.State().ACLRoleGetByID(nil, roleID)
index, role, err := s.fsm.State().ACLRoleGetByID(nil, roleID, nil)
if err != nil {
return true, nil, err
} else if role != nil {

View File

@ -2176,7 +2176,7 @@ func TestACL_filterHealthChecks(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2210,7 +2210,7 @@ service "foo" {
node "node1" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2268,7 +2268,7 @@ func TestACL_filterIntentions(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
assert.Nil(err)
perms, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
assert.Nil(err)
@ -2353,7 +2353,7 @@ func TestACL_filterServiceNodes(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2387,7 +2387,7 @@ service "foo" {
node "node1" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2459,7 +2459,7 @@ func TestACL_filterNodeServices(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2493,7 +2493,7 @@ service "foo" {
node "node1" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2565,7 +2565,7 @@ func TestACL_filterCheckServiceNodes(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2602,7 +2602,7 @@ service "foo" {
node "node1" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2756,7 +2756,7 @@ func TestACL_filterNodeDump(t *testing.T) {
service "foo" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -2796,7 +2796,7 @@ service "foo" {
node "node1" {
policy = "read"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3052,7 +3052,7 @@ func TestACL_vetRegisterWithACL(t *testing.T) {
node "node" {
policy = "write"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3097,7 +3097,7 @@ node "node" {
service "service" {
policy = "write"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3127,7 +3127,7 @@ service "service" {
service "other" {
policy = "write"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3201,7 +3201,7 @@ service "other" {
service "other" {
policy = "deny"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3231,7 +3231,7 @@ service "other" {
node "node" {
policy = "deny"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3278,7 +3278,7 @@ func TestACL_vetDeregisterWithACL(t *testing.T) {
node "node" {
policy = "write"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}
@ -3291,7 +3291,7 @@ node "node" {
service "my-service" {
policy = "write"
}
`, acl.SyntaxLegacy, nil)
`, acl.SyntaxLegacy, nil, nil)
if err != nil {
t.Fatalf("err %v", err)
}

View File

@ -9,6 +9,21 @@ import (
"github.com/mitchellh/mapstructure"
)
type Cache interface {
// GetValidator retrieves the Validator from the cache.
// It returns the modify index of struct that the validator was created from,
// the validator and a boolean indicating whether the value was found
GetValidator(method *structs.ACLAuthMethod) (uint64, Validator, bool)
// PutValidatorIfNewer inserts a new validator into the cache if the index is greater
// than the modify index of any existing entry in the cache. This method will return
// the newest validator which may or may not be the one from the method parameter
PutValidatorIfNewer(method *structs.ACLAuthMethod, validator Validator, idx uint64) Validator
// Purge removes all cached validators
Purge()
}
type ValidatorFactory func(method *structs.ACLAuthMethod) (Validator, error)
type Validator interface {
@ -64,6 +79,54 @@ func IsRegisteredType(typeName string) bool {
return ok
}
type authMethodValidatorEntry struct {
Validator Validator
ModifyIndex uint64
}
// authMethodCache is an non-thread-safe cache that maps ACLAuthMethods to their Validators
type authMethodCache struct {
entries map[string]*authMethodValidatorEntry
}
func newCache() Cache {
c := &authMethodCache{}
c.init()
return c
}
func (c *authMethodCache) init() {
c.Purge()
}
func (c *authMethodCache) GetValidator(method *structs.ACLAuthMethod) (uint64, Validator, bool) {
entry, ok := c.entries[method.Name]
if ok {
return entry.ModifyIndex, entry.Validator, true
}
return 0, nil, false
}
func (c *authMethodCache) PutValidatorIfNewer(method *structs.ACLAuthMethod, validator Validator, idx uint64) Validator {
prev, ok := c.entries[method.Name]
if ok {
if prev.ModifyIndex >= idx {
return prev.Validator
}
}
c.entries[method.Name] = &authMethodValidatorEntry{
Validator: validator,
ModifyIndex: idx,
}
return validator
}
func (c *authMethodCache) Purge() {
c.entries = make(map[string]*authMethodValidatorEntry)
}
// NewValidator instantiates a new Validator for the given auth method
// configuration. If no auth method is registered with the provided type an
// error is returned.

View File

@ -0,0 +1,38 @@
// +build !consulent
package authmethod
import (
"sync"
"github.com/hashicorp/consul/agent/structs"
)
type syncCache struct {
lock sync.RWMutex
cache authMethodCache
}
func NewCache() Cache {
c := &syncCache{}
c.cache.init()
return c
}
func (c *syncCache) GetValidator(method *structs.ACLAuthMethod) (uint64, Validator, bool) {
c.lock.RLock()
defer c.lock.RUnlock()
return c.cache.GetValidator(method)
}
func (c *syncCache) PutValidatorIfNewer(method *structs.ACLAuthMethod, validator Validator, idx uint64) Validator {
c.lock.Lock()
defer c.lock.Unlock()
return c.cache.PutValidatorIfNewer(method, validator, idx)
}
func (c *syncCache) Purge() {
c.lock.Lock()
c.cache.Purge()
c.lock.Unlock()
}

View File

@ -10,7 +10,7 @@ import (
func TestFilter_DirEnt(t *testing.T) {
t.Parallel()
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil)
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil, nil)
aclR, _ := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
type tcase struct {
@ -52,7 +52,7 @@ func TestFilter_DirEnt(t *testing.T) {
func TestFilter_Keys(t *testing.T) {
t.Parallel()
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil)
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil, nil)
aclR, _ := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
type tcase struct {
@ -84,7 +84,7 @@ func TestFilter_Keys(t *testing.T) {
func TestFilter_TxnResults(t *testing.T) {
t.Parallel()
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil)
policy, _ := acl.NewPolicyFromSource("", 0, testFilterRules, acl.SyntaxLegacy, nil, nil)
aclR, _ := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
type tcase struct {

View File

@ -172,7 +172,7 @@ func (c *FSM) applyACLOperation(buf []byte, index uint64) interface{} {
}
// No need to check expiration times as those did not exist in legacy tokens.
if _, token, err := c.state.ACLTokenGetBySecret(nil, req.ACL.ID); err != nil {
if _, token, err := c.state.ACLTokenGetBySecret(nil, req.ACL.ID, nil); err != nil {
return err
} else {
acl, err := token.Convert()
@ -188,7 +188,7 @@ func (c *FSM) applyACLOperation(buf []byte, index uint64) interface{} {
}
return req.ACL.ID
case structs.ACLDelete:
return c.state.ACLTokenDeleteBySecret(index, req.ACL.ID)
return c.state.ACLTokenDeleteBySecret(index, req.ACL.ID, nil)
default:
c.logger.Printf("[WARN] consul.fsm: Invalid ACL operation '%s'", req.Op)
return fmt.Errorf("Invalid ACL operation '%s'", req.Op)
@ -533,5 +533,5 @@ func (c *FSM) applyACLAuthMethodDeleteOperation(buf []byte, index uint64) interf
defer metrics.MeasureSinceWithLabels([]string{"fsm", "acl", "authmethod"}, time.Now(),
[]metrics.Label{{Name: "op", Value: "delete"}})
return c.state.ACLAuthMethodBatchDelete(index, req.AuthMethodNames)
return c.state.ACLAuthMethodBatchDelete(index, req.AuthMethodNames, &req.EnterpriseMeta)
}

View File

@ -816,7 +816,7 @@ func TestFSM_ACL_CRUD(t *testing.T) {
// Get the ACL.
id := resp.(string)
_, acl, err := fsm.state.ACLTokenGetBySecret(nil, id)
_, acl, err := fsm.state.ACLTokenGetBySecret(nil, id, nil)
if err != nil {
t.Fatalf("err: %v", err)
}
@ -852,7 +852,7 @@ func TestFSM_ACL_CRUD(t *testing.T) {
t.Fatalf("resp: %v", resp)
}
_, acl, err = fsm.state.ACLTokenGetBySecret(nil, id)
_, acl, err = fsm.state.ACLTokenGetBySecret(nil, id, nil)
if err != nil {
t.Fatalf("err: %v", err)
}

View File

@ -371,17 +371,17 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
}
// Verify ACL Binding Rule is restored
_, bindingRule2, err := fsm2.state.ACLBindingRuleGetByID(nil, bindingRule.ID)
_, bindingRule2, err := fsm2.state.ACLBindingRuleGetByID(nil, bindingRule.ID, nil)
require.NoError(err)
require.Equal(bindingRule, bindingRule2)
// Verify ACL Auth Method is restored
_, method2, err := fsm2.state.ACLAuthMethodGetByName(nil, method.Name)
_, method2, err := fsm2.state.ACLAuthMethodGetByName(nil, method.Name, nil)
require.NoError(err)
require.Equal(method, method2)
// Verify ACL Token is restored
_, token2, err := fsm2.state.ACLTokenGetByAccessor(nil, token.AccessorID)
_, token2, err := fsm2.state.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(err)
{
// time.Time is tricky to compare generically when it takes a ser/deserialization round trip.
@ -396,12 +396,12 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
require.True(index > 0)
// Verify ACL Role is restored
_, role2, err := fsm2.state.ACLRoleGetByID(nil, role.ID)
_, role2, err := fsm2.state.ACLRoleGetByID(nil, role.ID, nil)
require.NoError(err)
require.Equal(role, role2)
// Verify ACL Policy is restored
_, policy2, err := fsm2.state.ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID)
_, policy2, err := fsm2.state.ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID, nil)
require.NoError(err)
require.Equal(policy, policy2)

View File

@ -371,7 +371,7 @@ func (s *Server) initializeLegacyACL() error {
// Create anonymous token if missing.
state := s.fsm.State()
_, token, err := state.ACLTokenGetBySecret(nil, anonymousToken)
_, token, err := state.ACLTokenGetBySecret(nil, anonymousToken, nil)
if err != nil {
return fmt.Errorf("failed to get anonymous token: %v", err)
}
@ -395,7 +395,7 @@ func (s *Server) initializeLegacyACL() error {
// Check for configured master token.
if master := s.config.ACLMasterToken; len(master) > 0 {
_, token, err = state.ACLTokenGetBySecret(nil, master)
_, token, err = state.ACLTokenGetBySecret(nil, master, nil)
if err != nil {
return fmt.Errorf("failed to get master token: %v", err)
}
@ -473,11 +473,11 @@ func (s *Server) initializeACLs(upgrade bool) error {
// Purge the auth method validators since they could've changed while we
// were not leader.
s.purgeAuthMethodValidators()
s.aclAuthMethodValidators.Purge()
// Remove any token affected by CVE-2019-8336
if !s.InACLDatacenter() {
_, token, err := s.fsm.State().ACLTokenGetBySecret(nil, redactedToken)
_, token, err := s.fsm.State().ACLTokenGetBySecret(nil, redactedToken, nil)
if err == nil && token != nil {
req := structs.ACLTokenBatchDeleteRequest{
TokenIDs: []string{token.AccessorID},
@ -499,7 +499,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
s.logger.Printf("[INFO] acl: initializing acls")
// Create/Upgrade the builtin global-management policy
_, policy, err := s.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID)
_, policy, err := s.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID, structs.DefaultEnterpriseMeta())
if err != nil {
return fmt.Errorf("failed to get the builtin global-management policy")
}
@ -516,6 +516,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
newPolicy.Description = policy.Description
}
newPolicy.EnterpriseMeta.InitDefault()
newPolicy.SetHash(true)
req := structs.ACLPolicyBatchSetRequest{
@ -535,7 +536,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
s.logger.Printf("[WARN] consul: Configuring a non-UUID master token is deprecated")
}
_, token, err := state.ACLTokenGetBySecret(nil, master)
_, token, err := state.ACLTokenGetBySecret(nil, master, nil)
if err != nil {
return fmt.Errorf("failed to get master token: %v", err)
}
@ -562,6 +563,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
Type: structs.ACLTokenTypeManagement,
}
token.EnterpriseMeta.InitDefault()
token.SetHash(true)
done := false
@ -597,7 +599,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
}
state := s.fsm.State()
_, token, err := state.ACLTokenGetBySecret(nil, structs.ACLTokenAnonymousID)
_, token, err := state.ACLTokenGetBySecret(nil, structs.ACLTokenAnonymousID, nil)
if err != nil {
return fmt.Errorf("failed to get anonymous token: %v", err)
}
@ -605,7 +607,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
if token == nil {
// DEPRECATED (ACL-Legacy-Compat) - Don't need to query for previous "anonymous" token
// check for legacy token that needs an upgrade
_, legacyToken, err := state.ACLTokenGetBySecret(nil, anonymousToken)
_, legacyToken, err := state.ACLTokenGetBySecret(nil, anonymousToken, nil)
if err != nil {
return fmt.Errorf("failed to get anonymous token: %v", err)
}
@ -620,6 +622,7 @@ func (s *Server) initializeACLs(upgrade bool) error {
CreateTime: time.Now(),
}
token.SetHash(true)
token.EnterpriseMeta.InitDefault()
req := structs.ACLTokenBatchSetRequest{
Tokens: structs.ACLTokens{token},

View File

@ -1049,12 +1049,12 @@ func TestLeader_ACL_Initialization(t *testing.T) {
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
if tt.master != "" {
_, master, err := s1.fsm.State().ACLTokenGetBySecret(nil, tt.master)
_, master, err := s1.fsm.State().ACLTokenGetBySecret(nil, tt.master, nil)
require.NoError(t, err)
require.NotNil(t, master)
}
_, anon, err := s1.fsm.State().ACLTokenGetBySecret(nil, anonymousToken)
_, anon, err := s1.fsm.State().ACLTokenGetBySecret(nil, anonymousToken, nil)
require.NoError(t, err)
require.NotNil(t, anon)
@ -1062,7 +1062,7 @@ func TestLeader_ACL_Initialization(t *testing.T) {
require.NoError(t, err)
require.Equal(t, tt.bootstrap, canBootstrap)
_, policy, err := s1.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID)
_, policy, err := s1.fsm.State().ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID, nil)
require.NoError(t, err)
require.NotNil(t, policy)
})
@ -1096,7 +1096,7 @@ func TestLeader_ACLUpgrade(t *testing.T) {
// wait for it to be upgraded
retry.Run(t, func(t *retry.R) {
_, token, err := s1.fsm.State().ACLTokenGetBySecret(nil, mgmt_id)
_, token, err := s1.fsm.State().ACLTokenGetBySecret(nil, mgmt_id, nil)
require.NoError(t, err)
require.NotNil(t, token)
require.NotEqual(t, "", token.AccessorID)
@ -1121,7 +1121,7 @@ func TestLeader_ACLUpgrade(t *testing.T) {
// wait for it to be upgraded
retry.Run(t, func(t *retry.R) {
_, token, err := s1.fsm.State().ACLTokenGetBySecret(nil, client_id)
_, token, err := s1.fsm.State().ACLTokenGetBySecret(nil, client_id, nil)
require.NoError(t, err)
require.NotNil(t, token)
require.NotEqual(t, "", token.AccessorID)

View File

@ -10,7 +10,6 @@ import (
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/agent/agentpb"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/pool"
@ -388,7 +387,7 @@ func (s *Server) raftApplyMsgpack(t structs.MessageType, msg interface{}) (inter
// raftApplyProtobuf will protobuf encode the request and then run it through raft,
// then return the FSM response along with any errors.
func (s *Server) raftApplyProtobuf(t structs.MessageType, msg interface{}) (interface{}, error) {
return s.raftApplyWithEncoder(t, msg, agentpb.EncodeInterface)
return s.raftApplyWithEncoder(t, msg, structs.EncodeProtoInterface)
}
// raftApplyWithEncoder is used to encode a message, run it through raft,

View File

@ -20,6 +20,7 @@ import (
metrics "github.com/armon/go-metrics"
"github.com/hashicorp/consul/acl"
ca "github.com/hashicorp/consul/agent/connect/ca"
"github.com/hashicorp/consul/agent/consul/authmethod"
"github.com/hashicorp/consul/agent/consul/autopilot"
"github.com/hashicorp/consul/agent/consul/fsm"
"github.com/hashicorp/consul/agent/consul/state"
@ -114,8 +115,7 @@ type Server struct {
// acls is used to resolve tokens to effective policies
acls *ACLResolver
aclAuthMethodValidators map[string]*authMethodValidatorEntry
aclAuthMethodValidatorLock sync.RWMutex
aclAuthMethodValidators authmethod.Cache
// DEPRECATED (ACL-Legacy-Compat) - only needed while we support both
// useNewACLs is used to determine whether we can use new ACLs or not
@ -351,25 +351,26 @@ func NewServerLogger(config *Config, logger *log.Logger, tokens *token.Store, tl
// Create server.
s := &Server{
config: config,
tokens: tokens,
connPool: connPool,
eventChLAN: make(chan serf.Event, serfEventChSize),
eventChWAN: make(chan serf.Event, serfEventChSize),
logger: logger,
leaveCh: make(chan struct{}),
reconcileCh: make(chan serf.Member, reconcileChSize),
router: router.NewRouter(logger, config.Datacenter),
rpcServer: rpc.NewServer(),
insecureRPCServer: rpc.NewServer(),
tlsConfigurator: tlsConfigurator,
reassertLeaderCh: make(chan chan error),
segmentLAN: make(map[string]*serf.Serf, len(config.Segments)),
sessionTimers: NewSessionTimers(),
tombstoneGC: gc,
serverLookup: NewServerLookup(),
shutdownCh: shutdownCh,
leaderRoutineManager: NewLeaderRoutineManager(logger),
config: config,
tokens: tokens,
connPool: connPool,
eventChLAN: make(chan serf.Event, serfEventChSize),
eventChWAN: make(chan serf.Event, serfEventChSize),
logger: logger,
leaveCh: make(chan struct{}),
reconcileCh: make(chan serf.Member, reconcileChSize),
router: router.NewRouter(logger, config.Datacenter),
rpcServer: rpc.NewServer(),
insecureRPCServer: rpc.NewServer(),
tlsConfigurator: tlsConfigurator,
reassertLeaderCh: make(chan chan error),
segmentLAN: make(map[string]*serf.Serf, len(config.Segments)),
sessionTimers: NewSessionTimers(),
tombstoneGC: gc,
serverLookup: NewServerLookup(),
shutdownCh: shutdownCh,
leaderRoutineManager: NewLeaderRoutineManager(logger),
aclAuthMethodValidators: authmethod.NewCache(),
}
// Initialize enterprise specific server functionality

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,501 @@
// +build !consulent
package state
import (
"fmt"
"github.com/hashicorp/consul/agent/structs"
memdb "github.com/hashicorp/go-memdb"
)
///////////////////////////////////////////////////////////////////////////////
///// ACL Table Schemas /////
///////////////////////////////////////////////////////////////////////////////
func tokensTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acl-tokens",
Indexes: map[string]*memdb.IndexSchema{
"accessor": &memdb.IndexSchema{
Name: "accessor",
// DEPRECATED (ACL-Legacy-Compat) - we should not AllowMissing here once legacy compat is removed
AllowMissing: true,
Unique: true,
Indexer: &memdb.UUIDFieldIndex{
Field: "AccessorID",
},
},
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "SecretID",
Lowercase: false,
},
},
"policies": &memdb.IndexSchema{
Name: "policies",
// Need to allow missing for the anonymous token
AllowMissing: true,
Unique: false,
Indexer: &TokenPoliciesIndex{},
},
"roles": &memdb.IndexSchema{
Name: "roles",
AllowMissing: true,
Unique: false,
Indexer: &TokenRolesIndex{},
},
"authmethod": &memdb.IndexSchema{
Name: "authmethod",
AllowMissing: true,
Unique: false,
Indexer: &memdb.StringFieldIndex{
Field: "AuthMethod",
Lowercase: false,
},
},
"local": &memdb.IndexSchema{
Name: "local",
AllowMissing: false,
Unique: false,
Indexer: &memdb.ConditionalIndex{
Conditional: func(obj interface{}) (bool, error) {
if token, ok := obj.(*structs.ACLToken); ok {
return token.Local, nil
}
return false, nil
},
},
},
"expires-global": {
Name: "expires-global",
AllowMissing: true,
Unique: false,
Indexer: &TokenExpirationIndex{LocalFilter: false},
},
"expires-local": {
Name: "expires-local",
AllowMissing: true,
Unique: false,
Indexer: &TokenExpirationIndex{LocalFilter: true},
},
//DEPRECATED (ACL-Legacy-Compat) - This index is only needed while we support upgrading v1 to v2 acls
// This table indexes all the ACL tokens that do not have an AccessorID
"needs-upgrade": &memdb.IndexSchema{
Name: "needs-upgrade",
AllowMissing: false,
Unique: false,
Indexer: &memdb.ConditionalIndex{
Conditional: func(obj interface{}) (bool, error) {
if token, ok := obj.(*structs.ACLToken); ok {
return token.AccessorID == "", nil
}
return false, nil
},
},
},
},
}
}
func policiesTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acl-policies",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.UUIDFieldIndex{
Field: "ID",
},
},
"name": &memdb.IndexSchema{
Name: "name",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "Name",
// TODO (ACL-V2) - should we coerce to lowercase?
Lowercase: true,
},
},
},
}
}
func rolesTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acl-roles",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.UUIDFieldIndex{
Field: "ID",
},
},
"name": &memdb.IndexSchema{
Name: "name",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "Name",
Lowercase: true,
},
},
"policies": &memdb.IndexSchema{
Name: "policies",
// Need to allow missing for the anonymous token
AllowMissing: true,
Unique: false,
Indexer: &RolePoliciesIndex{},
},
},
}
}
func bindingRulesTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acl-binding-rules",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.UUIDFieldIndex{
Field: "ID",
},
},
"authmethod": &memdb.IndexSchema{
Name: "authmethod",
AllowMissing: false,
Unique: false,
Indexer: &memdb.StringFieldIndex{
Field: "AuthMethod",
Lowercase: true,
},
},
},
}
}
func authMethodsTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acl-auth-methods",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "Name",
Lowercase: true,
},
},
},
}
}
///////////////////////////////////////////////////////////////////////////////
///// ACL Policy Functions /////
///////////////////////////////////////////////////////////////////////////////
func (s *Store) aclPolicyInsert(tx *memdb.Txn, policy *structs.ACLPolicy, updateIndexes bool) error {
if err := tx.Insert("acl-policies", policy); err != nil {
return fmt.Errorf("failed inserting acl policy: %v", err)
}
if updateIndexes {
if err := tx.Insert("index", &IndexEntry{"acl-policies", policy.ModifyIndex}); err != nil {
return fmt.Errorf("failed updating acl policies index: %v", err)
}
}
return nil
}
func (s *Store) aclPolicyGetByID(tx *memdb.Txn, id string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-policies", "id", id)
}
func (s *Store) aclPolicyGetByName(tx *memdb.Txn, name string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-policies", "name", name)
}
func (s *Store) aclPolicyList(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-policies", "id")
}
func (s *Store) aclPolicyDeleteWithPolicy(tx *memdb.Txn, policy *structs.ACLPolicy, idx uint64) error {
// remove the policy
if err := tx.Delete("acl-policies", policy); err != nil {
return fmt.Errorf("failed deleting acl policy: %v", err)
}
// update the overall acl-policies index
if err := tx.Insert("index", &IndexEntry{"acl-policies", idx}); err != nil {
return fmt.Errorf("failed updating acl policies index: %v", err)
}
return nil
}
func (s *Store) aclPolicyMaxIndex(tx *memdb.Txn, _ *structs.ACLPolicy, _ *structs.EnterpriseMeta) uint64 {
return maxIndexTxn(tx, "acl-policies")
}
func (s *Store) aclPolicyUpsertValidateEnterprise(*memdb.Txn, *structs.ACLPolicy, *structs.ACLPolicy) error {
return nil
}
func (s *Store) ACLPolicyUpsertValidateEnterprise(*structs.ACLPolicy, *structs.ACLPolicy) error {
return nil
}
///////////////////////////////////////////////////////////////////////////////
///// ACL Token Functions /////
///////////////////////////////////////////////////////////////////////////////
func (s *Store) aclTokenInsert(tx *memdb.Txn, token *structs.ACLToken, updateIndexes bool) error {
// insert the token into memdb
if err := tx.Insert("acl-tokens", token); err != nil {
return fmt.Errorf("failed inserting acl token: %v", err)
}
if updateIndexes {
// update the overall acl-tokens index
if err := tx.Insert("index", &IndexEntry{"acl-tokens", token.ModifyIndex}); err != nil {
return fmt.Errorf("failed updating acl tokens index: %v", err)
}
}
return nil
}
func (s *Store) aclTokenGetFromIndex(tx *memdb.Txn, id string, index string, entMeta *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-tokens", index, id)
}
func (s *Store) aclTokenListAll(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "id")
}
func (s *Store) aclTokenListLocal(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "local", true)
}
func (s *Store) aclTokenListGlobal(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "local", false)
}
func (s *Store) aclTokenListByPolicy(tx *memdb.Txn, policy string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "policies", policy)
}
func (s *Store) aclTokenListByRole(tx *memdb.Txn, role string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "roles", role)
}
func (s *Store) aclTokenListByAuthMethod(tx *memdb.Txn, authMethod string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-tokens", "authmethod", authMethod)
}
func (s *Store) aclTokenDeleteWithToken(tx *memdb.Txn, token *structs.ACLToken, idx uint64) error {
// remove the token
if err := tx.Delete("acl-tokens", token); err != nil {
return fmt.Errorf("failed deleting acl token: %v", err)
}
// update the overall acl-tokens index
if err := tx.Insert("index", &IndexEntry{"acl-tokens", idx}); err != nil {
return fmt.Errorf("failed updating acl tokens index: %v", err)
}
return nil
}
func (s *Store) aclTokenMaxIndex(tx *memdb.Txn, _ *structs.ACLToken, entMeta *structs.EnterpriseMeta) uint64 {
return maxIndexTxn(tx, "acl-tokens")
}
func (s *Store) aclTokenUpsertValidateEnterprise(tx *memdb.Txn, token *structs.ACLToken, existing *structs.ACLToken) error {
return nil
}
func (s *Store) ACLTokenUpsertValidateEnterprise(token *structs.ACLToken, existing *structs.ACLToken) error {
return nil
}
///////////////////////////////////////////////////////////////////////////////
///// ACL Role Functions /////
///////////////////////////////////////////////////////////////////////////////
func (s *Store) aclRoleInsert(tx *memdb.Txn, role *structs.ACLRole, updateIndexes bool) error {
// insert the role into memdb
if err := tx.Insert("acl-roles", role); err != nil {
return fmt.Errorf("failed inserting acl role: %v", err)
}
if updateIndexes {
// update the overall acl-roles index
if err := tx.Insert("index", &IndexEntry{"acl-roles", role.ModifyIndex}); err != nil {
return fmt.Errorf("failed updating acl roles index: %v", err)
}
}
return nil
}
func (s *Store) aclRoleGetByID(tx *memdb.Txn, id string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-roles", "id", id)
}
func (s *Store) aclRoleGetByName(tx *memdb.Txn, name string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-roles", "name", name)
}
func (s *Store) aclRoleList(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-roles", "id")
}
func (s *Store) aclRoleListByPolicy(tx *memdb.Txn, policy string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-roles", "policies", policy)
}
func (s *Store) aclRoleDeleteWithRole(tx *memdb.Txn, role *structs.ACLRole, idx uint64) error {
// remove the role
if err := tx.Delete("acl-roles", role); err != nil {
return fmt.Errorf("failed deleting acl role: %v", err)
}
// update the overall acl-roles index
if err := tx.Insert("index", &IndexEntry{"acl-roles", idx}); err != nil {
return fmt.Errorf("failed updating acl policies index: %v", err)
}
return nil
}
func (s *Store) aclRoleMaxIndex(tx *memdb.Txn, _ *structs.ACLRole, _ *structs.EnterpriseMeta) uint64 {
return maxIndexTxn(tx, "acl-roles")
}
func (s *Store) aclRoleUpsertValidateEnterprise(tx *memdb.Txn, role *structs.ACLRole, existing *structs.ACLRole) error {
return nil
}
func (s *Store) ACLRoleUpsertValidateEnterprise(role *structs.ACLRole, existing *structs.ACLRole) error {
return nil
}
///////////////////////////////////////////////////////////////////////////////
///// ACL Binding Rule Functions /////
///////////////////////////////////////////////////////////////////////////////
func (s *Store) aclBindingRuleInsert(tx *memdb.Txn, rule *structs.ACLBindingRule, updateIndexes bool) error {
// insert the role into memdb
if err := tx.Insert("acl-binding-rules", rule); err != nil {
return fmt.Errorf("failed inserting acl role: %v", err)
}
if updateIndexes {
// update the overall acl-binding-rules index
if err := tx.Insert("index", &IndexEntry{"acl-binding-rules", rule.ModifyIndex}); err != nil {
return fmt.Errorf("failed updating acl binding-rules index: %v", err)
}
}
return nil
}
func (s *Store) aclBindingRuleGetByID(tx *memdb.Txn, id string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-binding-rules", "id", id)
}
func (s *Store) aclBindingRuleList(tx *memdb.Txn, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-binding-rules", "id")
}
func (s *Store) aclBindingRuleListByAuthMethod(tx *memdb.Txn, method string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-binding-rules", "authmethod", method)
}
func (s *Store) aclBindingRuleDeleteWithRule(tx *memdb.Txn, rule *structs.ACLBindingRule, idx uint64) error {
// remove the rule
if err := tx.Delete("acl-binding-rules", rule); err != nil {
return fmt.Errorf("failed deleting acl binding rule: %v", err)
}
// update the overall acl-binding-rules index
if err := tx.Insert("index", &IndexEntry{"acl-binding-rules", idx}); err != nil {
return fmt.Errorf("failed updating acl binding rules index: %v", err)
}
return nil
}
func (s *Store) aclBindingRuleMaxIndex(tx *memdb.Txn, _ *structs.ACLBindingRule, entMeta *structs.EnterpriseMeta) uint64 {
return maxIndexTxn(tx, "acl-binding-rules")
}
func (s *Store) aclBindingRuleUpsertValidateEnterprise(tx *memdb.Txn, rule *structs.ACLBindingRule, existing *structs.ACLBindingRule) error {
return nil
}
func (s *Store) ACLBindingRuleUpsertValidateEnterprise(rule *structs.ACLBindingRule, existing *structs.ACLBindingRule) error {
return nil
}
///////////////////////////////////////////////////////////////////////////////
///// ACL Auth Method Functions /////
///////////////////////////////////////////////////////////////////////////////
func (s *Store) aclAuthMethodInsert(tx *memdb.Txn, method *structs.ACLAuthMethod, updateIndexes bool) error {
// insert the role into memdb
if err := tx.Insert("acl-auth-methods", method); err != nil {
return fmt.Errorf("failed inserting acl role: %v", err)
}
if updateIndexes {
// update the overall acl-auth-methods index
if err := tx.Insert("index", &IndexEntry{"acl-auth-methods", method.ModifyIndex}); err != nil {
return fmt.Errorf("failed updating acl auth methods index: %v", err)
}
}
return nil
}
func (s *Store) aclAuthMethodGetByName(tx *memdb.Txn, method string, _ *structs.EnterpriseMeta) (<-chan struct{}, interface{}, error) {
return tx.FirstWatch("acl-auth-methods", "id", method)
}
func (s *Store) aclAuthMethodList(tx *memdb.Txn, entMeta *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get("acl-auth-methods", "id")
}
func (s *Store) aclAuthMethodDeleteWithMethod(tx *memdb.Txn, method *structs.ACLAuthMethod, idx uint64) error {
// remove the method
if err := tx.Delete("acl-auth-methods", method); err != nil {
return fmt.Errorf("failed deleting acl auth method: %v", err)
}
// update the overall acl-auth-methods index
if err := tx.Insert("index", &IndexEntry{"acl-auth-methods", idx}); err != nil {
return fmt.Errorf("failed updating acl auth methods index: %v", err)
}
return nil
}
func (s *Store) aclAuthMethodMaxIndex(tx *memdb.Txn, _ *structs.ACLAuthMethod, entMeta *structs.EnterpriseMeta) uint64 {
return maxIndexTxn(tx, "acl-auth-methods")
}
func (s *Store) aclAuthMethodUpsertValidateEnterprise(tx *memdb.Txn, method *structs.ACLAuthMethod, existing *structs.ACLAuthMethod) error {
return nil
}
func (s *Store) ACLAuthMethodUpsertValidateEnterprise(method *structs.ACLAuthMethod, existing *structs.ACLAuthMethod) error {
return nil
}

View File

@ -218,7 +218,7 @@ func TestStateStore_ACLBootstrap(t *testing.T) {
require.Equal(t, uint64(3), index)
// Make sure the ACLs are in an expected state.
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
require.Len(t, tokens, 1)
compareTokens(t, token1, tokens[0])
@ -232,7 +232,7 @@ func TestStateStore_ACLBootstrap(t *testing.T) {
err = s.ACLBootstrap(32, index, token2.Clone(), false)
require.NoError(t, err)
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
require.Len(t, tokens, 2)
}
@ -286,7 +286,7 @@ func TestStateStore_ACLToken_SetGet_Legacy(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token.Clone(), true))
idx, rtoken, err := s.ACLTokenGetBySecret(nil, token.SecretID)
idx, rtoken, err := s.ACLTokenGetBySecret(nil, token.SecretID, nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.NotNil(t, rtoken)
@ -319,7 +319,7 @@ func TestStateStore_ACLToken_SetGet_Legacy(t *testing.T) {
require.NoError(t, s.ACLTokenSet(3, update.Clone(), true))
idx, rtoken, err := s.ACLTokenGetBySecret(nil, original.SecretID)
idx, rtoken, err := s.ACLTokenGetBySecret(nil, original.SecretID, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.NotNil(t, rtoken)
@ -498,7 +498,7 @@ func TestStateStore_ACLToken_SetGet(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token.Clone(), false))
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a")
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a", nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
compareTokens(t, token, rtoken)
@ -554,7 +554,7 @@ func TestStateStore_ACLToken_SetGet(t *testing.T) {
require.NoError(t, s.ACLTokenSet(3, updated.Clone(), false))
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a")
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a", nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
compareTokens(t, updated, rtoken)
@ -588,7 +588,7 @@ func TestStateStore_ACLToken_SetGet(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token.Clone(), false))
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a")
idx, rtoken, err := s.ACLTokenGetByAccessor(nil, "daf37c07-d04d-4fd5-9678-a8206a57d61a", nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
compareTokens(t, token, rtoken)
@ -622,7 +622,7 @@ func TestStateStore_ACLTokens_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLTokenBatchSet(2, tokens, true, false, false))
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID)
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID, nil)
require.NoError(t, err)
require.Nil(t, token)
})
@ -654,7 +654,7 @@ func TestStateStore_ACLTokens_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLTokenBatchSet(6, updated, true, false, false))
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID)
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID, nil)
require.NoError(t, err)
require.NotNil(t, token)
require.Equal(t, "", token.Description)
@ -683,7 +683,7 @@ func TestStateStore_ACLTokens_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLTokenBatchSet(6, updated, true, false, false))
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID)
_, token, err := s.ACLTokenGetByAccessor(nil, tokens[0].AccessorID, nil)
require.NoError(t, err)
require.NotNil(t, token)
require.Equal(t, "", token.Description)
@ -1203,7 +1203,7 @@ func TestStateStore_ACLToken_List(t *testing.T) {
{testPolicyID_A, testRoleID_A, ""},
} {
t.Run(fmt.Sprintf("can't filter on more than one: %s/%s/%s", tc.policy, tc.role, tc.methodName), func(t *testing.T) {
_, _, err := s.ACLTokenList(nil, false, false, tc.policy, tc.role, tc.methodName)
_, _, err := s.ACLTokenList(nil, false, false, tc.policy, tc.role, tc.methodName, nil)
require.Error(t, err)
})
}
@ -1212,7 +1212,7 @@ func TestStateStore_ACLToken_List(t *testing.T) {
tc := tc // capture range variable
t.Run(tc.name, func(t *testing.T) {
t.Parallel()
_, tokens, err := s.ACLTokenList(nil, tc.local, tc.global, tc.policy, tc.role, tc.methodName)
_, tokens, err := s.ACLTokenList(nil, tc.local, tc.global, tc.policy, tc.role, tc.methodName, nil)
require.NoError(t, err)
require.Len(t, tokens, len(tc.accessors))
tokens.Sort()
@ -1246,7 +1246,7 @@ func TestStateStore_ACLToken_FixupPolicyLinks(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token, false))
_, retrieved, err := s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err := s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be identical
require.True(t, token == retrieved)
@ -1265,7 +1265,7 @@ func TestStateStore_ACLToken_FixupPolicyLinks(t *testing.T) {
require.NoError(t, s.ACLPolicySet(3, renamed))
// retrieve the token again
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, token != retrieved)
@ -1273,7 +1273,7 @@ func TestStateStore_ACLToken_FixupPolicyLinks(t *testing.T) {
require.Equal(t, "node-read-renamed", retrieved.Policies[0].Name)
// list tokens without stale links
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
found := false
@ -1307,17 +1307,17 @@ func TestStateStore_ACLToken_FixupPolicyLinks(t *testing.T) {
require.True(t, found)
// delete the policy
require.NoError(t, s.ACLPolicyDeleteByID(4, testPolicyID_A))
require.NoError(t, s.ACLPolicyDeleteByID(4, testPolicyID_A, nil))
// retrieve the token again
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, token != retrieved)
require.Len(t, retrieved.Policies, 0)
// list tokens without stale links
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
found = false
@ -1372,7 +1372,7 @@ func TestStateStore_ACLToken_FixupRoleLinks(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token, false))
_, retrieved, err := s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err := s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be identical
require.True(t, token == retrieved)
@ -1394,7 +1394,7 @@ func TestStateStore_ACLToken_FixupRoleLinks(t *testing.T) {
require.NoError(t, s.ACLRoleSet(3, renamed))
// retrieve the token again
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, token != retrieved)
@ -1402,7 +1402,7 @@ func TestStateStore_ACLToken_FixupRoleLinks(t *testing.T) {
require.Equal(t, "node-read-role-renamed", retrieved.Roles[0].Name)
// list tokens without stale links
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err := s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
found := false
@ -1436,17 +1436,17 @@ func TestStateStore_ACLToken_FixupRoleLinks(t *testing.T) {
require.True(t, found)
// delete the role
require.NoError(t, s.ACLRoleDeleteByID(4, testRoleID_A))
require.NoError(t, s.ACLRoleDeleteByID(4, testRoleID_A, nil))
// retrieve the token again
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID)
_, retrieved, err = s.ACLTokenGetByAccessor(nil, token.AccessorID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, token != retrieved)
require.Len(t, retrieved.Roles, 0)
// list tokens without stale links
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "")
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
found = false
@ -1498,13 +1498,13 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token.Clone(), false))
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.NotNil(t, rtoken)
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "f1093997-b6c7-496d-bfb8-6b1b1895641b"))
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil))
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.Nil(t, rtoken)
})
@ -1526,13 +1526,13 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
require.NoError(t, s.ACLTokenSet(2, token.Clone(), false))
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.NotNil(t, rtoken)
require.NoError(t, s.ACLTokenDeleteBySecret(3, "34ec8eb3-095d-417a-a937-b439af7a8e8b"))
require.NoError(t, s.ACLTokenDeleteBySecret(3, "34ec8eb3-095d-417a-a937-b439af7a8e8b", nil))
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.Nil(t, rtoken)
})
@ -1566,10 +1566,10 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
require.NoError(t, s.ACLTokenBatchSet(2, tokens, false, false, false))
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err := s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.NotNil(t, rtoken)
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab")
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab", nil)
require.NoError(t, err)
require.NotNil(t, rtoken)
@ -1577,10 +1577,10 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
"f1093997-b6c7-496d-bfb8-6b1b1895641b",
"a0bfe8d4-b2f3-4b48-b387-f28afb820eab"}))
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.Nil(t, rtoken)
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab")
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab", nil)
require.NoError(t, err)
require.Nil(t, rtoken)
})
@ -1589,8 +1589,8 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
t.Parallel()
s := testACLTokensStateStore(t)
require.Error(t, s.ACLTokenDeleteByAccessor(3, structs.ACLTokenAnonymousID))
require.Error(t, s.ACLTokenDeleteBySecret(3, "anonymous"))
require.Error(t, s.ACLTokenDeleteByAccessor(3, structs.ACLTokenAnonymousID, nil))
require.Error(t, s.ACLTokenDeleteBySecret(3, "anonymous", nil))
})
t.Run("Not Found", func(t *testing.T) {
@ -1598,8 +1598,8 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
s := testACLStateStore(t)
// deletion of non-existent policies is not an error
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "ea58a09c-2100-4aef-816b-8ee0ade77dcd"))
require.NoError(t, s.ACLTokenDeleteBySecret(3, "376d0cae-dd50-4213-9668-2c7797a7fb2d"))
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "ea58a09c-2100-4aef-816b-8ee0ade77dcd", nil))
require.NoError(t, s.ACLTokenDeleteBySecret(3, "376d0cae-dd50-4213-9668-2c7797a7fb2d", nil))
})
}
@ -1675,7 +1675,7 @@ func TestStateStore_ACLPolicy_SetGet(t *testing.T) {
require.NoError(t, s.ACLPolicySet(3, &policy))
_, rpolicy, err := s.ACLPolicyGetByName(nil, "management")
_, rpolicy, err := s.ACLPolicyGetByName(nil, "management", nil)
require.NoError(t, err)
require.NotNil(t, rpolicy)
require.Equal(t, structs.ACLPolicyGlobalManagementID, rpolicy.ID)
@ -1702,7 +1702,7 @@ func TestStateStore_ACLPolicy_SetGet(t *testing.T) {
require.NoError(t, s.ACLPolicySet(3, &policy))
idx, rpolicy, err := s.ACLPolicyGetByID(nil, testPolicyID_A)
idx, rpolicy, err := s.ACLPolicyGetByID(nil, testPolicyID_A, nil)
require.Equal(t, uint64(3), idx)
require.NoError(t, err)
require.NotNil(t, rpolicy)
@ -1716,7 +1716,7 @@ func TestStateStore_ACLPolicy_SetGet(t *testing.T) {
require.Equal(t, uint64(3), rpolicy.ModifyIndex)
// also verify the global management policy that testACLStateStore Set while we are at it.
idx, rpolicy, err = s.ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID)
idx, rpolicy, err = s.ACLPolicyGetByID(nil, structs.ACLPolicyGlobalManagementID, nil)
require.Equal(t, uint64(3), idx)
require.NoError(t, err)
require.NotNil(t, rpolicy)
@ -1750,19 +1750,19 @@ func TestStateStore_ACLPolicy_SetGet(t *testing.T) {
expect.ModifyIndex = 3
// policy found via id
idx, rpolicy, err := s.ACLPolicyGetByID(nil, testPolicyID_A)
idx, rpolicy, err := s.ACLPolicyGetByID(nil, testPolicyID_A, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.Equal(t, expect, rpolicy)
// policy no longer found via old name
idx, rpolicy, err = s.ACLPolicyGetByName(nil, "node-read")
idx, rpolicy, err = s.ACLPolicyGetByName(nil, "node-read", nil)
require.Equal(t, uint64(3), idx)
require.NoError(t, err)
require.Nil(t, rpolicy)
// policy is found via new name
idx, rpolicy, err = s.ACLPolicyGetByName(nil, "node-read-modified")
idx, rpolicy, err = s.ACLPolicyGetByName(nil, "node-read-modified", nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.Equal(t, expect, rpolicy)
@ -1892,7 +1892,7 @@ func TestStateStore_ACLPolicy_List(t *testing.T) {
require.NoError(t, s.ACLPolicyBatchSet(2, policies))
_, policies, err := s.ACLPolicyList(nil)
_, policies, err := s.ACLPolicyList(nil, nil)
require.NoError(t, err)
require.Len(t, policies, 3)
policies.Sort()
@ -1936,14 +1936,14 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
require.NoError(t, s.ACLPolicySet(2, policy))
_, rpolicy, err := s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rpolicy, err := s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.NotNil(t, rpolicy)
require.NoError(t, s.ACLPolicyDeleteByID(3, "f1093997-b6c7-496d-bfb8-6b1b1895641b"))
require.NoError(t, s.ACLPolicyDeleteByID(3, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil))
require.NoError(t, err)
_, rpolicy, err = s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rpolicy, err = s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.Nil(t, rpolicy)
})
@ -1960,14 +1960,14 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
require.NoError(t, s.ACLPolicySet(2, policy))
_, rpolicy, err := s.ACLPolicyGetByName(nil, "test-policy")
_, rpolicy, err := s.ACLPolicyGetByName(nil, "test-policy", nil)
require.NoError(t, err)
require.NotNil(t, rpolicy)
require.NoError(t, s.ACLPolicyDeleteByName(3, "test-policy"))
require.NoError(t, s.ACLPolicyDeleteByName(3, "test-policy", nil))
require.NoError(t, err)
_, rpolicy, err = s.ACLPolicyGetByName(nil, "test-policy")
_, rpolicy, err = s.ACLPolicyGetByName(nil, "test-policy", nil)
require.NoError(t, err)
require.Nil(t, rpolicy)
})
@ -1991,10 +1991,10 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
require.NoError(t, s.ACLPolicyBatchSet(2, policies))
_, rpolicy, err := s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rpolicy, err := s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.NotNil(t, rpolicy)
_, rpolicy, err = s.ACLPolicyGetByID(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab")
_, rpolicy, err = s.ACLPolicyGetByID(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab", nil)
require.NoError(t, err)
require.NotNil(t, rpolicy)
@ -2002,10 +2002,10 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
"f1093997-b6c7-496d-bfb8-6b1b1895641b",
"a0bfe8d4-b2f3-4b48-b387-f28afb820eab"}))
_, rpolicy, err = s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b")
_, rpolicy, err = s.ACLPolicyGetByID(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
require.NoError(t, err)
require.Nil(t, rpolicy)
_, rpolicy, err = s.ACLPolicyGetByID(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab")
_, rpolicy, err = s.ACLPolicyGetByID(nil, "a0bfe8d4-b2f3-4b48-b387-f28afb820eab", nil)
require.NoError(t, err)
require.Nil(t, rpolicy)
})
@ -2014,8 +2014,8 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
t.Parallel()
s := testACLStateStore(t)
require.Error(t, s.ACLPolicyDeleteByID(5, structs.ACLPolicyGlobalManagementID))
require.Error(t, s.ACLPolicyDeleteByName(5, "global-management"))
require.Error(t, s.ACLPolicyDeleteByID(5, structs.ACLPolicyGlobalManagementID, nil))
require.Error(t, s.ACLPolicyDeleteByName(5, "global-management", nil))
})
t.Run("Not Found", func(t *testing.T) {
@ -2023,8 +2023,8 @@ func TestStateStore_ACLPolicy_Delete(t *testing.T) {
s := testACLStateStore(t)
// deletion of non-existent policies is not an error
require.NoError(t, s.ACLPolicyDeleteByName(3, "not-found"))
require.NoError(t, s.ACLPolicyDeleteByID(3, "376d0cae-dd50-4213-9668-2c7797a7fb2d"))
require.NoError(t, s.ACLPolicyDeleteByName(3, "not-found", nil))
require.NoError(t, s.ACLPolicyDeleteByID(3, "376d0cae-dd50-4213-9668-2c7797a7fb2d", nil))
})
}
@ -2162,10 +2162,10 @@ func TestStateStore_ACLRole_SetGet(t *testing.T) {
require.Equal(t, "node-read", rrole.Policies[0].Name)
}
idx, rpolicy, err := s.ACLRoleGetByID(nil, testRoleID_A)
idx, rpolicy, err := s.ACLRoleGetByID(nil, testRoleID_A, nil)
verify(idx, rpolicy, err)
idx, rpolicy, err = s.ACLRoleGetByName(nil, "my-new-role")
idx, rpolicy, err = s.ACLRoleGetByName(nil, "my-new-role", nil)
verify(idx, rpolicy, err)
})
@ -2218,17 +2218,17 @@ func TestStateStore_ACLRole_SetGet(t *testing.T) {
}
// role found via id
idx, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A)
idx, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A, nil)
verify(idx, rrole, err)
// role no longer found via old name
idx, rrole, err = s.ACLRoleGetByName(nil, "node-read-role")
idx, rrole, err = s.ACLRoleGetByName(nil, "node-read-role", nil)
require.Equal(t, uint64(3), idx)
require.NoError(t, err)
require.Nil(t, rrole)
// role is found via new name
idx, rrole, err = s.ACLRoleGetByName(nil, "node-read-role-modified")
idx, rrole, err = s.ACLRoleGetByName(nil, "node-read-role-modified", nil)
verify(idx, rrole, err)
})
}
@ -2461,7 +2461,7 @@ func TestStateStore_ACLRole_List(t *testing.T) {
tc := tc // capture range variable
t.Run(tc.name, func(t *testing.T) {
// t.Parallel()
_, rroles, err := s.ACLRoleList(nil, tc.policy)
_, rroles, err := s.ACLRoleList(nil, tc.policy, nil)
require.NoError(t, err)
require.Len(t, rroles, len(tc.ids))
@ -2515,7 +2515,7 @@ func TestStateStore_ACLRole_FixupPolicyLinks(t *testing.T) {
require.NoError(t, s.ACLRoleSet(2, role))
_, retrieved, err := s.ACLRoleGetByID(nil, role.ID)
_, retrieved, err := s.ACLRoleGetByID(nil, role.ID, nil)
require.NoError(t, err)
// pointer equality check these should be identical
require.True(t, role == retrieved)
@ -2534,7 +2534,7 @@ func TestStateStore_ACLRole_FixupPolicyLinks(t *testing.T) {
require.NoError(t, s.ACLPolicySet(3, renamed))
// retrieve the role again
_, retrieved, err = s.ACLRoleGetByID(nil, role.ID)
_, retrieved, err = s.ACLRoleGetByID(nil, role.ID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, role != retrieved)
@ -2542,7 +2542,7 @@ func TestStateStore_ACLRole_FixupPolicyLinks(t *testing.T) {
require.Equal(t, "node-read-renamed", retrieved.Policies[0].Name)
// list roles without stale links
_, roles, err := s.ACLRoleList(nil, "")
_, roles, err := s.ACLRoleList(nil, "", nil)
require.NoError(t, err)
found := false
@ -2576,17 +2576,17 @@ func TestStateStore_ACLRole_FixupPolicyLinks(t *testing.T) {
require.True(t, found)
// delete the policy
require.NoError(t, s.ACLPolicyDeleteByID(4, testPolicyID_A))
require.NoError(t, s.ACLPolicyDeleteByID(4, testPolicyID_A, nil))
// retrieve the role again
_, retrieved, err = s.ACLRoleGetByID(nil, role.ID)
_, retrieved, err = s.ACLRoleGetByID(nil, role.ID, nil)
require.NoError(t, err)
// pointer equality check these should be different if we cloned things appropriately
require.True(t, role != retrieved)
require.Len(t, retrieved.Policies, 0)
// list roles without stale links
_, roles, err = s.ACLRoleList(nil, "")
_, roles, err = s.ACLRoleList(nil, "", nil)
require.NoError(t, err)
found = false
@ -2638,14 +2638,14 @@ func TestStateStore_ACLRole_Delete(t *testing.T) {
require.NoError(t, s.ACLRoleSet(2, role))
_, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A)
_, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A, nil)
require.NoError(t, err)
require.NotNil(t, rrole)
require.NoError(t, s.ACLRoleDeleteByID(3, testRoleID_A))
require.NoError(t, s.ACLRoleDeleteByID(3, testRoleID_A, nil))
require.NoError(t, err)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_A)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_A, nil)
require.NoError(t, err)
require.Nil(t, rrole)
})
@ -2667,14 +2667,14 @@ func TestStateStore_ACLRole_Delete(t *testing.T) {
require.NoError(t, s.ACLRoleSet(2, role))
_, rrole, err := s.ACLRoleGetByName(nil, "role1")
_, rrole, err := s.ACLRoleGetByName(nil, "role1", nil)
require.NoError(t, err)
require.NotNil(t, rrole)
require.NoError(t, s.ACLRoleDeleteByName(3, "role1"))
require.NoError(t, s.ACLRoleDeleteByName(3, "role1", nil))
require.NoError(t, err)
_, rrole, err = s.ACLRoleGetByName(nil, "role1")
_, rrole, err = s.ACLRoleGetByName(nil, "role1", nil)
require.NoError(t, err)
require.Nil(t, rrole)
})
@ -2708,19 +2708,19 @@ func TestStateStore_ACLRole_Delete(t *testing.T) {
require.NoError(t, s.ACLRoleBatchSet(2, roles, false))
_, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A)
_, rrole, err := s.ACLRoleGetByID(nil, testRoleID_A, nil)
require.NoError(t, err)
require.NotNil(t, rrole)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_B)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_B, nil)
require.NoError(t, err)
require.NotNil(t, rrole)
require.NoError(t, s.ACLRoleBatchDelete(3, []string{testRoleID_A, testRoleID_B}))
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_A)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_A, nil)
require.NoError(t, err)
require.Nil(t, rrole)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_B)
_, rrole, err = s.ACLRoleGetByID(nil, testRoleID_B, nil)
require.NoError(t, err)
require.Nil(t, rrole)
})
@ -2730,8 +2730,8 @@ func TestStateStore_ACLRole_Delete(t *testing.T) {
s := testACLStateStore(t)
// deletion of non-existent roles is not an error
require.NoError(t, s.ACLRoleDeleteByName(3, "not-found"))
require.NoError(t, s.ACLRoleDeleteByID(3, testRoleID_A))
require.NoError(t, s.ACLRoleDeleteByName(3, "not-found", nil))
require.NoError(t, s.ACLRoleDeleteByID(3, testRoleID_A, nil))
})
}
@ -2779,7 +2779,7 @@ func TestStateStore_ACLAuthMethod_SetGet(t *testing.T) {
require.NoError(t, s.ACLAuthMethodSet(3, &method))
idx, rmethod, err := s.ACLAuthMethodGetByName(nil, "test")
idx, rmethod, err := s.ACLAuthMethodGetByName(nil, "test", nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.NotNil(t, rmethod)
@ -2815,7 +2815,7 @@ func TestStateStore_ACLAuthMethod_SetGet(t *testing.T) {
require.NoError(t, s.ACLAuthMethodSet(3, &update))
idx, rmethod, err := s.ACLAuthMethodGetByName(nil, "test")
idx, rmethod, err := s.ACLAuthMethodGetByName(nil, "test", nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.NotNil(t, rmethod)
@ -2850,7 +2850,7 @@ func TestStateStore_ACLAuthMethods_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLAuthMethodBatchSet(2, methods))
idx, rmethods, err := s.ACLAuthMethodList(nil)
idx, rmethods, err := s.ACLAuthMethodList(nil, nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.Len(t, rmethods, 2)
@ -2904,7 +2904,7 @@ func TestStateStore_ACLAuthMethods_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLAuthMethodBatchSet(3, updates))
idx, rmethods, err := s.ACLAuthMethodList(nil)
idx, rmethods, err := s.ACLAuthMethodList(nil, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.Len(t, rmethods, 2)
@ -2936,7 +2936,7 @@ func TestStateStore_ACLAuthMethod_List(t *testing.T) {
require.NoError(t, s.ACLAuthMethodBatchSet(2, methods))
_, rmethods, err := s.ACLAuthMethodList(nil)
_, rmethods, err := s.ACLAuthMethodList(nil, nil)
require.NoError(t, err)
require.Len(t, rmethods, 2)
@ -2970,14 +2970,14 @@ func TestStateStore_ACLAuthMethod_Delete(t *testing.T) {
require.NoError(t, s.ACLAuthMethodSet(2, &method))
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test")
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test", nil)
require.NoError(t, err)
require.NotNil(t, rmethod)
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "test"))
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "test", nil))
require.NoError(t, err)
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test")
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test", nil)
require.NoError(t, err)
require.Nil(t, rmethod)
})
@ -3001,19 +3001,19 @@ func TestStateStore_ACLAuthMethod_Delete(t *testing.T) {
require.NoError(t, s.ACLAuthMethodBatchSet(2, methods))
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test-1")
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test-1", nil)
require.NoError(t, err)
require.NotNil(t, rmethod)
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-2")
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-2", nil)
require.NoError(t, err)
require.NotNil(t, rmethod)
require.NoError(t, s.ACLAuthMethodBatchDelete(3, []string{"test-1", "test-2"}))
require.NoError(t, s.ACLAuthMethodBatchDelete(3, []string{"test-1", "test-2"}, nil))
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-1")
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-1", nil)
require.NoError(t, err)
require.Nil(t, rmethod)
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-2")
_, rmethod, err = s.ACLAuthMethodGetByName(nil, "test-2", nil)
require.NoError(t, err)
require.Nil(t, rmethod)
})
@ -3023,7 +3023,7 @@ func TestStateStore_ACLAuthMethod_Delete(t *testing.T) {
s := testACLStateStore(t)
// deletion of non-existent methods is not an error
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "not-found"))
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "not-found", nil))
})
}
@ -3114,33 +3114,33 @@ func TestStateStore_ACLAuthMethod_Delete_RuleAndTokenCascade(t *testing.T) {
require.NoError(t, s.ACLTokenBatchSet(4, tokens, false, false, false))
// Delete one method.
require.NoError(t, s.ACLAuthMethodDeleteByName(4, "test-1"))
require.NoError(t, s.ACLAuthMethodDeleteByName(4, "test-1", nil))
// Make sure the method is gone.
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test-1")
_, rmethod, err := s.ACLAuthMethodGetByName(nil, "test-1", nil)
require.NoError(t, err)
require.Nil(t, rmethod)
// Make sure the rules and tokens are gone.
for _, ruleID := range []string{method1_rule1, method1_rule2} {
_, rrule, err := s.ACLBindingRuleGetByID(nil, ruleID)
_, rrule, err := s.ACLBindingRuleGetByID(nil, ruleID, nil)
require.NoError(t, err)
require.Nil(t, rrule)
}
for _, tokID := range []string{method1_tok1, method1_tok2} {
_, tok, err := s.ACLTokenGetByAccessor(nil, tokID)
_, tok, err := s.ACLTokenGetByAccessor(nil, tokID, nil)
require.NoError(t, err)
require.Nil(t, tok)
}
// Make sure the rules and tokens for the untouched method are still there.
for _, ruleID := range []string{method2_rule1, method2_rule2} {
_, rrule, err := s.ACLBindingRuleGetByID(nil, ruleID)
_, rrule, err := s.ACLBindingRuleGetByID(nil, ruleID, nil)
require.NoError(t, err)
require.NotNil(t, rrule)
}
for _, tokID := range []string{method2_tok1, method2_tok2} {
_, tok, err := s.ACLTokenGetByAccessor(nil, tokID)
_, tok, err := s.ACLTokenGetByAccessor(nil, tokID, nil)
require.NoError(t, err)
require.NotNil(t, tok)
}
@ -3207,7 +3207,7 @@ func TestStateStore_ACLBindingRule_SetGet(t *testing.T) {
require.NoError(t, s.ACLBindingRuleSet(3, &rule))
idx, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID)
idx, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.NotNil(t, rrule)
@ -3243,7 +3243,7 @@ func TestStateStore_ACLBindingRule_SetGet(t *testing.T) {
require.NoError(t, s.ACLBindingRuleSet(3, &update))
idx, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID)
idx, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.NotNil(t, rrule)
@ -3280,7 +3280,7 @@ func TestStateStore_ACLBindingRules_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLBindingRuleBatchSet(2, rules))
idx, rrules, err := s.ACLBindingRuleList(nil, "test")
idx, rrules, err := s.ACLBindingRuleList(nil, "test", nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.Len(t, rrules, 2)
@ -3333,7 +3333,7 @@ func TestStateStore_ACLBindingRules_UpsertBatchRead(t *testing.T) {
require.NoError(t, s.ACLBindingRuleBatchSet(3, updates))
idx, rrules, err := s.ACLBindingRuleList(nil, "test")
idx, rrules, err := s.ACLBindingRuleList(nil, "test", nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx)
require.Len(t, rrules, 2)
@ -3366,7 +3366,7 @@ func TestStateStore_ACLBindingRule_List(t *testing.T) {
require.NoError(t, s.ACLBindingRuleBatchSet(2, rules))
_, rrules, err := s.ACLBindingRuleList(nil, "")
_, rrules, err := s.ACLBindingRuleList(nil, "", nil)
require.NoError(t, err)
require.Len(t, rrules, 2)
@ -3401,14 +3401,14 @@ func TestStateStore_ACLBindingRule_Delete(t *testing.T) {
require.NoError(t, s.ACLBindingRuleSet(2, &rule))
_, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID)
_, rrule, err := s.ACLBindingRuleGetByID(nil, rule.ID, nil)
require.NoError(t, err)
require.NotNil(t, rrule)
require.NoError(t, s.ACLBindingRuleDeleteByID(3, rule.ID))
require.NoError(t, s.ACLBindingRuleDeleteByID(3, rule.ID, nil))
require.NoError(t, err)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rule.ID)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rule.ID, nil)
require.NoError(t, err)
require.Nil(t, rrule)
})
@ -3433,19 +3433,19 @@ func TestStateStore_ACLBindingRule_Delete(t *testing.T) {
require.NoError(t, s.ACLBindingRuleBatchSet(2, rules))
_, rrule, err := s.ACLBindingRuleGetByID(nil, rules[0].ID)
_, rrule, err := s.ACLBindingRuleGetByID(nil, rules[0].ID, nil)
require.NoError(t, err)
require.NotNil(t, rrule)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[1].ID)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[1].ID, nil)
require.NoError(t, err)
require.NotNil(t, rrule)
require.NoError(t, s.ACLBindingRuleBatchDelete(3, []string{rules[0].ID, rules[1].ID}))
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[0].ID)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[0].ID, nil)
require.NoError(t, err)
require.Nil(t, rrule)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[1].ID)
_, rrule, err = s.ACLBindingRuleGetByID(nil, rules[1].ID, nil)
require.NoError(t, err)
require.Nil(t, rrule)
})
@ -3455,7 +3455,7 @@ func TestStateStore_ACLBindingRule_Delete(t *testing.T) {
s := testACLStateStore(t)
// deletion of non-existent rules is not an error
require.NoError(t, s.ACLBindingRuleDeleteByID(3, "ed3ce1b8-3a16-4e2f-b82e-f92e3b92410d"))
require.NoError(t, s.ACLBindingRuleDeleteByID(3, "ed3ce1b8-3a16-4e2f-b82e-f92e3b92410d", nil))
})
}
@ -3576,7 +3576,7 @@ func TestStateStore_ACLTokens_Snapshot_Restore(t *testing.T) {
defer snap.Close()
// Alter the real state store.
require.NoError(t, s.ACLTokenDeleteByAccessor(3, tokens[0].AccessorID))
require.NoError(t, s.ACLTokenDeleteByAccessor(3, tokens[0].AccessorID, nil))
// Verify the snapshot.
require.Equal(t, uint64(4), snap.LastIndex())
@ -3590,6 +3590,9 @@ func TestStateStore_ACLTokens_Snapshot_Restore(t *testing.T) {
}
require.ElementsMatch(t, dump, tokens)
indexes, err := snapshotIndexes(snap)
require.NoError(t, err)
// Restore the values into a new state store.
func() {
s := testStateStore(t)
@ -3597,6 +3600,7 @@ func TestStateStore_ACLTokens_Snapshot_Restore(t *testing.T) {
for _, token := range dump {
require.NoError(t, restore.ACLToken(token))
}
require.NoError(t, restoreIndexes(indexes, restore))
restore.Commit()
// need to ensure we have the policies or else the links will be removed
@ -3606,7 +3610,7 @@ func TestStateStore_ACLTokens_Snapshot_Restore(t *testing.T) {
require.NoError(t, s.ACLRoleBatchSet(2, roles, false))
// Read the restored ACLs back out and verify that they match.
idx, res, err := s.ACLTokenList(nil, true, true, "", "", "")
idx, res, err := s.ACLTokenList(nil, true, true, "", "", "", nil)
require.NoError(t, err)
require.Equal(t, uint64(4), idx)
require.ElementsMatch(t, tokens, res)
@ -3643,7 +3647,7 @@ func TestStateStore_ACLPolicies_Snapshot_Restore(t *testing.T) {
defer snap.Close()
// Alter the real state store.
require.NoError(t, s.ACLPolicyDeleteByID(3, policies[0].ID))
require.NoError(t, s.ACLPolicyDeleteByID(3, policies[0].ID, nil))
// Verify the snapshot.
require.Equal(t, uint64(2), snap.LastIndex())
@ -3657,6 +3661,9 @@ func TestStateStore_ACLPolicies_Snapshot_Restore(t *testing.T) {
}
require.ElementsMatch(t, dump, policies)
indexes, err := snapshotIndexes(snap)
require.NoError(t, err)
// Restore the values into a new state store.
func() {
s := testStateStore(t)
@ -3664,10 +3671,11 @@ func TestStateStore_ACLPolicies_Snapshot_Restore(t *testing.T) {
for _, policy := range dump {
require.NoError(t, restore.ACLPolicy(policy))
}
require.NoError(t, restoreIndexes(indexes, restore))
restore.Commit()
// Read the restored ACLs back out and verify that they match.
idx, res, err := s.ACLPolicyList(nil)
idx, res, err := s.ACLPolicyList(nil, nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.ElementsMatch(t, policies, res)
@ -3912,7 +3920,7 @@ func TestStateStore_ACLRoles_Snapshot_Restore(t *testing.T) {
defer snap.Close()
// Alter the real state store.
require.NoError(t, s.ACLRoleDeleteByID(3, roles[0].ID))
require.NoError(t, s.ACLRoleDeleteByID(3, roles[0].ID, nil))
// Verify the snapshot.
require.Equal(t, uint64(2), snap.LastIndex())
@ -3926,6 +3934,9 @@ func TestStateStore_ACLRoles_Snapshot_Restore(t *testing.T) {
}
require.ElementsMatch(t, dump, roles)
indexes, err := snapshotIndexes(snap)
require.NoError(t, err)
// Restore the values into a new state store.
func() {
s := testStateStore(t)
@ -3933,13 +3944,14 @@ func TestStateStore_ACLRoles_Snapshot_Restore(t *testing.T) {
for _, role := range dump {
require.NoError(t, restore.ACLRole(role))
}
require.NoError(t, restoreIndexes(indexes, restore))
restore.Commit()
// need to ensure we have the policies or else the links will be removed
require.NoError(t, s.ACLPolicyBatchSet(2, policies))
// Read the restored ACLs back out and verify that they match.
idx, res, err := s.ACLRoleList(nil, "")
idx, res, err := s.ACLRoleList(nil, "", nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.ElementsMatch(t, roles, res)
@ -3972,7 +3984,7 @@ func TestStateStore_ACLAuthMethods_Snapshot_Restore(t *testing.T) {
defer snap.Close()
// Alter the real state store.
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "test-1"))
require.NoError(t, s.ACLAuthMethodDeleteByName(3, "test-1", nil))
// Verify the snapshot.
require.Equal(t, uint64(2), snap.LastIndex())
@ -3986,6 +3998,9 @@ func TestStateStore_ACLAuthMethods_Snapshot_Restore(t *testing.T) {
}
require.ElementsMatch(t, dump, methods)
indexes, err := snapshotIndexes(snap)
require.NoError(t, err)
// Restore the values into a new state store.
func() {
s := testStateStore(t)
@ -3993,10 +4008,11 @@ func TestStateStore_ACLAuthMethods_Snapshot_Restore(t *testing.T) {
for _, method := range dump {
require.NoError(t, restore.ACLAuthMethod(method))
}
require.NoError(t, restoreIndexes(indexes, restore))
restore.Commit()
// Read the restored methods back out and verify that they match.
idx, res, err := s.ACLAuthMethodList(nil)
idx, res, err := s.ACLAuthMethodList(nil, nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.ElementsMatch(t, methods, res)
@ -4030,7 +4046,7 @@ func TestStateStore_ACLBindingRules_Snapshot_Restore(t *testing.T) {
defer snap.Close()
// Alter the real state store.
require.NoError(t, s.ACLBindingRuleDeleteByID(3, rules[0].ID))
require.NoError(t, s.ACLBindingRuleDeleteByID(3, rules[0].ID, nil))
// Verify the snapshot.
require.Equal(t, uint64(2), snap.LastIndex())
@ -4044,6 +4060,9 @@ func TestStateStore_ACLBindingRules_Snapshot_Restore(t *testing.T) {
}
require.ElementsMatch(t, dump, rules)
indexes, err := snapshotIndexes(snap)
require.NoError(t, err)
// Restore the values into a new state store.
func() {
s := testStateStore(t)
@ -4053,10 +4072,11 @@ func TestStateStore_ACLBindingRules_Snapshot_Restore(t *testing.T) {
for _, rule := range dump {
require.NoError(t, restore.ACLBindingRule(rule))
}
require.NoError(t, restoreIndexes(indexes, restore))
restore.Commit()
// Read the restored rules back out and verify that they match.
idx, res, err := s.ACLBindingRuleList(nil, "")
idx, res, err := s.ACLBindingRuleList(nil, "", nil)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.ElementsMatch(t, rules, res)

View File

@ -26,6 +26,27 @@ func testUUID() string {
buf[10:16])
}
func snapshotIndexes(snap *Snapshot) ([]*IndexEntry, error) {
iter, err := snap.Indexes()
if err != nil {
return nil, err
}
var indexes []*IndexEntry
for index := iter.Next(); index != nil; index = iter.Next() {
indexes = append(indexes, index.(*IndexEntry))
}
return indexes, nil
}
func restoreIndexes(indexes []*IndexEntry, r *Restore) error {
for _, index := range indexes {
if err := r.IndexRestore(index); err != nil {
return err
}
}
return nil
}
func testStateStore(t *testing.T) *Store {
s, err := NewStateStore(nil)
if err != nil {

View File

@ -192,7 +192,7 @@ var endpoints map[string]unboundEndpoint
// allowedMethods is a map from endpoint prefix to supported HTTP methods.
// An empty slice means an endpoint handles OPTIONS requests and MethodNotFound errors itself.
var allowedMethods map[string][]string
var allowedMethods map[string][]string = make(map[string][]string)
// registerEndpoint registers a new endpoint, which should be done at package
// init() time.

View File

@ -1,114 +1,12 @@
// +build !consulent
package agent
func init() {
allowedMethods = make(map[string][]string)
import (
"net/http"
registerEndpoint("/v1/acl/bootstrap", []string{"PUT"}, (*HTTPServer).ACLBootstrap)
registerEndpoint("/v1/acl/create", []string{"PUT"}, (*HTTPServer).ACLCreate)
registerEndpoint("/v1/acl/update", []string{"PUT"}, (*HTTPServer).ACLUpdate)
registerEndpoint("/v1/acl/destroy/", []string{"PUT"}, (*HTTPServer).ACLDestroy)
registerEndpoint("/v1/acl/info/", []string{"GET"}, (*HTTPServer).ACLGet)
registerEndpoint("/v1/acl/clone/", []string{"PUT"}, (*HTTPServer).ACLClone)
registerEndpoint("/v1/acl/list", []string{"GET"}, (*HTTPServer).ACLList)
registerEndpoint("/v1/acl/login", []string{"POST"}, (*HTTPServer).ACLLogin)
registerEndpoint("/v1/acl/logout", []string{"POST"}, (*HTTPServer).ACLLogout)
registerEndpoint("/v1/acl/replication", []string{"GET"}, (*HTTPServer).ACLReplicationStatus)
registerEndpoint("/v1/acl/policies", []string{"GET"}, (*HTTPServer).ACLPolicyList)
registerEndpoint("/v1/acl/policy", []string{"PUT"}, (*HTTPServer).ACLPolicyCreate)
registerEndpoint("/v1/acl/policy/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLPolicyCRUD)
registerEndpoint("/v1/acl/roles", []string{"GET"}, (*HTTPServer).ACLRoleList)
registerEndpoint("/v1/acl/role", []string{"PUT"}, (*HTTPServer).ACLRoleCreate)
registerEndpoint("/v1/acl/role/name/", []string{"GET"}, (*HTTPServer).ACLRoleReadByName)
registerEndpoint("/v1/acl/role/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLRoleCRUD)
registerEndpoint("/v1/acl/binding-rules", []string{"GET"}, (*HTTPServer).ACLBindingRuleList)
registerEndpoint("/v1/acl/binding-rule", []string{"PUT"}, (*HTTPServer).ACLBindingRuleCreate)
registerEndpoint("/v1/acl/binding-rule/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLBindingRuleCRUD)
registerEndpoint("/v1/acl/auth-methods", []string{"GET"}, (*HTTPServer).ACLAuthMethodList)
registerEndpoint("/v1/acl/auth-method", []string{"PUT"}, (*HTTPServer).ACLAuthMethodCreate)
registerEndpoint("/v1/acl/auth-method/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLAuthMethodCRUD)
registerEndpoint("/v1/acl/rules/translate", []string{"POST"}, (*HTTPServer).ACLRulesTranslate)
registerEndpoint("/v1/acl/rules/translate/", []string{"GET"}, (*HTTPServer).ACLRulesTranslateLegacyToken)
registerEndpoint("/v1/acl/tokens", []string{"GET"}, (*HTTPServer).ACLTokenList)
registerEndpoint("/v1/acl/token", []string{"PUT"}, (*HTTPServer).ACLTokenCreate)
registerEndpoint("/v1/acl/token/self", []string{"GET"}, (*HTTPServer).ACLTokenSelf)
registerEndpoint("/v1/acl/token/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLTokenCRUD)
registerEndpoint("/v1/agent/token/", []string{"PUT"}, (*HTTPServer).AgentToken)
registerEndpoint("/v1/agent/self", []string{"GET"}, (*HTTPServer).AgentSelf)
registerEndpoint("/v1/agent/host", []string{"GET"}, (*HTTPServer).AgentHost)
registerEndpoint("/v1/agent/maintenance", []string{"PUT"}, (*HTTPServer).AgentNodeMaintenance)
registerEndpoint("/v1/agent/reload", []string{"PUT"}, (*HTTPServer).AgentReload)
registerEndpoint("/v1/agent/monitor", []string{"GET"}, (*HTTPServer).AgentMonitor)
registerEndpoint("/v1/agent/metrics", []string{"GET"}, (*HTTPServer).AgentMetrics)
registerEndpoint("/v1/agent/services", []string{"GET"}, (*HTTPServer).AgentServices)
registerEndpoint("/v1/agent/service/", []string{"GET"}, (*HTTPServer).AgentService)
registerEndpoint("/v1/agent/checks", []string{"GET"}, (*HTTPServer).AgentChecks)
registerEndpoint("/v1/agent/members", []string{"GET"}, (*HTTPServer).AgentMembers)
registerEndpoint("/v1/agent/join/", []string{"PUT"}, (*HTTPServer).AgentJoin)
registerEndpoint("/v1/agent/leave", []string{"PUT"}, (*HTTPServer).AgentLeave)
registerEndpoint("/v1/agent/force-leave/", []string{"PUT"}, (*HTTPServer).AgentForceLeave)
registerEndpoint("/v1/agent/health/service/id/", []string{"GET"}, (*HTTPServer).AgentHealthServiceByID)
registerEndpoint("/v1/agent/health/service/name/", []string{"GET"}, (*HTTPServer).AgentHealthServiceByName)
registerEndpoint("/v1/agent/check/register", []string{"PUT"}, (*HTTPServer).AgentRegisterCheck)
registerEndpoint("/v1/agent/check/deregister/", []string{"PUT"}, (*HTTPServer).AgentDeregisterCheck)
registerEndpoint("/v1/agent/check/pass/", []string{"PUT"}, (*HTTPServer).AgentCheckPass)
registerEndpoint("/v1/agent/check/warn/", []string{"PUT"}, (*HTTPServer).AgentCheckWarn)
registerEndpoint("/v1/agent/check/fail/", []string{"PUT"}, (*HTTPServer).AgentCheckFail)
registerEndpoint("/v1/agent/check/update/", []string{"PUT"}, (*HTTPServer).AgentCheckUpdate)
registerEndpoint("/v1/agent/connect/authorize", []string{"POST"}, (*HTTPServer).AgentConnectAuthorize)
registerEndpoint("/v1/agent/connect/ca/roots", []string{"GET"}, (*HTTPServer).AgentConnectCARoots)
registerEndpoint("/v1/agent/connect/ca/leaf/", []string{"GET"}, (*HTTPServer).AgentConnectCALeafCert)
registerEndpoint("/v1/agent/service/register", []string{"PUT"}, (*HTTPServer).AgentRegisterService)
registerEndpoint("/v1/agent/service/deregister/", []string{"PUT"}, (*HTTPServer).AgentDeregisterService)
registerEndpoint("/v1/agent/service/maintenance/", []string{"PUT"}, (*HTTPServer).AgentServiceMaintenance)
registerEndpoint("/v1/catalog/register", []string{"PUT"}, (*HTTPServer).CatalogRegister)
registerEndpoint("/v1/catalog/connect/", []string{"GET"}, (*HTTPServer).CatalogConnectServiceNodes)
registerEndpoint("/v1/catalog/deregister", []string{"PUT"}, (*HTTPServer).CatalogDeregister)
registerEndpoint("/v1/catalog/datacenters", []string{"GET"}, (*HTTPServer).CatalogDatacenters)
registerEndpoint("/v1/catalog/nodes", []string{"GET"}, (*HTTPServer).CatalogNodes)
registerEndpoint("/v1/catalog/services", []string{"GET"}, (*HTTPServer).CatalogServices)
registerEndpoint("/v1/catalog/service/", []string{"GET"}, (*HTTPServer).CatalogServiceNodes)
registerEndpoint("/v1/catalog/node/", []string{"GET"}, (*HTTPServer).CatalogNodeServices)
registerEndpoint("/v1/config/", []string{"GET", "DELETE"}, (*HTTPServer).Config)
registerEndpoint("/v1/config", []string{"PUT"}, (*HTTPServer).ConfigApply)
registerEndpoint("/v1/connect/ca/configuration", []string{"GET", "PUT"}, (*HTTPServer).ConnectCAConfiguration)
registerEndpoint("/v1/connect/ca/roots", []string{"GET"}, (*HTTPServer).ConnectCARoots)
registerEndpoint("/v1/connect/intentions", []string{"GET", "POST"}, (*HTTPServer).IntentionEndpoint)
registerEndpoint("/v1/connect/intentions/match", []string{"GET"}, (*HTTPServer).IntentionMatch)
registerEndpoint("/v1/connect/intentions/check", []string{"GET"}, (*HTTPServer).IntentionCheck)
registerEndpoint("/v1/connect/intentions/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).IntentionSpecific)
registerEndpoint("/v1/coordinate/datacenters", []string{"GET"}, (*HTTPServer).CoordinateDatacenters)
registerEndpoint("/v1/coordinate/nodes", []string{"GET"}, (*HTTPServer).CoordinateNodes)
registerEndpoint("/v1/coordinate/node/", []string{"GET"}, (*HTTPServer).CoordinateNode)
registerEndpoint("/v1/coordinate/update", []string{"PUT"}, (*HTTPServer).CoordinateUpdate)
registerEndpoint("/v1/discovery-chain/", []string{"GET", "POST"}, (*HTTPServer).DiscoveryChainRead)
registerEndpoint("/v1/event/fire/", []string{"PUT"}, (*HTTPServer).EventFire)
registerEndpoint("/v1/event/list", []string{"GET"}, (*HTTPServer).EventList)
registerEndpoint("/v1/health/node/", []string{"GET"}, (*HTTPServer).HealthNodeChecks)
registerEndpoint("/v1/health/checks/", []string{"GET"}, (*HTTPServer).HealthServiceChecks)
registerEndpoint("/v1/health/state/", []string{"GET"}, (*HTTPServer).HealthChecksInState)
registerEndpoint("/v1/health/service/", []string{"GET"}, (*HTTPServer).HealthServiceNodes)
registerEndpoint("/v1/health/connect/", []string{"GET"}, (*HTTPServer).HealthConnectServiceNodes)
registerEndpoint("/v1/internal/ui/nodes", []string{"GET"}, (*HTTPServer).UINodes)
registerEndpoint("/v1/internal/ui/node/", []string{"GET"}, (*HTTPServer).UINodeInfo)
registerEndpoint("/v1/internal/ui/services", []string{"GET"}, (*HTTPServer).UIServices)
registerEndpoint("/v1/kv/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).KVSEndpoint)
registerEndpoint("/v1/operator/raft/configuration", []string{"GET"}, (*HTTPServer).OperatorRaftConfiguration)
registerEndpoint("/v1/operator/raft/peer", []string{"DELETE"}, (*HTTPServer).OperatorRaftPeer)
registerEndpoint("/v1/operator/keyring", []string{"GET", "POST", "PUT", "DELETE"}, (*HTTPServer).OperatorKeyringEndpoint)
registerEndpoint("/v1/operator/autopilot/configuration", []string{"GET", "PUT"}, (*HTTPServer).OperatorAutopilotConfiguration)
registerEndpoint("/v1/operator/autopilot/health", []string{"GET"}, (*HTTPServer).OperatorServerHealth)
registerEndpoint("/v1/query", []string{"GET", "POST"}, (*HTTPServer).PreparedQueryGeneral)
// specific prepared query endpoints have more complex rules for allowed methods, so
// the prefix is registered with no methods.
registerEndpoint("/v1/query/", []string{}, (*HTTPServer).PreparedQuerySpecific)
registerEndpoint("/v1/session/create", []string{"PUT"}, (*HTTPServer).SessionCreate)
registerEndpoint("/v1/session/destroy/", []string{"PUT"}, (*HTTPServer).SessionDestroy)
registerEndpoint("/v1/session/renew/", []string{"PUT"}, (*HTTPServer).SessionRenew)
registerEndpoint("/v1/session/info/", []string{"GET"}, (*HTTPServer).SessionGet)
registerEndpoint("/v1/session/node/", []string{"GET"}, (*HTTPServer).SessionsForNode)
registerEndpoint("/v1/session/list", []string{"GET"}, (*HTTPServer).SessionList)
registerEndpoint("/v1/status/leader", []string{"GET"}, (*HTTPServer).StatusLeader)
registerEndpoint("/v1/status/peers", []string{"GET"}, (*HTTPServer).StatusPeers)
registerEndpoint("/v1/snapshot", []string{"GET", "PUT"}, (*HTTPServer).Snapshot)
registerEndpoint("/v1/txn", []string{"PUT"}, (*HTTPServer).Txn)
"github.com/hashicorp/consul/agent/structs"
)
func (s *HTTPServer) parseEntMeta(req *http.Request, entMeta *structs.EnterpriseMeta) {
}

112
agent/http_register.go Normal file
View File

@ -0,0 +1,112 @@
package agent
func init() {
registerEndpoint("/v1/acl/bootstrap", []string{"PUT"}, (*HTTPServer).ACLBootstrap)
registerEndpoint("/v1/acl/create", []string{"PUT"}, (*HTTPServer).ACLCreate)
registerEndpoint("/v1/acl/update", []string{"PUT"}, (*HTTPServer).ACLUpdate)
registerEndpoint("/v1/acl/destroy/", []string{"PUT"}, (*HTTPServer).ACLDestroy)
registerEndpoint("/v1/acl/info/", []string{"GET"}, (*HTTPServer).ACLGet)
registerEndpoint("/v1/acl/clone/", []string{"PUT"}, (*HTTPServer).ACLClone)
registerEndpoint("/v1/acl/list", []string{"GET"}, (*HTTPServer).ACLList)
registerEndpoint("/v1/acl/login", []string{"POST"}, (*HTTPServer).ACLLogin)
registerEndpoint("/v1/acl/logout", []string{"POST"}, (*HTTPServer).ACLLogout)
registerEndpoint("/v1/acl/replication", []string{"GET"}, (*HTTPServer).ACLReplicationStatus)
registerEndpoint("/v1/acl/policies", []string{"GET"}, (*HTTPServer).ACLPolicyList)
registerEndpoint("/v1/acl/policy", []string{"PUT"}, (*HTTPServer).ACLPolicyCreate)
registerEndpoint("/v1/acl/policy/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLPolicyCRUD)
registerEndpoint("/v1/acl/roles", []string{"GET"}, (*HTTPServer).ACLRoleList)
registerEndpoint("/v1/acl/role", []string{"PUT"}, (*HTTPServer).ACLRoleCreate)
registerEndpoint("/v1/acl/role/name/", []string{"GET"}, (*HTTPServer).ACLRoleReadByName)
registerEndpoint("/v1/acl/role/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLRoleCRUD)
registerEndpoint("/v1/acl/binding-rules", []string{"GET"}, (*HTTPServer).ACLBindingRuleList)
registerEndpoint("/v1/acl/binding-rule", []string{"PUT"}, (*HTTPServer).ACLBindingRuleCreate)
registerEndpoint("/v1/acl/binding-rule/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLBindingRuleCRUD)
registerEndpoint("/v1/acl/auth-methods", []string{"GET"}, (*HTTPServer).ACLAuthMethodList)
registerEndpoint("/v1/acl/auth-method", []string{"PUT"}, (*HTTPServer).ACLAuthMethodCreate)
registerEndpoint("/v1/acl/auth-method/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLAuthMethodCRUD)
registerEndpoint("/v1/acl/rules/translate", []string{"POST"}, (*HTTPServer).ACLRulesTranslate)
registerEndpoint("/v1/acl/rules/translate/", []string{"GET"}, (*HTTPServer).ACLRulesTranslateLegacyToken)
registerEndpoint("/v1/acl/tokens", []string{"GET"}, (*HTTPServer).ACLTokenList)
registerEndpoint("/v1/acl/token", []string{"PUT"}, (*HTTPServer).ACLTokenCreate)
registerEndpoint("/v1/acl/token/self", []string{"GET"}, (*HTTPServer).ACLTokenSelf)
registerEndpoint("/v1/acl/token/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).ACLTokenCRUD)
registerEndpoint("/v1/agent/token/", []string{"PUT"}, (*HTTPServer).AgentToken)
registerEndpoint("/v1/agent/self", []string{"GET"}, (*HTTPServer).AgentSelf)
registerEndpoint("/v1/agent/host", []string{"GET"}, (*HTTPServer).AgentHost)
registerEndpoint("/v1/agent/maintenance", []string{"PUT"}, (*HTTPServer).AgentNodeMaintenance)
registerEndpoint("/v1/agent/reload", []string{"PUT"}, (*HTTPServer).AgentReload)
registerEndpoint("/v1/agent/monitor", []string{"GET"}, (*HTTPServer).AgentMonitor)
registerEndpoint("/v1/agent/metrics", []string{"GET"}, (*HTTPServer).AgentMetrics)
registerEndpoint("/v1/agent/services", []string{"GET"}, (*HTTPServer).AgentServices)
registerEndpoint("/v1/agent/service/", []string{"GET"}, (*HTTPServer).AgentService)
registerEndpoint("/v1/agent/checks", []string{"GET"}, (*HTTPServer).AgentChecks)
registerEndpoint("/v1/agent/members", []string{"GET"}, (*HTTPServer).AgentMembers)
registerEndpoint("/v1/agent/join/", []string{"PUT"}, (*HTTPServer).AgentJoin)
registerEndpoint("/v1/agent/leave", []string{"PUT"}, (*HTTPServer).AgentLeave)
registerEndpoint("/v1/agent/force-leave/", []string{"PUT"}, (*HTTPServer).AgentForceLeave)
registerEndpoint("/v1/agent/health/service/id/", []string{"GET"}, (*HTTPServer).AgentHealthServiceByID)
registerEndpoint("/v1/agent/health/service/name/", []string{"GET"}, (*HTTPServer).AgentHealthServiceByName)
registerEndpoint("/v1/agent/check/register", []string{"PUT"}, (*HTTPServer).AgentRegisterCheck)
registerEndpoint("/v1/agent/check/deregister/", []string{"PUT"}, (*HTTPServer).AgentDeregisterCheck)
registerEndpoint("/v1/agent/check/pass/", []string{"PUT"}, (*HTTPServer).AgentCheckPass)
registerEndpoint("/v1/agent/check/warn/", []string{"PUT"}, (*HTTPServer).AgentCheckWarn)
registerEndpoint("/v1/agent/check/fail/", []string{"PUT"}, (*HTTPServer).AgentCheckFail)
registerEndpoint("/v1/agent/check/update/", []string{"PUT"}, (*HTTPServer).AgentCheckUpdate)
registerEndpoint("/v1/agent/connect/authorize", []string{"POST"}, (*HTTPServer).AgentConnectAuthorize)
registerEndpoint("/v1/agent/connect/ca/roots", []string{"GET"}, (*HTTPServer).AgentConnectCARoots)
registerEndpoint("/v1/agent/connect/ca/leaf/", []string{"GET"}, (*HTTPServer).AgentConnectCALeafCert)
registerEndpoint("/v1/agent/service/register", []string{"PUT"}, (*HTTPServer).AgentRegisterService)
registerEndpoint("/v1/agent/service/deregister/", []string{"PUT"}, (*HTTPServer).AgentDeregisterService)
registerEndpoint("/v1/agent/service/maintenance/", []string{"PUT"}, (*HTTPServer).AgentServiceMaintenance)
registerEndpoint("/v1/catalog/register", []string{"PUT"}, (*HTTPServer).CatalogRegister)
registerEndpoint("/v1/catalog/connect/", []string{"GET"}, (*HTTPServer).CatalogConnectServiceNodes)
registerEndpoint("/v1/catalog/deregister", []string{"PUT"}, (*HTTPServer).CatalogDeregister)
registerEndpoint("/v1/catalog/datacenters", []string{"GET"}, (*HTTPServer).CatalogDatacenters)
registerEndpoint("/v1/catalog/nodes", []string{"GET"}, (*HTTPServer).CatalogNodes)
registerEndpoint("/v1/catalog/services", []string{"GET"}, (*HTTPServer).CatalogServices)
registerEndpoint("/v1/catalog/service/", []string{"GET"}, (*HTTPServer).CatalogServiceNodes)
registerEndpoint("/v1/catalog/node/", []string{"GET"}, (*HTTPServer).CatalogNodeServices)
registerEndpoint("/v1/config/", []string{"GET", "DELETE"}, (*HTTPServer).Config)
registerEndpoint("/v1/config", []string{"PUT"}, (*HTTPServer).ConfigApply)
registerEndpoint("/v1/connect/ca/configuration", []string{"GET", "PUT"}, (*HTTPServer).ConnectCAConfiguration)
registerEndpoint("/v1/connect/ca/roots", []string{"GET"}, (*HTTPServer).ConnectCARoots)
registerEndpoint("/v1/connect/intentions", []string{"GET", "POST"}, (*HTTPServer).IntentionEndpoint)
registerEndpoint("/v1/connect/intentions/match", []string{"GET"}, (*HTTPServer).IntentionMatch)
registerEndpoint("/v1/connect/intentions/check", []string{"GET"}, (*HTTPServer).IntentionCheck)
registerEndpoint("/v1/connect/intentions/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).IntentionSpecific)
registerEndpoint("/v1/coordinate/datacenters", []string{"GET"}, (*HTTPServer).CoordinateDatacenters)
registerEndpoint("/v1/coordinate/nodes", []string{"GET"}, (*HTTPServer).CoordinateNodes)
registerEndpoint("/v1/coordinate/node/", []string{"GET"}, (*HTTPServer).CoordinateNode)
registerEndpoint("/v1/coordinate/update", []string{"PUT"}, (*HTTPServer).CoordinateUpdate)
registerEndpoint("/v1/discovery-chain/", []string{"GET", "POST"}, (*HTTPServer).DiscoveryChainRead)
registerEndpoint("/v1/event/fire/", []string{"PUT"}, (*HTTPServer).EventFire)
registerEndpoint("/v1/event/list", []string{"GET"}, (*HTTPServer).EventList)
registerEndpoint("/v1/health/node/", []string{"GET"}, (*HTTPServer).HealthNodeChecks)
registerEndpoint("/v1/health/checks/", []string{"GET"}, (*HTTPServer).HealthServiceChecks)
registerEndpoint("/v1/health/state/", []string{"GET"}, (*HTTPServer).HealthChecksInState)
registerEndpoint("/v1/health/service/", []string{"GET"}, (*HTTPServer).HealthServiceNodes)
registerEndpoint("/v1/health/connect/", []string{"GET"}, (*HTTPServer).HealthConnectServiceNodes)
registerEndpoint("/v1/internal/ui/nodes", []string{"GET"}, (*HTTPServer).UINodes)
registerEndpoint("/v1/internal/ui/node/", []string{"GET"}, (*HTTPServer).UINodeInfo)
registerEndpoint("/v1/internal/ui/services", []string{"GET"}, (*HTTPServer).UIServices)
registerEndpoint("/v1/kv/", []string{"GET", "PUT", "DELETE"}, (*HTTPServer).KVSEndpoint)
registerEndpoint("/v1/operator/raft/configuration", []string{"GET"}, (*HTTPServer).OperatorRaftConfiguration)
registerEndpoint("/v1/operator/raft/peer", []string{"DELETE"}, (*HTTPServer).OperatorRaftPeer)
registerEndpoint("/v1/operator/keyring", []string{"GET", "POST", "PUT", "DELETE"}, (*HTTPServer).OperatorKeyringEndpoint)
registerEndpoint("/v1/operator/autopilot/configuration", []string{"GET", "PUT"}, (*HTTPServer).OperatorAutopilotConfiguration)
registerEndpoint("/v1/operator/autopilot/health", []string{"GET"}, (*HTTPServer).OperatorServerHealth)
registerEndpoint("/v1/query", []string{"GET", "POST"}, (*HTTPServer).PreparedQueryGeneral)
// specific prepared query endpoints have more complex rules for allowed methods, so
// the prefix is registered with no methods.
registerEndpoint("/v1/query/", []string{}, (*HTTPServer).PreparedQuerySpecific)
registerEndpoint("/v1/session/create", []string{"PUT"}, (*HTTPServer).SessionCreate)
registerEndpoint("/v1/session/destroy/", []string{"PUT"}, (*HTTPServer).SessionDestroy)
registerEndpoint("/v1/session/renew/", []string{"PUT"}, (*HTTPServer).SessionRenew)
registerEndpoint("/v1/session/info/", []string{"GET"}, (*HTTPServer).SessionGet)
registerEndpoint("/v1/session/node/", []string{"GET"}, (*HTTPServer).SessionsForNode)
registerEndpoint("/v1/session/list", []string{"GET"}, (*HTTPServer).SessionList)
registerEndpoint("/v1/status/leader", []string{"GET"}, (*HTTPServer).StatusLeader)
registerEndpoint("/v1/status/peers", []string{"GET"}, (*HTTPServer).StatusPeers)
registerEndpoint("/v1/snapshot", []string{"GET", "PUT"}, (*HTTPServer).Snapshot)
registerEndpoint("/v1/txn", []string{"PUT"}, (*HTTPServer).Txn)
}

View File

@ -86,22 +86,6 @@ session_prefix "" {
ACLTokenAnonymousID = "00000000-0000-0000-0000-000000000002"
ACLReservedPrefix = "00000000-0000-0000-0000-0000000000"
// aclPolicyTemplateServiceIdentity is the template used for synthesizing
// policies for service identities.
aclPolicyTemplateServiceIdentity = `
service "%s" {
policy = "write"
}
service "%s-sidecar-proxy" {
policy = "write"
}
service_prefix "" {
policy = "read"
}
node_prefix "" {
policy = "read"
}`
)
func ACLIDReserved(id string) bool {
@ -134,6 +118,7 @@ type ACLIdentity interface {
EmbeddedPolicy() *ACLPolicy
ServiceIdentityList() []*ACLServiceIdentity
IsExpired(asOf time.Time) bool
EnterpriseMetadata() *EnterpriseMeta
}
type ACLTokenPolicyLink struct {
@ -181,10 +166,11 @@ func (s *ACLServiceIdentity) EstimateSize() int {
return size
}
func (s *ACLServiceIdentity) SyntheticPolicy() *ACLPolicy {
func (s *ACLServiceIdentity) SyntheticPolicy(entMeta *EnterpriseMeta) *ACLPolicy {
// Given that we validate this string name before persisting, we do not
// have to escape it before doing the following interpolation.
rules := fmt.Sprintf(aclPolicyTemplateServiceIdentity, s.ServiceName, s.ServiceName)
// TODO (namespaces) include namespace
rules := aclServiceIdentityRules(s.ServiceName, entMeta)
hasher := fnv.New128a()
hashID := fmt.Sprintf("%x", hasher.Sum([]byte(rules)))
@ -268,6 +254,9 @@ type ACLToken struct {
// unnecessary calls to the authoritative DC
Hash []byte
// Embedded Enterprise Metadata
EnterpriseMeta `mapstructure:",squash"`
// Embedded Raft Metadata
RaftIndex
}
@ -390,6 +379,10 @@ func (t *ACLToken) EmbeddedPolicy() *ACLPolicy {
return policy
}
func (t *ACLToken) EnterpriseMetadata() *EnterpriseMeta {
return &t.EnterpriseMeta
}
func (t *ACLToken) SetHash(force bool) []byte {
if force || t.Hash == nil {
// Initialize a 256bit Blake2 hash (32 bytes)
@ -429,6 +422,8 @@ func (t *ACLToken) SetHash(force bool) []byte {
srvid.AddToHash(hash)
}
t.EnterpriseMeta.addToHash(hash)
// Finalize the hash
hashVal := hash.Sum(nil)
@ -450,7 +445,7 @@ func (t *ACLToken) EstimateSize() int {
for _, srvid := range t.ServiceIdentities {
size += srvid.EstimateSize()
}
return size
return size + t.EnterpriseMeta.estimateSize()
}
// ACLTokens is a slice of ACLTokens.
@ -470,6 +465,7 @@ type ACLTokenListStub struct {
CreateIndex uint64
ModifyIndex uint64
Legacy bool `json:",omitempty"`
EnterpriseMeta
}
type ACLTokenListStubs []*ACLTokenListStub
@ -489,6 +485,7 @@ func (token *ACLToken) Stub() *ACLTokenListStub {
CreateIndex: token.CreateIndex,
ModifyIndex: token.ModifyIndex,
Legacy: token.Rules != "",
EnterpriseMeta: token.EnterpriseMeta,
}
}
@ -536,6 +533,9 @@ type ACLPolicy struct {
// unnecessary calls to the authoritative DC
Hash []byte
// Embedded Enterprise ACL Metadata
EnterpriseMeta `mapstructure:",squash"`
// Embedded Raft Metadata
RaftIndex `hash:"ignore"`
}
@ -554,17 +554,19 @@ type ACLPolicyListStub struct {
Hash []byte
CreateIndex uint64
ModifyIndex uint64
EnterpriseMeta
}
func (p *ACLPolicy) Stub() *ACLPolicyListStub {
return &ACLPolicyListStub{
ID: p.ID,
Name: p.Name,
Description: p.Description,
Datacenters: p.Datacenters,
Hash: p.Hash,
CreateIndex: p.CreateIndex,
ModifyIndex: p.ModifyIndex,
ID: p.ID,
Name: p.Name,
Description: p.Description,
Datacenters: p.Datacenters,
Hash: p.Hash,
CreateIndex: p.CreateIndex,
ModifyIndex: p.ModifyIndex,
EnterpriseMeta: p.EnterpriseMeta,
}
}
@ -595,6 +597,8 @@ func (p *ACLPolicy) SetHash(force bool) []byte {
hash.Write([]byte(dc))
}
p.EnterpriseMeta.addToHash(hash)
// Finalize the hash
hashVal := hash.Sum(nil)
@ -614,7 +618,7 @@ func (p *ACLPolicy) EstimateSize() int {
size += len(dc)
}
return size
return size + p.EnterpriseMeta.estimateSize()
}
// HashKey returns a consistent hash for a set of policies.
@ -657,7 +661,7 @@ func (policies ACLPolicies) resolveWithCache(cache *ACLCaches, entConf *acl.Ente
continue
}
p, err := acl.NewPolicyFromSource(policy.ID, policy.ModifyIndex, policy.Rules, policy.Syntax, entConf)
p, err := acl.NewPolicyFromSource(policy.ID, policy.ModifyIndex, policy.Rules, policy.Syntax, entConf, policy.EnterprisePolicyMeta())
if err != nil {
return nil, fmt.Errorf("failed to parse %q: %v", policy.Name, err)
}
@ -758,6 +762,9 @@ type ACLRole struct {
// unnecessary calls to the authoritative DC
Hash []byte
// Embedded Enterprise ACL metadata
EnterpriseMeta `mapstructure:",squash"`
// Embedded Raft Metadata
RaftIndex `hash:"ignore"`
}
@ -806,6 +813,8 @@ func (r *ACLRole) SetHash(force bool) []byte {
srvid.AddToHash(hash)
}
r.EnterpriseMeta.addToHash(hash)
// Finalize the hash
hashVal := hash.Sum(nil)
@ -828,7 +837,7 @@ func (r *ACLRole) EstimateSize() int {
size += srvid.EstimateSize()
}
return size
return size + r.EnterpriseMeta.estimateSize()
}
const (
@ -888,6 +897,9 @@ type ACLBindingRule struct {
// upon the BindType.
BindName string
// Embedded Enterprise ACL metadata
EnterpriseMeta `mapstructure:",squash"`
// Embedded Raft Metadata
RaftIndex `hash:"ignore"`
}
@ -911,15 +923,17 @@ type ACLAuthMethodListStub struct {
Type string
CreateIndex uint64
ModifyIndex uint64
EnterpriseMeta
}
func (p *ACLAuthMethod) Stub() *ACLAuthMethodListStub {
return &ACLAuthMethodListStub{
Name: p.Name,
Description: p.Description,
Type: p.Type,
CreateIndex: p.CreateIndex,
ModifyIndex: p.ModifyIndex,
Name: p.Name,
Description: p.Description,
Type: p.Type,
CreateIndex: p.CreateIndex,
ModifyIndex: p.ModifyIndex,
EnterpriseMeta: p.EnterpriseMeta,
}
}
@ -957,6 +971,9 @@ type ACLAuthMethod struct {
// maps).
Config map[string]interface{}
// Embedded Enterprise ACL Meta
EnterpriseMeta `mapstructure:",squash"`
// Embedded Raft Metadata
RaftIndex `hash:"ignore"`
}
@ -1017,6 +1034,7 @@ type ACLTokenGetRequest struct {
TokenID string // id used for the token lookup
TokenIDType ACLTokenIDType // The Type of ID used to lookup the token
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1028,6 +1046,7 @@ func (r *ACLTokenGetRequest) RequestDatacenter() string {
type ACLTokenDeleteRequest struct {
TokenID string // ID of the token to delete
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
WriteRequest
}
@ -1043,6 +1062,7 @@ type ACLTokenListRequest struct {
Role string // Role filter
AuthMethod string // Auth Method filter
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1063,6 +1083,8 @@ type ACLTokenListResponse struct {
type ACLTokenBatchGetRequest struct {
AccessorIDs []string // List of accessor ids to fetch
Datacenter string // The datacenter to perform the request within
// TODO (namespaces) should this use enterprise meta? - it would be hard to
// update in a backwards compatible manner an accessor ids should be unique
QueryOptions
}
@ -1089,6 +1111,7 @@ type ACLTokenBatchSetRequest struct {
// multiple tokens need to be removed from the local DCs state.
type ACLTokenBatchDeleteRequest struct {
TokenIDs []string // Tokens to delete
// TODO (namespaces) should we update with ent meta?
}
// ACLTokenBootstrapRequest is used only at the Raft layer
@ -1113,6 +1136,7 @@ type ACLTokenResponse struct {
type ACLTokenBatchResponse struct {
Tokens []*ACLToken
Redacted bool // whether the token secrets were redacted.
Removed bool // whether any tokens were completely removed
QueryMeta
}
@ -1131,6 +1155,7 @@ func (r *ACLPolicySetRequest) RequestDatacenter() string {
type ACLPolicyDeleteRequest struct {
PolicyID string // The id of the policy to delete
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
WriteRequest
}
@ -1142,6 +1167,7 @@ func (r *ACLPolicyDeleteRequest) RequestDatacenter() string {
type ACLPolicyGetRequest struct {
PolicyID string // id used for the policy lookup
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1152,6 +1178,7 @@ func (r *ACLPolicyGetRequest) RequestDatacenter() string {
// ACLPolicyListRequest is used at the RPC layer to request a listing of policies
type ACLPolicyListRequest struct {
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1227,6 +1254,7 @@ func (r *ACLRoleSetRequest) RequestDatacenter() string {
type ACLRoleDeleteRequest struct {
RoleID string // id of the role to delete
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
WriteRequest
}
@ -1239,6 +1267,7 @@ type ACLRoleGetRequest struct {
RoleID string // id used for the role lookup (one of RoleID or RoleName is allowed)
RoleName string // name used for the role lookup (one of RoleID or RoleName is allowed)
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1250,6 +1279,7 @@ func (r *ACLRoleGetRequest) RequestDatacenter() string {
type ACLRoleListRequest struct {
Policy string // Policy filter
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1317,6 +1347,7 @@ func (r *ACLBindingRuleSetRequest) RequestDatacenter() string {
type ACLBindingRuleDeleteRequest struct {
BindingRuleID string // id of the rule to delete
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
WriteRequest
}
@ -1328,6 +1359,7 @@ func (r *ACLBindingRuleDeleteRequest) RequestDatacenter() string {
type ACLBindingRuleGetRequest struct {
BindingRuleID string // id used for the rule lookup
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1339,6 +1371,7 @@ func (r *ACLBindingRuleGetRequest) RequestDatacenter() string {
type ACLBindingRuleListRequest struct {
AuthMethod string // optional filter
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1384,6 +1417,7 @@ func (r *ACLAuthMethodSetRequest) RequestDatacenter() string {
type ACLAuthMethodDeleteRequest struct {
AuthMethodName string // name of the auth method to delete
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
WriteRequest
}
@ -1395,6 +1429,7 @@ func (r *ACLAuthMethodDeleteRequest) RequestDatacenter() string {
type ACLAuthMethodGetRequest struct {
AuthMethodName string // name used for the auth method lookup
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1405,6 +1440,7 @@ func (r *ACLAuthMethodGetRequest) RequestDatacenter() string {
// ACLAuthMethodListRequest is used at the RPC layer to request a listing of auth methods
type ACLAuthMethodListRequest struct {
Datacenter string // The datacenter to perform the request within
EnterpriseMeta
QueryOptions
}
@ -1433,12 +1469,19 @@ type ACLAuthMethodBatchSetRequest struct {
// multiple auth method deletions
type ACLAuthMethodBatchDeleteRequest struct {
AuthMethodNames []string
// While it may seem odd that AuthMethodNames is associated with a single
// EnterpriseMeta, it is okay as this struct is only ever used to
// delete a single entry. This is because AuthMethods unlike tokens, policies
// and roles are not replicated between datacenters and therefore never
// batch applied.
EnterpriseMeta
}
type ACLLoginParams struct {
AuthMethod string
BearerToken string
Meta map[string]string `json:",omitempty"`
EnterpriseMeta
}
type ACLLoginRequest struct {
@ -1453,6 +1496,8 @@ func (r *ACLLoginRequest) RequestDatacenter() string {
type ACLLogoutRequest struct {
Datacenter string // The datacenter to perform the request within
// TODO (namespaces) do we need the ent meta here? tokens are again
// unique across namespaces so its likely we don't need it.
WriteRequest
}

View File

@ -2,6 +2,36 @@
package structs
import (
"fmt"
"github.com/hashicorp/consul/acl"
)
const (
EnterpriseACLPolicyGlobalManagement = ""
// aclPolicyTemplateServiceIdentity is the template used for synthesizing
// policies for service identities.
aclPolicyTemplateServiceIdentity = `
service "%[1]s" {
policy = "write"
}
service "%[1]s-sidecar-proxy" {
policy = "write"
}
service_prefix "" {
policy = "read"
}
node_prefix "" {
policy = "read"
}`
)
func aclServiceIdentityRules(svc string, _ *EnterpriseMeta) string {
return fmt.Sprintf(aclPolicyTemplateServiceIdentity, svc)
}
func (p *ACLPolicy) EnterprisePolicyMeta() *acl.EnterprisePolicyMeta {
return nil
}

View File

@ -192,7 +192,7 @@ node_prefix "" {
Rules: test.expectRules,
}
got := svcid.SyntheticPolicy()
got := svcid.SyntheticPolicy(nil)
require.NotEmpty(t, got.ID)
require.True(t, strings.HasPrefix(got.Name, "synthetic-policy-"))
// strip irrelevant fields before equality

View File

@ -24,7 +24,7 @@ func TestConfigEntries_ListRelatedServices_AndACLs(t *testing.T) {
buf.WriteString(fmt.Sprintf("service %q { policy = %q }\n", s, "write"))
}
policy, err := acl.NewPolicyFromSource("", 0, buf.String(), acl.SyntaxCurrent, nil)
policy, err := acl.NewPolicyFromSource("", 0, buf.String(), acl.SyntaxCurrent, nil, nil)
require.NoError(t, err)
authorizer, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)

View File

@ -1808,6 +1808,35 @@ func Encode(t MessageType, msg interface{}) ([]byte, error) {
return buf.Bytes(), err
}
type ProtoMarshaller interface {
Size() int
MarshalTo([]byte) (int, error)
Unmarshal([]byte) error
ProtoMessage()
}
func EncodeProtoInterface(t MessageType, message interface{}) ([]byte, error) {
if marshaller, ok := message.(ProtoMarshaller); ok {
return EncodeProto(t, marshaller)
}
return nil, fmt.Errorf("message does not implement the ProtoMarshaller interface: %T", message)
}
func EncodeProto(t MessageType, message ProtoMarshaller) ([]byte, error) {
data := make([]byte, message.Size()+1)
data[0] = uint8(t)
if _, err := message.MarshalTo(data[1:]); err != nil {
return nil, err
}
return data, nil
}
func DecodeProto(buf []byte, out ProtoMarshaller) error {
// Note that this assumes the leading byte indicating the type as already been stripped off.
return out.Unmarshal(buf)
}
// CompoundResponse is an interface for gathering multiple responses. It is
// used in cross-datacenter RPC calls where more than 1 datacenter is
// expected to reply.

View File

@ -0,0 +1,36 @@
// +build !consulent
package structs
import (
"hash"
"github.com/hashicorp/consul/acl"
)
// EnterpriseMeta stub
type EnterpriseMeta struct{}
func (m *EnterpriseMeta) estimateSize() int {
return 0
}
func (m *EnterpriseMeta) addToHash(hasher hash.Hash) {
// do nothing
}
// ReplicationEnterpriseMeta stub
func ReplicationEnterpriseMeta() *EnterpriseMeta {
return nil
}
// DefaultEnterpriseMeta stub
func DefaultEnterpriseMeta() *EnterpriseMeta {
return nil
}
// InitDefault stub
func (m *EnterpriseMeta) InitDefault() {}
// FillAuthzContext stub
func (m *EnterpriseMeta) FillAuthzContext(*acl.EnterpriseAuthorizerContext) {}

View File

@ -448,7 +448,7 @@ func TestServer_StreamAggregatedResources_ACLEnforcement(t *testing.T) {
// Ensure the correct token was passed
require.Equal(t, tt.token, id)
// Parse the ACL and enforce it
policy, err := acl.NewPolicyFromSource("", 0, tt.acl, acl.SyntaxLegacy, nil)
policy, err := acl.NewPolicyFromSource("", 0, tt.acl, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
return acl.NewPolicyAuthorizerWithDefaults(acl.RootAuthorizer("deny"), []*acl.Policy{policy}, nil)
}
@ -508,7 +508,7 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedDuring
aclRules := `service "web" { policy = "write" }`
token := "service-write-on-web"
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil)
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
var validToken atomic.Value
@ -599,7 +599,7 @@ func TestServer_StreamAggregatedResources_ACLTokenDeleted_StreamTerminatedInBack
aclRules := `service "web" { policy = "write" }`
token := "service-write-on-web"
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil)
policy, err := acl.NewPolicyFromSource("", 0, aclRules, acl.SyntaxLegacy, nil, nil)
require.NoError(t, err)
var validToken atomic.Value