Merge pull request #11110 from hashicorp/dnephin/acl-legacy-remove-initialize
acl: remove initializeLegacyACL and the rest of the legacy FSM commands
This commit is contained in:
commit
afb1dd5827
|
@ -10,7 +10,9 @@ func (a *ACL) Bootstrap(*structs.DCSpecificRequest, *structs.ACL) error {
|
||||||
return fmt.Errorf("ACL.Bootstrap: the legacy ACL system has been removed")
|
return fmt.Errorf("ACL.Bootstrap: the legacy ACL system has been removed")
|
||||||
}
|
}
|
||||||
|
|
||||||
func (a *ACL) Apply(*structs.ACLRequest, *string) error {
|
type LegacyACLRequest struct{}
|
||||||
|
|
||||||
|
func (a *ACL) Apply(*LegacyACLRequest, *string) error {
|
||||||
return fmt.Errorf("ACL.Apply: the legacy ACL system has been removed")
|
return fmt.Errorf("ACL.Apply: the legacy ACL system has been removed")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -108,7 +108,7 @@ func init() {
|
||||||
registerCommand(structs.KVSRequestType, (*FSM).applyKVSOperation)
|
registerCommand(structs.KVSRequestType, (*FSM).applyKVSOperation)
|
||||||
registerCommand(structs.SessionRequestType, (*FSM).applySessionOperation)
|
registerCommand(structs.SessionRequestType, (*FSM).applySessionOperation)
|
||||||
// DEPRECATED (ACL-Legacy-Compat) - Only needed for v1 ACL compat
|
// DEPRECATED (ACL-Legacy-Compat) - Only needed for v1 ACL compat
|
||||||
registerCommand(structs.ACLRequestType, (*FSM).applyACLOperation)
|
registerCommand(structs.DeprecatedACLRequestType, (*FSM).deprecatedApplyACLOperation)
|
||||||
registerCommand(structs.TombstoneRequestType, (*FSM).applyTombstoneOperation)
|
registerCommand(structs.TombstoneRequestType, (*FSM).applyTombstoneOperation)
|
||||||
registerCommand(structs.CoordinateBatchUpdateType, (*FSM).applyCoordinateBatchUpdate)
|
registerCommand(structs.CoordinateBatchUpdateType, (*FSM).applyCoordinateBatchUpdate)
|
||||||
registerCommand(structs.PreparedQueryRequestType, (*FSM).applyPreparedQueryOperation)
|
registerCommand(structs.PreparedQueryRequestType, (*FSM).applyPreparedQueryOperation)
|
||||||
|
@ -243,37 +243,8 @@ func (c *FSM) applySessionOperation(buf []byte, index uint64) interface{} {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// DEPRECATED (ACL-Legacy-Compat) - Only needed for legacy compat
|
func (c *FSM) deprecatedApplyACLOperation(_ []byte, _ uint64) interface{} {
|
||||||
func (c *FSM) applyACLOperation(buf []byte, index uint64) interface{} {
|
return fmt.Errorf("legacy ACL command has been removed with the legacy ACL system")
|
||||||
// TODO (ACL-Legacy-Compat) - Should we warn here somehow about using deprecated features
|
|
||||||
// maybe emit a second metric?
|
|
||||||
var req structs.ACLRequest
|
|
||||||
if err := structs.Decode(buf, &req); err != nil {
|
|
||||||
panic(fmt.Errorf("failed to decode request: %v", err))
|
|
||||||
}
|
|
||||||
defer metrics.MeasureSinceWithLabels([]string{"fsm", "acl"}, time.Now(),
|
|
||||||
[]metrics.Label{{Name: "op", Value: string(req.Op)}})
|
|
||||||
switch req.Op {
|
|
||||||
case structs.ACLBootstrapInit:
|
|
||||||
enabled, _, err := c.state.CanBootstrapACLToken()
|
|
||||||
if err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
return enabled
|
|
||||||
case structs.ACLSet:
|
|
||||||
if err := c.state.ACLTokenSet(index, req.ACL.Convert(), true); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
return req.ACL.ID
|
|
||||||
case structs.ACLDelete:
|
|
||||||
return c.state.ACLTokenDeleteBySecret(index, req.ACL.ID, nil)
|
|
||||||
// Legacy commands that have been removed
|
|
||||||
case "bootstrap-now", "force-set":
|
|
||||||
return fmt.Errorf("command %v has been removed with the legacy ACL system", req.Op)
|
|
||||||
default:
|
|
||||||
c.logger.Warn("Invalid ACL operation", "operation", req.Op)
|
|
||||||
return fmt.Errorf("Invalid ACL operation '%s'", req.Op)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *FSM) applyTombstoneOperation(buf []byte, index uint64) interface{} {
|
func (c *FSM) applyTombstoneOperation(buf []byte, index uint64) interface{} {
|
||||||
|
@ -500,7 +471,6 @@ func (c *FSM) applyACLTokenSetOperation(buf []byte, index uint64) interface{} {
|
||||||
CAS: req.CAS,
|
CAS: req.CAS,
|
||||||
AllowMissingPolicyAndRoleIDs: req.AllowMissingLinks,
|
AllowMissingPolicyAndRoleIDs: req.AllowMissingLinks,
|
||||||
ProhibitUnprivileged: req.ProhibitUnprivileged,
|
ProhibitUnprivileged: req.ProhibitUnprivileged,
|
||||||
Legacy: false,
|
|
||||||
FromReplication: req.FromReplication,
|
FromReplication: req.FromReplication,
|
||||||
}
|
}
|
||||||
return c.state.ACLTokenBatchSet(index, req.Tokens, opts)
|
return c.state.ACLTokenBatchSet(index, req.Tokens, opts)
|
||||||
|
@ -524,7 +494,7 @@ func (c *FSM) applyACLTokenBootstrap(buf []byte, index uint64) interface{} {
|
||||||
}
|
}
|
||||||
defer metrics.MeasureSinceWithLabels([]string{"fsm", "acl", "token"}, time.Now(),
|
defer metrics.MeasureSinceWithLabels([]string{"fsm", "acl", "token"}, time.Now(),
|
||||||
[]metrics.Label{{Name: "op", Value: "bootstrap"}})
|
[]metrics.Label{{Name: "op", Value: "bootstrap"}})
|
||||||
return c.state.ACLBootstrap(index, req.ResetIndex, &req.Token, false)
|
return c.state.ACLBootstrap(index, req.ResetIndex, &req.Token)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *FSM) applyACLPolicySetOperation(buf []byte, index uint64) interface{} {
|
func (c *FSM) applyACLPolicySetOperation(buf []byte, index uint64) interface{} {
|
||||||
|
|
|
@ -829,102 +829,6 @@ func TestFSM_SessionCreate_Destroy(t *testing.T) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestFSM_ACL_CRUD(t *testing.T) {
|
|
||||||
t.Parallel()
|
|
||||||
logger := testutil.Logger(t)
|
|
||||||
fsm, err := New(nil, logger)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Create a new ACL.
|
|
||||||
req := structs.ACLRequest{
|
|
||||||
Datacenter: "dc1",
|
|
||||||
Op: structs.ACLSet,
|
|
||||||
ACL: structs.ACL{
|
|
||||||
ID: generateUUID(),
|
|
||||||
Name: "User token",
|
|
||||||
Type: structs.ACLTokenTypeClient,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
buf, err := structs.Encode(structs.ACLRequestType, req)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
resp := fsm.Apply(makeLog(buf))
|
|
||||||
if err, ok := resp.(error); ok {
|
|
||||||
t.Fatalf("resp: %v", err)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Get the ACL.
|
|
||||||
id := resp.(string)
|
|
||||||
_, acl, err := fsm.state.ACLTokenGetBySecret(nil, id, nil)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
if acl == nil {
|
|
||||||
t.Fatalf("missing")
|
|
||||||
}
|
|
||||||
|
|
||||||
// Verify the ACL.
|
|
||||||
if acl.SecretID != id {
|
|
||||||
t.Fatalf("bad: %v", *acl)
|
|
||||||
}
|
|
||||||
if acl.Description != "User token" {
|
|
||||||
t.Fatalf("bad: %v", *acl)
|
|
||||||
}
|
|
||||||
if acl.Type != structs.ACLTokenTypeClient {
|
|
||||||
t.Fatalf("bad: %v", *acl)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Try to destroy.
|
|
||||||
destroy := structs.ACLRequest{
|
|
||||||
Datacenter: "dc1",
|
|
||||||
Op: structs.ACLDelete,
|
|
||||||
ACL: structs.ACL{
|
|
||||||
ID: id,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
buf, err = structs.Encode(structs.ACLRequestType, destroy)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
resp = fsm.Apply(makeLog(buf))
|
|
||||||
if resp != nil {
|
|
||||||
t.Fatalf("resp: %v", resp)
|
|
||||||
}
|
|
||||||
|
|
||||||
_, acl, err = fsm.state.ACLTokenGetBySecret(nil, id, nil)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
if acl != nil {
|
|
||||||
t.Fatalf("should be destroyed")
|
|
||||||
}
|
|
||||||
|
|
||||||
// Initialize bootstrap (should work since we haven't made a management
|
|
||||||
// token).
|
|
||||||
init := structs.ACLRequest{
|
|
||||||
Datacenter: "dc1",
|
|
||||||
Op: structs.ACLBootstrapInit,
|
|
||||||
}
|
|
||||||
buf, err = structs.Encode(structs.ACLRequestType, init)
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
resp = fsm.Apply(makeLog(buf))
|
|
||||||
if enabled, ok := resp.(bool); !ok || !enabled {
|
|
||||||
t.Fatalf("resp: %v", resp)
|
|
||||||
}
|
|
||||||
canBootstrap, _, err := fsm.state.CanBootstrapACLToken()
|
|
||||||
if err != nil {
|
|
||||||
t.Fatalf("err: %v", err)
|
|
||||||
}
|
|
||||||
if !canBootstrap {
|
|
||||||
t.Fatalf("bad: shouldn't be able to bootstrap")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func TestFSM_PreparedQuery_CRUD(t *testing.T) {
|
func TestFSM_PreparedQuery_CRUD(t *testing.T) {
|
||||||
t.Parallel()
|
t.Parallel()
|
||||||
logger := testutil.Logger(t)
|
logger := testutil.Logger(t)
|
||||||
|
|
|
@ -15,7 +15,7 @@ func init() {
|
||||||
registerRestorer(structs.KVSRequestType, restoreKV)
|
registerRestorer(structs.KVSRequestType, restoreKV)
|
||||||
registerRestorer(structs.TombstoneRequestType, restoreTombstone)
|
registerRestorer(structs.TombstoneRequestType, restoreTombstone)
|
||||||
registerRestorer(structs.SessionRequestType, restoreSession)
|
registerRestorer(structs.SessionRequestType, restoreSession)
|
||||||
registerRestorer(structs.ACLRequestType, restoreACL)
|
registerRestorer(structs.DeprecatedACLRequestType, restoreACL)
|
||||||
registerRestorer(structs.ACLBootstrapRequestType, restoreACLBootstrap)
|
registerRestorer(structs.ACLBootstrapRequestType, restoreACLBootstrap)
|
||||||
registerRestorer(structs.CoordinateBatchUpdateType, restoreCoordinates)
|
registerRestorer(structs.CoordinateBatchUpdateType, restoreCoordinates)
|
||||||
registerRestorer(structs.PreparedQueryRequestType, restorePreparedQuery)
|
registerRestorer(structs.PreparedQueryRequestType, restorePreparedQuery)
|
||||||
|
|
|
@ -113,7 +113,7 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
|
||||||
// DEPRECATED (ACL-Legacy-Compat) - This is used so that the bootstrap token is still visible via the v1 acl APIs
|
// DEPRECATED (ACL-Legacy-Compat) - This is used so that the bootstrap token is still visible via the v1 acl APIs
|
||||||
Type: structs.ACLTokenTypeManagement,
|
Type: structs.ACLTokenTypeManagement,
|
||||||
}
|
}
|
||||||
require.NoError(t, fsm.state.ACLBootstrap(10, 0, token, false))
|
require.NoError(t, fsm.state.ACLBootstrap(10, 0, token))
|
||||||
|
|
||||||
method := &structs.ACLAuthMethod{
|
method := &structs.ACLAuthMethod{
|
||||||
Name: "some-method",
|
Name: "some-method",
|
||||||
|
@ -452,7 +452,7 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
|
||||||
// Persist a legacy ACL token - this is not done in newer code
|
// Persist a legacy ACL token - this is not done in newer code
|
||||||
// but we want to ensure that restoring legacy tokens works as
|
// but we want to ensure that restoring legacy tokens works as
|
||||||
// expected so we must inject one here manually
|
// expected so we must inject one here manually
|
||||||
_, err = sink.Write([]byte{byte(structs.ACLRequestType)})
|
_, err = sink.Write([]byte{byte(structs.DeprecatedACLRequestType)})
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
acl := structs.ACL{
|
acl := structs.ACL{
|
||||||
|
|
|
@ -398,102 +398,6 @@ func (s *Server) revokeLeadership() {
|
||||||
<-s.autopilot.Stop()
|
<-s.autopilot.Stop()
|
||||||
}
|
}
|
||||||
|
|
||||||
// DEPRECATED (ACL-Legacy-Compat) - Remove once old ACL compatibility is removed
|
|
||||||
func (s *Server) initializeLegacyACL() error {
|
|
||||||
if !s.config.ACLsEnabled {
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
authDC := s.config.PrimaryDatacenter
|
|
||||||
|
|
||||||
// Create anonymous token if missing.
|
|
||||||
state := s.fsm.State()
|
|
||||||
_, token, err := state.ACLTokenGetBySecret(nil, anonymousToken, nil)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed to get anonymous token: %v", err)
|
|
||||||
}
|
|
||||||
// Ignoring expiration times to avoid an insertion collision.
|
|
||||||
if token == nil {
|
|
||||||
req := structs.ACLRequest{
|
|
||||||
Datacenter: authDC,
|
|
||||||
Op: structs.ACLSet,
|
|
||||||
ACL: structs.ACL{
|
|
||||||
ID: anonymousToken,
|
|
||||||
Name: "Anonymous Token",
|
|
||||||
Type: structs.ACLTokenTypeClient,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
_, err := s.raftApply(structs.ACLRequestType, &req)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed to create anonymous token: %v", err)
|
|
||||||
}
|
|
||||||
s.logger.Info("Created the anonymous token")
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check for configured master token.
|
|
||||||
if master := s.config.ACLMasterToken; len(master) > 0 {
|
|
||||||
_, token, err = state.ACLTokenGetBySecret(nil, master, nil)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed to get master token: %v", err)
|
|
||||||
}
|
|
||||||
// Ignoring expiration times to avoid an insertion collision.
|
|
||||||
if token == nil {
|
|
||||||
req := structs.ACLRequest{
|
|
||||||
Datacenter: authDC,
|
|
||||||
Op: structs.ACLSet,
|
|
||||||
ACL: structs.ACL{
|
|
||||||
ID: master,
|
|
||||||
Name: "Master Token",
|
|
||||||
Type: structs.ACLTokenTypeManagement,
|
|
||||||
},
|
|
||||||
}
|
|
||||||
_, err := s.raftApply(structs.ACLRequestType, &req)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed to create master token: %v", err)
|
|
||||||
}
|
|
||||||
s.logger.Info("Created ACL master token from configuration")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check to see if we need to initialize the ACL bootstrap info. This
|
|
||||||
// needs a Consul version check since it introduces a new Raft operation
|
|
||||||
// that'll produce an error on older servers, and it also makes a piece
|
|
||||||
// of state in the state store that will cause problems with older
|
|
||||||
// servers consuming snapshots, so we have to wait to create it.
|
|
||||||
var minVersion = version.Must(version.NewVersion("0.9.1"))
|
|
||||||
if ok, _ := ServersInDCMeetMinimumVersion(s, s.config.Datacenter, minVersion); ok {
|
|
||||||
canBootstrap, _, err := state.CanBootstrapACLToken()
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed looking for ACL bootstrap info: %v", err)
|
|
||||||
}
|
|
||||||
if canBootstrap {
|
|
||||||
req := structs.ACLRequest{
|
|
||||||
Datacenter: authDC,
|
|
||||||
Op: structs.ACLBootstrapInit,
|
|
||||||
}
|
|
||||||
resp, err := s.raftApply(structs.ACLRequestType, &req)
|
|
||||||
if err != nil {
|
|
||||||
return fmt.Errorf("failed to initialize ACL bootstrap: %v", err)
|
|
||||||
}
|
|
||||||
switch v := resp.(type) {
|
|
||||||
case bool:
|
|
||||||
if v {
|
|
||||||
s.logger.Info("ACL bootstrap enabled")
|
|
||||||
} else {
|
|
||||||
s.logger.Info("ACL bootstrap disabled, existing management tokens found")
|
|
||||||
}
|
|
||||||
|
|
||||||
default:
|
|
||||||
return fmt.Errorf("unexpected response trying to initialize ACL bootstrap: %T", v)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
s.logger.Warn("Can't initialize ACL bootstrap until all servers are >= " + minVersion.String())
|
|
||||||
}
|
|
||||||
|
|
||||||
return nil
|
|
||||||
}
|
|
||||||
|
|
||||||
// initializeACLs is used to setup the ACLs if we are the leader
|
// initializeACLs is used to setup the ACLs if we are the leader
|
||||||
// and need to do this.
|
// and need to do this.
|
||||||
func (s *Server) initializeACLs(ctx context.Context, upgrade bool) error {
|
func (s *Server) initializeACLs(ctx context.Context, upgrade bool) error {
|
||||||
|
@ -525,11 +429,6 @@ func (s *Server) initializeACLs(ctx context.Context, upgrade bool) error {
|
||||||
}
|
}
|
||||||
|
|
||||||
if s.InACLDatacenter() {
|
if s.InACLDatacenter() {
|
||||||
if s.UseLegacyACLs() && !upgrade {
|
|
||||||
s.logger.Info("initializing legacy acls")
|
|
||||||
return s.initializeLegacyACL()
|
|
||||||
}
|
|
||||||
|
|
||||||
s.logger.Info("initializing acls")
|
s.logger.Info("initializing acls")
|
||||||
|
|
||||||
// TODO(partitions): initialize acls in all of the partitions?
|
// TODO(partitions): initialize acls in all of the partitions?
|
||||||
|
|
|
@ -74,7 +74,7 @@ func (s *Restore) ACLAuthMethod(method *structs.ACLAuthMethod) error {
|
||||||
|
|
||||||
// ACLBootstrap is used to perform a one-time ACL bootstrap operation on a
|
// ACLBootstrap is used to perform a one-time ACL bootstrap operation on a
|
||||||
// cluster to get the first management token.
|
// cluster to get the first management token.
|
||||||
func (s *Store) ACLBootstrap(idx, resetIndex uint64, token *structs.ACLToken, legacy bool) error {
|
func (s *Store) ACLBootstrap(idx, resetIndex uint64, token *structs.ACLToken) error {
|
||||||
tx := s.db.WriteTxn(idx)
|
tx := s.db.WriteTxn(idx)
|
||||||
defer tx.Abort()
|
defer tx.Abort()
|
||||||
|
|
||||||
|
@ -91,7 +91,7 @@ func (s *Store) ACLBootstrap(idx, resetIndex uint64, token *structs.ACLToken, le
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := aclTokenSetTxn(tx, idx, token, ACLTokenSetOptions{Legacy: legacy}); err != nil {
|
if err := aclTokenSetTxn(tx, idx, token, ACLTokenSetOptions{}); err != nil {
|
||||||
return fmt.Errorf("failed inserting bootstrap token: %v", err)
|
return fmt.Errorf("failed inserting bootstrap token: %v", err)
|
||||||
}
|
}
|
||||||
if err := tx.Insert(tableIndex, &IndexEntry{"acl-token-bootstrap", idx}); err != nil {
|
if err := tx.Insert(tableIndex, &IndexEntry{"acl-token-bootstrap", idx}); err != nil {
|
||||||
|
@ -429,7 +429,7 @@ type ACLTokenSetOptions struct {
|
||||||
CAS bool
|
CAS bool
|
||||||
AllowMissingPolicyAndRoleIDs bool
|
AllowMissingPolicyAndRoleIDs bool
|
||||||
ProhibitUnprivileged bool
|
ProhibitUnprivileged bool
|
||||||
Legacy bool
|
Legacy bool // TODO(ACL-Legacy-Compat): remove
|
||||||
FromReplication bool
|
FromReplication bool
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -806,13 +806,6 @@ func (s *Store) expiresIndexName(local bool) string {
|
||||||
return indexExpiresGlobal
|
return indexExpiresGlobal
|
||||||
}
|
}
|
||||||
|
|
||||||
// ACLTokenDeleteBySecret is used to remove an existing ACL from the state store. If
|
|
||||||
// the ACL does not exist this is a no-op and no error is returned.
|
|
||||||
// Deprecated (ACL-Legacy-Compat)
|
|
||||||
func (s *Store) ACLTokenDeleteBySecret(idx uint64, secret string, entMeta *structs.EnterpriseMeta) error {
|
|
||||||
return s.aclTokenDelete(idx, secret, "id", entMeta)
|
|
||||||
}
|
|
||||||
|
|
||||||
// ACLTokenDeleteByAccessor is used to remove an existing ACL from the state store. If
|
// ACLTokenDeleteByAccessor is used to remove an existing ACL from the state store. If
|
||||||
// the ACL does not exist this is a no-op and no error is returned.
|
// the ACL does not exist this is a no-op and no error is returned.
|
||||||
func (s *Store) ACLTokenDeleteByAccessor(idx uint64, accessor string, entMeta *structs.EnterpriseMeta) error {
|
func (s *Store) ACLTokenDeleteByAccessor(idx uint64, accessor string, entMeta *structs.EnterpriseMeta) error {
|
||||||
|
|
|
@ -199,7 +199,7 @@ func TestStateStore_ACLBootstrap(t *testing.T) {
|
||||||
require.Equal(t, uint64(0), index)
|
require.Equal(t, uint64(0), index)
|
||||||
|
|
||||||
// Perform a regular bootstrap.
|
// Perform a regular bootstrap.
|
||||||
require.NoError(t, s.ACLBootstrap(3, 0, token1.Clone(), false))
|
require.NoError(t, s.ACLBootstrap(3, 0, token1.Clone()))
|
||||||
|
|
||||||
// Make sure we can't bootstrap again
|
// Make sure we can't bootstrap again
|
||||||
canBootstrap, index, err = s.CanBootstrapACLToken()
|
canBootstrap, index, err = s.CanBootstrapACLToken()
|
||||||
|
@ -208,7 +208,7 @@ func TestStateStore_ACLBootstrap(t *testing.T) {
|
||||||
require.Equal(t, uint64(3), index)
|
require.Equal(t, uint64(3), index)
|
||||||
|
|
||||||
// Make sure another attempt fails.
|
// Make sure another attempt fails.
|
||||||
err = s.ACLBootstrap(4, 0, token2.Clone(), false)
|
err = s.ACLBootstrap(4, 0, token2.Clone())
|
||||||
require.Error(t, err)
|
require.Error(t, err)
|
||||||
require.Equal(t, structs.ACLBootstrapNotAllowedErr, err)
|
require.Equal(t, structs.ACLBootstrapNotAllowedErr, err)
|
||||||
|
|
||||||
|
@ -225,12 +225,12 @@ func TestStateStore_ACLBootstrap(t *testing.T) {
|
||||||
compareTokens(t, token1, tokens[0])
|
compareTokens(t, token1, tokens[0])
|
||||||
|
|
||||||
// bootstrap reset
|
// bootstrap reset
|
||||||
err = s.ACLBootstrap(32, index-1, token2.Clone(), false)
|
err = s.ACLBootstrap(32, index-1, token2.Clone())
|
||||||
require.Error(t, err)
|
require.Error(t, err)
|
||||||
require.Equal(t, structs.ACLBootstrapInvalidResetIndexErr, err)
|
require.Equal(t, structs.ACLBootstrapInvalidResetIndexErr, err)
|
||||||
|
|
||||||
// bootstrap reset
|
// bootstrap reset
|
||||||
err = s.ACLBootstrap(32, index, token2.Clone(), false)
|
err = s.ACLBootstrap(32, index, token2.Clone())
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "", nil, nil)
|
_, tokens, err = s.ACLTokenList(nil, true, true, "", "", "", nil, nil)
|
||||||
|
@ -1511,34 +1511,6 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
|
||||||
require.Nil(t, rtoken)
|
require.Nil(t, rtoken)
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Secret", func(t *testing.T) {
|
|
||||||
t.Parallel()
|
|
||||||
s := testACLTokensStateStore(t)
|
|
||||||
|
|
||||||
token := &structs.ACLToken{
|
|
||||||
AccessorID: "f1093997-b6c7-496d-bfb8-6b1b1895641b",
|
|
||||||
SecretID: "34ec8eb3-095d-417a-a937-b439af7a8e8b",
|
|
||||||
Policies: []structs.ACLTokenPolicyLink{
|
|
||||||
{
|
|
||||||
ID: structs.ACLPolicyGlobalManagementID,
|
|
||||||
},
|
|
||||||
},
|
|
||||||
Local: true,
|
|
||||||
}
|
|
||||||
|
|
||||||
require.NoError(t, s.ACLTokenSet(2, token.Clone(), false))
|
|
||||||
|
|
||||||
_, 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", nil))
|
|
||||||
|
|
||||||
_, rtoken, err = s.ACLTokenGetByAccessor(nil, "f1093997-b6c7-496d-bfb8-6b1b1895641b", nil)
|
|
||||||
require.NoError(t, err)
|
|
||||||
require.Nil(t, rtoken)
|
|
||||||
})
|
|
||||||
|
|
||||||
t.Run("Multiple", func(t *testing.T) {
|
t.Run("Multiple", func(t *testing.T) {
|
||||||
t.Parallel()
|
t.Parallel()
|
||||||
s := testACLTokensStateStore(t)
|
s := testACLTokensStateStore(t)
|
||||||
|
@ -1592,7 +1564,6 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
|
||||||
s := testACLTokensStateStore(t)
|
s := testACLTokensStateStore(t)
|
||||||
|
|
||||||
require.Error(t, s.ACLTokenDeleteByAccessor(3, structs.ACLTokenAnonymousID, nil))
|
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) {
|
t.Run("Not Found", func(t *testing.T) {
|
||||||
|
@ -1601,7 +1572,6 @@ func TestStateStore_ACLToken_Delete(t *testing.T) {
|
||||||
|
|
||||||
// deletion of non-existent policies is not an error
|
// deletion of non-existent policies is not an error
|
||||||
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "ea58a09c-2100-4aef-816b-8ee0ade77dcd", nil))
|
require.NoError(t, s.ACLTokenDeleteByAccessor(3, "ea58a09c-2100-4aef-816b-8ee0ade77dcd", nil))
|
||||||
require.NoError(t, s.ACLTokenDeleteBySecret(3, "376d0cae-dd50-4213-9668-2c7797a7fb2d", nil))
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -32,9 +32,6 @@ const (
|
||||||
ACLModeUnknown ACLMode = "3"
|
ACLModeUnknown ACLMode = "3"
|
||||||
)
|
)
|
||||||
|
|
||||||
// ACLOp is used in RPCs to encode ACL operations.
|
|
||||||
type ACLOp string
|
|
||||||
|
|
||||||
type ACLTokenIDType string
|
type ACLTokenIDType string
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
@ -89,16 +86,6 @@ func ACLIDReserved(id string) bool {
|
||||||
return strings.HasPrefix(id, ACLReservedPrefix)
|
return strings.HasPrefix(id, ACLReservedPrefix)
|
||||||
}
|
}
|
||||||
|
|
||||||
const (
|
|
||||||
// ACLSet creates or updates a token.
|
|
||||||
// TODO(ACL-Legacy-Compat): remove
|
|
||||||
ACLSet ACLOp = "set"
|
|
||||||
|
|
||||||
// ACLDelete deletes a token.
|
|
||||||
// TODO(ACL-Legacy-Compat): remove
|
|
||||||
ACLDelete ACLOp = "delete"
|
|
||||||
)
|
|
||||||
|
|
||||||
// ACLBootstrapNotAllowedErr is returned once we know that a bootstrap can no
|
// ACLBootstrapNotAllowedErr is returned once we know that a bootstrap can no
|
||||||
// longer be done since the cluster was bootstrapped
|
// longer be done since the cluster was bootstrapped
|
||||||
var ACLBootstrapNotAllowedErr = errors.New("ACL bootstrap no longer allowed")
|
var ACLBootstrapNotAllowedErr = errors.New("ACL bootstrap no longer allowed")
|
||||||
|
@ -432,6 +419,7 @@ func (t *ACLToken) HasExpirationTime() bool {
|
||||||
return t.ExpirationTime != nil && !t.ExpirationTime.IsZero()
|
return t.ExpirationTime != nil && !t.ExpirationTime.IsZero()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO(ACL-Legacy-Compat): remove
|
||||||
func (t *ACLToken) UsesNonLegacyFields() bool {
|
func (t *ACLToken) UsesNonLegacyFields() bool {
|
||||||
return len(t.Policies) > 0 ||
|
return len(t.Policies) > 0 ||
|
||||||
len(t.ServiceIdentities) > 0 ||
|
len(t.ServiceIdentities) > 0 ||
|
||||||
|
|
|
@ -13,14 +13,6 @@ import (
|
||||||
"github.com/hashicorp/consul/acl"
|
"github.com/hashicorp/consul/acl"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
|
||||||
// ACLBootstrapInit is used to perform a scan for existing tokens which
|
|
||||||
// will decide whether bootstrapping is allowed for a cluster. This is
|
|
||||||
// initiated by the leader when it steps up, if necessary.
|
|
||||||
// TODO(ACL-Legacy-Compat): remove
|
|
||||||
ACLBootstrapInit ACLOp = "bootstrap-init"
|
|
||||||
)
|
|
||||||
|
|
||||||
const (
|
const (
|
||||||
// ACLTokenTypeClient tokens have rules applied
|
// ACLTokenTypeClient tokens have rules applied
|
||||||
ACLTokenTypeClient = "client"
|
ACLTokenTypeClient = "client"
|
||||||
|
@ -95,21 +87,6 @@ func (tok *ACLToken) Convert() (*ACL, error) {
|
||||||
return compat, nil
|
return compat, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// ACLRequest is used to create, update or delete an ACL
|
|
||||||
type ACLRequest struct {
|
|
||||||
Datacenter string
|
|
||||||
Op ACLOp
|
|
||||||
ACL ACL
|
|
||||||
WriteRequest
|
|
||||||
}
|
|
||||||
|
|
||||||
func (r *ACLRequest) RequestDatacenter() string {
|
|
||||||
return r.Datacenter
|
|
||||||
}
|
|
||||||
|
|
||||||
// ACLRequests is a list of ACL change requests.
|
|
||||||
type ACLRequests []*ACLRequest
|
|
||||||
|
|
||||||
// ACLSpecificRequest is used to request an ACL by ID
|
// ACLSpecificRequest is used to request an ACL by ID
|
||||||
type ACLSpecificRequest struct {
|
type ACLSpecificRequest struct {
|
||||||
Datacenter string
|
Datacenter string
|
||||||
|
|
|
@ -41,7 +41,7 @@ const (
|
||||||
DeregisterRequestType = 1
|
DeregisterRequestType = 1
|
||||||
KVSRequestType = 2
|
KVSRequestType = 2
|
||||||
SessionRequestType = 3
|
SessionRequestType = 3
|
||||||
ACLRequestType = 4 // DEPRECATED (ACL-Legacy-Compat)
|
DeprecatedACLRequestType = 4 // Removed with the legacy ACL system
|
||||||
TombstoneRequestType = 5
|
TombstoneRequestType = 5
|
||||||
CoordinateBatchUpdateType = 6
|
CoordinateBatchUpdateType = 6
|
||||||
PreparedQueryRequestType = 7
|
PreparedQueryRequestType = 7
|
||||||
|
@ -81,7 +81,7 @@ var requestTypeStrings = map[MessageType]string{
|
||||||
DeregisterRequestType: "Deregister",
|
DeregisterRequestType: "Deregister",
|
||||||
KVSRequestType: "KVS",
|
KVSRequestType: "KVS",
|
||||||
SessionRequestType: "Session",
|
SessionRequestType: "Session",
|
||||||
ACLRequestType: "ACL", // DEPRECATED (ACL-Legacy-Compat)
|
DeprecatedACLRequestType: "ACL", // DEPRECATED (ACL-Legacy-Compat)
|
||||||
TombstoneRequestType: "Tombstone",
|
TombstoneRequestType: "Tombstone",
|
||||||
CoordinateBatchUpdateType: "CoordinateBatchUpdate",
|
CoordinateBatchUpdateType: "CoordinateBatchUpdate",
|
||||||
PreparedQueryRequestType: "PreparedQuery",
|
PreparedQueryRequestType: "PreparedQuery",
|
||||||
|
|
|
@ -380,7 +380,6 @@ These metrics are used to monitor the health of the Consul servers.
|
||||||
| `consul.catalog.deregister` | Measures the time it takes to complete a catalog deregister operation. | ms | timer |
|
| `consul.catalog.deregister` | Measures the time it takes to complete a catalog deregister operation. | ms | timer |
|
||||||
| `consul.fsm.register` | Measures the time it takes to apply a catalog register operation to the FSM. | ms | timer |
|
| `consul.fsm.register` | Measures the time it takes to apply a catalog register operation to the FSM. | ms | timer |
|
||||||
| `consul.fsm.deregister` | Measures the time it takes to apply a catalog deregister operation to the FSM. | ms | timer |
|
| `consul.fsm.deregister` | Measures the time it takes to apply a catalog deregister operation to the FSM. | ms | timer |
|
||||||
| `consul.fsm.acl.` | Measures the time it takes to apply the given ACL operation to the FSM. | ms | timer |
|
|
||||||
| `consul.fsm.session.` | Measures the time it takes to apply the given session operation to the FSM. | ms | timer |
|
| `consul.fsm.session.` | Measures the time it takes to apply the given session operation to the FSM. | ms | timer |
|
||||||
| `consul.fsm.kvs.` | Measures the time it takes to apply the given KV operation to the FSM. | ms | timer |
|
| `consul.fsm.kvs.` | Measures the time it takes to apply the given KV operation to the FSM. | ms | timer |
|
||||||
| `consul.fsm.tombstone.` | Measures the time it takes to apply the given tombstone operation to the FSM. | ms | timer |
|
| `consul.fsm.tombstone.` | Measures the time it takes to apply the given tombstone operation to the FSM. | ms | timer |
|
||||||
|
|
Loading…
Reference in New Issue