Unverified Commit 7a1e05f7 authored by James Rasell's avatar James Rasell
Browse files

core: add ACL role state schema and functionality.

This commit includes the new state schema for ACL roles along with
state interaction functions for CRUD actions.

The change also includes snapshot persist and restore
functionality and the addition of FSM messages for Raft updates
which will come via RPC endpoints.
parent 892ab8a0
Showing with 1268 additions and 0 deletions
+1268 -0
......@@ -58,6 +58,8 @@ var msgTypeNames = map[structs.MessageType]string{
structs.SecureVariableDeleteRequestType: "SecureVariableDeleteRequestType",
structs.RootKeyMetaUpsertRequestType: "RootKeyMetaUpsertRequestType",
structs.RootKeyMetaDeleteRequestType: "RootKeyMetaDeleteRequestType",
structs.ACLRolesUpsertRequestType: "ACLRolesUpsertRequestType",
structs.ACLRolesDeleteByIDRequestType: "ACLRolesDeleteByIDRequestType",
structs.NamespaceUpsertRequestType: "NamespaceUpsertRequestType",
structs.NamespaceDeleteRequestType: "NamespaceDeleteRequestType",
}
......@@ -58,6 +58,7 @@ const (
SecureVariablesSnapshot SnapshotType = 22
SecureVariablesQuotaSnapshot SnapshotType = 23
RootKeyMetaSnapshot SnapshotType = 24
ACLRoleSnapshot SnapshotType = 25
// Namespace appliers were moved from enterprise and therefore start at 64
NamespaceSnapshot SnapshotType = 64
......@@ -325,6 +326,10 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} {
return n.applyRootKeyMetaUpsert(msgType, buf[1:], log.Index)
case structs.RootKeyMetaDeleteRequestType:
return n.applyRootKeyMetaDelete(msgType, buf[1:], log.Index)
case structs.ACLRolesUpsertRequestType:
return n.applyACLRolesUpsert(msgType, buf[1:], log.Index)
case structs.ACLRolesDeleteByIDRequestType:
return n.applyACLRolesDeleteByID(msgType, buf[1:], log.Index)
}
// Check enterprise only message types.
......@@ -1750,6 +1755,20 @@ func (n *nomadFSM) restoreImpl(old io.ReadCloser, filter *FSMFilter) error {
if err := restore.RootKeyMetaRestore(keyMeta); err != nil {
return err
}
case ACLRoleSnapshot:
// Create a new ACLRole object, so we can decode the message into
// it.
aclRole := new(structs.ACLRole)
if err := dec.Decode(aclRole); err != nil {
return err
}
// Perform the restoration.
if err := restore.ACLRoleRestore(aclRole); err != nil {
return err
}
default:
// Check if this is an enterprise only object being restored
......@@ -2010,6 +2029,36 @@ func (n *nomadFSM) applyDeleteServiceRegistrationByNodeID(msgType structs.Messag
return nil
}
func (n *nomadFSM) applyACLRolesUpsert(msgType structs.MessageType, buf []byte, index uint64) interface{} {
defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_acl_role_upsert"}, time.Now())
var req structs.ACLRolesUpsertRequest
if err := structs.Decode(buf, &req); err != nil {
panic(fmt.Errorf("failed to decode request: %v", err))
}
if err := n.state.UpsertACLRoles(msgType, index, req.ACLRoles); err != nil {
n.logger.Error("UpsertACLRoles failed", "error", err)
return err
}
return nil
}
func (n *nomadFSM) applyACLRolesDeleteByID(msgType structs.MessageType, buf []byte, index uint64) interface{} {
defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_acl_role_delete_by_id"}, time.Now())
var req structs.ACLRolesDeleteByIDRequest
if err := structs.Decode(buf, &req); err != nil {
panic(fmt.Errorf("failed to decode request: %v", err))
}
if err := n.state.DeleteACLRolesByID(msgType, index, req.ACLRoleIDs); err != nil {
n.logger.Error("DeleteACLRolesByID failed", "error", err)
return err
}
return nil
}
type FSMFilter struct {
evaluator *bexpr.Evaluator
}
......@@ -2218,6 +2267,10 @@ func (s *nomadSnapshot) Persist(sink raft.SnapshotSink) error {
sink.Cancel()
return err
}
if err := s.persistACLRoles(sink, encoder); err != nil {
sink.Cancel()
return err
}
return nil
}
......@@ -2845,6 +2898,33 @@ func (s *nomadSnapshot) persistRootKeyMeta(sink raft.SnapshotSink,
return nil
}
func (s *nomadSnapshot) persistACLRoles(sink raft.SnapshotSink,
encoder *codec.Encoder) error {
// Get all the ACL roles.
ws := memdb.NewWatchSet()
aclRolesIter, err := s.snap.GetACLRoles(ws)
if err != nil {
return err
}
for {
// Get the next item.
for raw := aclRolesIter.Next(); raw != nil; raw = aclRolesIter.Next() {
// Prepare the request struct.
role := raw.(*structs.ACLRole)
// Write out an ACL role snapshot.
sink.Write([]byte{byte(ACLRoleSnapshot)})
if err := encoder.Encode(role); err != nil {
return err
}
}
return nil
}
}
// Release is a no-op, as we just need to GC the pointer
// to the state store snapshot. There is nothing to explicitly
// cleanup.
......
......@@ -2893,6 +2893,43 @@ func TestFSM_SnapshotRestore_ServiceRegistrations(t *testing.T) {
require.ElementsMatch(t, restoredRegs, serviceRegs)
}
func TestFSM_SnapshotRestore_ACLRoles(t *testing.T) {
ci.Parallel(t)
// Create our initial FSM which will be snapshotted.
fsm := testFSM(t)
testState := fsm.State()
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate and upsert some ACL roles.
aclRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, aclRoles))
// Perform a snapshot restore.
restoredFSM := testSnapshotRestore(t, fsm)
restoredState := restoredFSM.State()
// List the ACL roles from restored state and ensure everything is as
// expected.
iter, err := restoredState.GetACLRoles(memdb.NewWatchSet())
require.NoError(t, err)
var restoredACLRoles []*structs.ACLRole
for raw := iter.Next(); raw != nil; raw = iter.Next() {
restoredACLRoles = append(restoredACLRoles, raw.(*structs.ACLRole))
}
require.ElementsMatch(t, restoredACLRoles, aclRoles)
}
func TestFSM_ReconcileSummaries(t *testing.T) {
ci.Parallel(t)
// Add some state
......@@ -3413,6 +3450,73 @@ func TestFSM_SnapshotRestore_SecureVariables(t *testing.T) {
require.ElementsMatch(t, restoredSVs, svs)
}
func TestFSM_ApplyACLRolesUpsert(t *testing.T) {
ci.Parallel(t)
fsm := testFSM(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, fsm.State().UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate the upsert request and apply the change.
req := structs.ACLRolesUpsertRequest{
ACLRoles: []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()},
}
buf, err := structs.Encode(structs.ACLRolesUpsertRequestType, req)
require.NoError(t, err)
require.Nil(t, fsm.Apply(makeLog(buf)))
// Read out both ACL roles and perform an equality check using the hash.
ws := memdb.NewWatchSet()
out, err := fsm.State().GetACLRoleByName(ws, req.ACLRoles[0].Name)
require.NoError(t, err)
require.Equal(t, req.ACLRoles[0].Hash, out.Hash)
out, err = fsm.State().GetACLRoleByName(ws, req.ACLRoles[1].Name)
require.NoError(t, err)
require.Equal(t, req.ACLRoles[1].Hash, out.Hash)
}
func TestFSM_ApplyACLRolesDeleteByID(t *testing.T) {
ci.Parallel(t)
fsm := testFSM(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, fsm.State().UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate and upsert two ACL roles.
aclRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, fsm.State().UpsertACLRoles(structs.MsgTypeTestSetup, 10, aclRoles))
// Build and apply our message.
req := structs.ACLRolesDeleteByIDRequest{ACLRoleIDs: []string{aclRoles[0].ID, aclRoles[1].ID}}
buf, err := structs.Encode(structs.ACLRolesDeleteByIDRequestType, req)
require.NoError(t, err)
require.Nil(t, fsm.Apply(makeLog(buf)))
// List all ACL roles within state to ensure both have been removed.
ws := memdb.NewWatchSet()
iter, err := fsm.State().GetACLRoles(ws)
require.NoError(t, err)
var count int
for raw := iter.Next(); raw != nil; raw = iter.Next() {
count++
}
require.Equal(t, 0, count)
}
func TestFSM_ACLEvents(t *testing.T) {
ci.Parallel(t)
......
......@@ -2438,3 +2438,19 @@ func mockSecureVariableMetadata() structs.SecureVariableMetadata {
}
return out
}
func ACLRole() *structs.ACLRole {
role := structs.ACLRole{
ID: uuid.Generate(),
Name: fmt.Sprintf("acl-role-%s", uuid.Short()),
Description: "mocked-test-acl-role",
Policies: []*structs.ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
}
role.SetHash()
return &role
}
......@@ -17,6 +17,7 @@ const (
TableSecureVariables = "secure_variables"
TableSecureVariablesQuotas = "secure_variables_quota"
TableRootKeyMeta = "secure_variables_root_key_meta"
TableACLRoles = "acl_roles"
)
const (
......@@ -29,6 +30,7 @@ const (
indexExpiresLocal = "expires-local"
indexKeyID = "key_id"
indexPath = "path"
indexName = "name"
)
var (
......@@ -80,6 +82,7 @@ func init() {
secureVariablesTableSchema,
secureVariablesQuotasTableSchema,
secureVariablesRootKeyMetaSchema,
aclRolesTableSchema,
}...)
}
......@@ -1390,3 +1393,27 @@ func secureVariablesRootKeyMetaSchema() *memdb.TableSchema {
},
}
}
func aclRolesTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: TableACLRoles,
Indexes: map[string]*memdb.IndexSchema{
indexID: {
Name: indexID,
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "ID",
},
},
indexName: {
Name: indexName,
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "Name",
},
},
},
}
}
package state
import (
"errors"
"fmt"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/nomad/nomad/structs"
)
// ACLTokensByExpired returns an array accessor IDs of expired ACL tokens.
......@@ -31,3 +33,197 @@ func expiresIndexName(global bool) string {
}
return indexExpiresLocal
}
// UpsertACLRoles is used to insert a number of ACL roles into the state store.
// It uses a single write transaction for efficiency, however, any error means
// no entries will be committed.
func (s *StateStore) UpsertACLRoles(
msgType structs.MessageType, index uint64, roles []*structs.ACLRole) error {
// Grab a write transaction.
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
// updated tracks whether any inserts have been made. This allows us to
// skip updating the index table if we do not need to.
var updated bool
// Iterate the array of roles. In the event of a single error, all inserts
// fail via the txn.Abort() defer.
for _, role := range roles {
roleUpdated, err := s.upsertACLRoleTxn(index, txn, role)
if err != nil {
return err
}
// Ensure we track whether any inserts have been made.
updated = updated || roleUpdated
}
// If we did not perform any inserts, exit early.
if !updated {
return nil
}
// Perform the index table update to mark the new insert.
if err := txn.Insert(tableIndex, &IndexEntry{TableACLRoles, index}); err != nil {
return fmt.Errorf("index update failed: %v", err)
}
return txn.Commit()
}
// upsertACLRoleTxn inserts a single ACL role into the state store using the
// provided write transaction. It is the responsibility of the caller to update
// the index table.
func (s *StateStore) upsertACLRoleTxn(
index uint64, txn *txn, role *structs.ACLRole) (bool, error) {
// Ensure the role hash is not zero to provide defense in depth. This
// should be done outside the state store, so we do not spend time here
// and thus Raft, when it, can be avoided.
if len(role.Hash) == 0 {
role.SetHash()
}
// This validation also happens within the RPC handler, but Raft latency
// could mean that by the state call is invoked, another Raft update has
// deleted policies detailed in role. Therefore, check again.
if err := s.validateACLRolePolicyLinksTxn(txn, role); err != nil {
return false, err
}
existing, err := txn.First(TableACLRoles, indexID, role.ID)
if err != nil {
return false, fmt.Errorf("ACL role lookup failed: %v", err)
}
// Set up the indexes correctly to ensure existing indexes are maintained.
if existing != nil {
exist := existing.(*structs.ACLRole)
if exist.Equals(role) {
return false, nil
}
role.CreateIndex = exist.CreateIndex
role.ModifyIndex = index
} else {
role.CreateIndex = index
role.ModifyIndex = index
}
// Insert the role into the table.
if err := txn.Insert(TableACLRoles, role); err != nil {
return false, fmt.Errorf("ACL role insert failed: %v", err)
}
return true, nil
}
// ValidateACLRolePolicyLinks ensures all ACL policies linked to from the ACL
// role exist within state.
func (s *StateStore) ValidateACLRolePolicyLinks(role *structs.ACLRole) error {
txn := s.db.ReadTxn()
return s.validateACLRolePolicyLinksTxn(txn, role)
}
// validateACLRolePolicyLinksTxn is the same as ValidateACLRolePolicyLinks but
// allows callers to pass their own transaction.
func (s *StateStore) validateACLRolePolicyLinksTxn(txn *txn, role *structs.ACLRole) error {
for _, policyLink := range role.Policies {
_, existing, err := txn.FirstWatch("acl_policy", indexID, policyLink.Name)
if err != nil {
return fmt.Errorf("ACL policy lookup failed: %v", err)
}
if existing == nil {
return errors.New("ACL policy not found")
}
}
return nil
}
// DeleteACLRolesByID is responsible for batch deleting ACL roles based on
// their ID. It uses a single write transaction for efficiency, however, any
// error means no entries will be committed. An error is produced if a role is
// not found within state which has been passed within the array.
func (s *StateStore) DeleteACLRolesByID(
msgType structs.MessageType, index uint64, roleIDs []string) error {
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
for _, roleID := range roleIDs {
existing, err := txn.First(TableACLRoles, indexID, roleID)
if err != nil {
return fmt.Errorf("ACL role lookup failed: %v", err)
}
if existing == nil {
return errors.New("ACL role not found")
}
// Delete the existing entry from the table.
if err := txn.Delete(TableACLRoles, existing); err != nil {
return fmt.Errorf("ACL role deletion failed: %v", err)
}
}
// Update the index table to indicate an update has occurred.
if err := txn.Insert(tableIndex, &IndexEntry{TableACLRoles, index}); err != nil {
return fmt.Errorf("index update failed: %v", err)
}
return txn.Commit()
}
// GetACLRoles returns an iterator that contains all ACL roles stored within
// state.
func (s *StateStore) GetACLRoles(ws memdb.WatchSet) (memdb.ResultIterator, error) {
txn := s.db.ReadTxn()
// Walk the entire table to get all ACL roles.
iter, err := txn.Get(TableACLRoles, indexID)
if err != nil {
return nil, fmt.Errorf("ACL role lookup failed: %v", err)
}
ws.Add(iter.WatchCh())
return iter, nil
}
// GetACLRoleByID returns a single ACL role specified by the input ID. The role
// object will be nil, if no matching entry was found; it is the responsibility
// of the caller to check for this.
func (s *StateStore) GetACLRoleByID(ws memdb.WatchSet, roleID string) (*structs.ACLRole, error) {
txn := s.db.ReadTxn()
// Perform the ACL role lookup using the "id" index.
watchCh, existing, err := txn.FirstWatch(TableACLRoles, indexID, roleID)
if err != nil {
return nil, fmt.Errorf("ACL role lookup failed: %v", err)
}
ws.Add(watchCh)
if existing != nil {
return existing.(*structs.ACLRole), nil
}
return nil, nil
}
// GetACLRoleByName returns a single ACL role specified by the input name. The
// role object will be nil, if no matching entry was found; it is the
// responsibility of the caller to check for this.
func (s *StateStore) GetACLRoleByName(ws memdb.WatchSet, roleName string) (*structs.ACLRole, error) {
txn := s.db.ReadTxn()
// Perform the ACL role lookup using the "name" index.
watchCh, existing, err := txn.FirstWatch(TableACLRoles, indexName, roleName)
if err != nil {
return nil, fmt.Errorf("ACL role lookup failed: %v", err)
}
ws.Add(watchCh)
if existing != nil {
return existing.(*structs.ACLRole), nil
}
return nil, nil
}
......@@ -9,6 +9,7 @@ import (
"github.com/hashicorp/nomad/helper/pointer"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/shoenig/test/must"
"github.com/stretchr/testify/require"
)
......@@ -148,3 +149,297 @@ func Test_expiresIndexName(t *testing.T) {
})
}
}
func TestStateStore_UpsertACLRoles(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Generate a mocked ACL role for testing and attempt to upsert this
// straight into state. It should fail because the ACL policies do not
// exist.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole()}
err := testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles)
require.ErrorContains(t, err, "policy not found")
// Create the policies our ACL roles wants to link to and then try the
// upsert again.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 20, mockedACLRoles))
// Check that the index for the table was modified as expected.
initialIndex, err := testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 20, initialIndex)
// List all the ACL roles in the table, so we can perform a number of tests
// on the return array.
ws := memdb.NewWatchSet()
iter, err := testState.GetACLRoles(ws)
require.NoError(t, err)
// Count how many table entries we have, to ensure it is the expected
// number.
var count int
for raw := iter.Next(); raw != nil; raw = iter.Next() {
count++
// Ensure the create and modify indexes are populated correctly.
aclRole := raw.(*structs.ACLRole)
must.Eq(t, 20, aclRole.CreateIndex)
must.Eq(t, 20, aclRole.ModifyIndex)
}
require.Equal(t, 1, count, "incorrect number of ACL roles found")
// Try writing the same ACL roles to state which should not result in an
// update to the table index.
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 30, mockedACLRoles))
reInsertActualIndex, err := testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 20, reInsertActualIndex)
// Make a change to one of the ACL roles and ensure this update is accepted
// and the table index is updated.
updatedMockedACLRole := mockedACLRoles[0].Copy()
updatedMockedACLRole.Policies = []*structs.ACLRolePolicyLink{{Name: "mocked-test-policy-1"}}
updatedMockedACLRole.SetHash()
require.NoError(t, testState.UpsertACLRoles(
structs.MsgTypeTestSetup, 30, []*structs.ACLRole{updatedMockedACLRole}))
// Check that the index for the table was modified as expected.
updatedIndex, err := testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 30, updatedIndex)
// List the ACL roles in state.
iter, err = testState.GetACLRoles(ws)
require.NoError(t, err)
// Count how many table entries we have, to ensure it is the expected
// number.
count = 0
for raw := iter.Next(); raw != nil; raw = iter.Next() {
count++
// Ensure the create and modify indexes are populated correctly.
aclRole := raw.(*structs.ACLRole)
must.Eq(t, 20, aclRole.CreateIndex)
must.Eq(t, 30, aclRole.ModifyIndex)
}
require.Equal(t, 1, count, "incorrect number of ACL roles found")
}
func TestStateStore_ValidateACLRolePolicyLinks(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Create our mocked role which includes two ACL policy links.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole()}
// This should error as no policies exist within state.
err := testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles)
require.ErrorContains(t, err, "ACL policy not found")
// Upsert one ACL policy and retry the role which should still fail.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
require.NoError(t, testState.UpsertACLPolicies(structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1}))
err = testState.UpsertACLRoles(structs.MsgTypeTestSetup, 20, mockedACLRoles)
require.ErrorContains(t, err, "ACL policy not found")
// Upsert the second ACL policy. The ACL role should now upsert into state
// without error.
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(structs.MsgTypeTestSetup, 20, []*structs.ACLPolicy{policy2}))
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 30, mockedACLRoles))
}
func TestStateStore_DeleteACLRolesByID(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate a some mocked ACL roles for testing and upsert these straight
// into state.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles))
// Try and delete a role using a name that doesn't exist. This should
// return an error and not change the index for the table.
err := testState.DeleteACLRolesByID(structs.MsgTypeTestSetup, 20, []string{"not-a-role"})
require.ErrorContains(t, err, "ACL role not found")
tableIndex, err := testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 10, tableIndex)
// Delete one of the previously upserted ACL roles. This should succeed
// and modify the table index.
err = testState.DeleteACLRolesByID(structs.MsgTypeTestSetup, 20, []string{mockedACLRoles[0].ID})
require.NoError(t, err)
tableIndex, err = testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 20, tableIndex)
// List the ACL roles and ensure we now only have one present and that it
// is the one we expect.
ws := memdb.NewWatchSet()
iter, err := testState.GetACLRoles(ws)
require.NoError(t, err)
var aclRoles []*structs.ACLRole
for raw := iter.Next(); raw != nil; raw = iter.Next() {
aclRoles = append(aclRoles, raw.(*structs.ACLRole))
}
require.Len(t, aclRoles, 1, "incorrect number of ACL roles found")
require.True(t, aclRoles[0].Equals(mockedACLRoles[1]))
// Delete the final remaining ACL role. This should succeed and modify the
// table index.
err = testState.DeleteACLRolesByID(structs.MsgTypeTestSetup, 30, []string{mockedACLRoles[1].ID})
require.NoError(t, err)
tableIndex, err = testState.Index(TableACLRoles)
require.NoError(t, err)
must.Eq(t, 30, tableIndex)
// List the ACL roles and ensure we have zero entries.
iter, err = testState.GetACLRoles(ws)
require.NoError(t, err)
aclRoles = []*structs.ACLRole{}
for raw := iter.Next(); raw != nil; raw = iter.Next() {
aclRoles = append(aclRoles, raw.(*structs.ACLRole))
}
require.Len(t, aclRoles, 0, "incorrect number of ACL roles found")
}
func TestStateStore_GetACLRoles(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate a some mocked ACL roles for testing and upsert these straight
// into state.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles))
// List the ACL roles and ensure they are exactly as we expect.
ws := memdb.NewWatchSet()
iter, err := testState.GetACLRoles(ws)
require.NoError(t, err)
var aclRoles []*structs.ACLRole
for raw := iter.Next(); raw != nil; raw = iter.Next() {
aclRoles = append(aclRoles, raw.(*structs.ACLRole))
}
expected := mockedACLRoles
for i := range expected {
expected[i].CreateIndex = 10
expected[i].ModifyIndex = 10
}
require.ElementsMatch(t, aclRoles, expected)
}
func TestStateStore_GetACLRoleByID(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate a some mocked ACL roles for testing and upsert these straight
// into state.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles))
ws := memdb.NewWatchSet()
// Try reading an ACL role that does not exist.
aclRole, err := testState.GetACLRoleByID(ws, "not-a-role")
require.NoError(t, err)
require.Nil(t, aclRole)
// Read the two ACL roles that we should find.
aclRole, err = testState.GetACLRoleByID(ws, mockedACLRoles[0].ID)
require.NoError(t, err)
require.Equal(t, mockedACLRoles[0], aclRole)
aclRole, err = testState.GetACLRoleByID(ws, mockedACLRoles[1].ID)
require.NoError(t, err)
require.Equal(t, mockedACLRoles[1], aclRole)
}
func TestStateStore_GetACLRoleByName(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Create the policies our ACL roles wants to link to.
policy1 := mock.ACLPolicy()
policy1.Name = "mocked-test-policy-1"
policy2 := mock.ACLPolicy()
policy2.Name = "mocked-test-policy-2"
require.NoError(t, testState.UpsertACLPolicies(
structs.MsgTypeTestSetup, 10, []*structs.ACLPolicy{policy1, policy2}))
// Generate a some mocked ACL roles for testing and upsert these straight
// into state.
mockedACLRoles := []*structs.ACLRole{mock.ACLRole(), mock.ACLRole()}
require.NoError(t, testState.UpsertACLRoles(structs.MsgTypeTestSetup, 10, mockedACLRoles))
ws := memdb.NewWatchSet()
// Try reading an ACL role that does not exist.
aclRole, err := testState.GetACLRoleByName(ws, "not-a-role")
require.NoError(t, err)
require.Nil(t, aclRole)
// Read the two ACL roles that we should find.
aclRole, err = testState.GetACLRoleByName(ws, mockedACLRoles[0].Name)
require.NoError(t, err)
require.Equal(t, mockedACLRoles[0], aclRole)
aclRole, err = testState.GetACLRoleByName(ws, mockedACLRoles[1].Name)
require.NoError(t, err)
require.Equal(t, mockedACLRoles[1], aclRole)
}
......@@ -224,3 +224,12 @@ func (r *StateRestore) RootKeyMetaRestore(quota *structs.RootKeyMeta) error {
}
return nil
}
// ACLRoleRestore is used to restore a single ACL role into the acl_roles
// table.
func (r *StateRestore) ACLRoleRestore(aclRole *structs.ACLRole) error {
if err := r.txn.Insert(TableACLRoles, aclRole); err != nil {
return fmt.Errorf("ACL role insert failed: %v", err)
}
return nil
}
......@@ -604,3 +604,26 @@ func TestStateStore_SecureVariablesRestore(t *testing.T) {
require.Equal(t, svs[i], out)
}
}
func TestStateStore_ACLRoleRestore(t *testing.T) {
ci.Parallel(t)
testState := testStateStore(t)
// Set up our test registrations and index.
expectedIndex := uint64(13)
aclRole := mock.ACLRole()
aclRole.CreateIndex = expectedIndex
aclRole.ModifyIndex = expectedIndex
restore, err := testState.Restore()
require.NoError(t, err)
require.NoError(t, restore.ACLRoleRestore(aclRole))
require.NoError(t, restore.Commit())
// Check the state is now populated as we expect and that we can find the
// restored registrations.
ws := memdb.NewWatchSet()
out, err := testState.GetACLRoleByName(ws, aclRole.Name)
require.NoError(t, err)
require.Equal(t, aclRole, out)
}
package structs
import (
"bytes"
"errors"
"fmt"
"regexp"
"time"
"github.com/hashicorp/go-multierror"
"github.com/hashicorp/nomad/helper/pointer"
"github.com/hashicorp/nomad/helper/uuid"
"golang.org/x/crypto/blake2b"
"golang.org/x/exp/slices"
)
const (
......@@ -40,6 +44,14 @@ const (
// large number of expired tokens pending garbage collection, this value is
// a potential limiting factor.
ACLMaxExpiredBatchSize = 4096
// maxACLRoleDescriptionLength limits an ACL roles description length.
maxACLRoleDescriptionLength = 256
)
var (
// validACLRoleName is used to validate an ACL role name.
validACLRoleName = regexp.MustCompile("^[a-zA-Z0-9-]{1,128}$")
)
// Canonicalize performs basic canonicalization on the ACL token object. It is
......@@ -161,3 +173,152 @@ func (a *ACLToken) IsExpired(t time.Time) bool {
return a.ExpirationTime.Before(t) || t.IsZero()
}
// ACLRole is an abstraction for the ACL system which allows the grouping of
// ACL policies into a single object. ACL tokens can be created and linked to
// a role; the token then inherits all the permissions granted by the policies.
type ACLRole struct {
// ID is an internally generated UUID for this role and is controlled by
// Nomad.
ID string
// Name is unique across the entire set of federated clusters and is
// supplied by the operator on role creation. The name can be modified by
// updating the role and including the Nomad generated ID. This update will
// not affect tokens created and linked to this role. This is a required
// field.
Name string
// Description is a human-readable, operator set description that can
// provide additional context about the role. This is an operational field.
Description string
// Policies is an array of ACL policy links. Although currently policies
// can only be linked using their name, in the future we will want to add
// IDs also and thus allow operators to specify either a name, an ID, or
// both.
Policies []*ACLRolePolicyLink
// Hash is the hashed value of the role and is generated using all fields
// above this point.
Hash []byte
CreateIndex uint64
ModifyIndex uint64
}
// ACLRolePolicyLink is used to link a policy to an ACL role. We use a struct
// rather than a list of strings as in the future we will want to add IDs to
// policies and then link via these.
type ACLRolePolicyLink struct {
// Name is the ACLPolicy.Name value which will be linked to the ACL role.
Name string
}
// SetHash is used to compute and set the hash of the ACL role. This should be
// called every and each time a user specified field on the role is changed
// before updating the Nomad state store.
func (a *ACLRole) SetHash() []byte {
// Initialize a 256bit Blake2 hash (32 bytes).
hash, err := blake2b.New256(nil)
if err != nil {
panic(err)
}
// Write all the user set fields.
_, _ = hash.Write([]byte(a.Name))
_, _ = hash.Write([]byte(a.Description))
for _, policyLink := range a.Policies {
_, _ = hash.Write([]byte(policyLink.Name))
}
// Finalize the hash.
hashVal := hash.Sum(nil)
// Set and return the hash.
a.Hash = hashVal
return hashVal
}
// Validate ensure the ACL role contains valid information which meets Nomad's
// internal requirements. This does not include any state calls, such as
// ensuring the linked policies exist.
func (a *ACLRole) Validate() error {
var mErr multierror.Error
if !validACLRoleName.MatchString(a.Name) {
mErr.Errors = append(mErr.Errors, fmt.Errorf("invalid name '%s'", a.Name))
}
if len(a.Description) > maxACLRoleDescriptionLength {
mErr.Errors = append(mErr.Errors, fmt.Errorf("description longer than %d", maxACLRoleDescriptionLength))
}
if len(a.Policies) < 1 {
mErr.Errors = append(mErr.Errors, errors.New("at least one policy should be specified"))
}
return mErr.ErrorOrNil()
}
// Equals performs an equality check on the two service registrations. It
// handles nil objects.
func (a *ACLRole) Equals(o *ACLRole) bool {
if a == nil || o == nil {
return a == o
}
if len(a.Hash) == 0 {
a.SetHash()
}
if len(o.Hash) == 0 {
o.SetHash()
}
return bytes.Equal(a.Hash, o.Hash)
}
// Copy creates a deep copy of the ACL role. This copy can then be safely
// modified. It handles nil objects.
func (a *ACLRole) Copy() *ACLRole {
if a == nil {
return nil
}
c := new(ACLRole)
*c = *a
c.Policies = slices.Clone(a.Policies)
c.Hash = slices.Clone(a.Hash)
return c
}
// ACLRolesUpsertRequest is the request object used to upsert one or more ACL
// roles.
type ACLRolesUpsertRequest struct {
ACLRoles []*ACLRole
WriteRequest
}
// ACLRolesUpsertResponse is the response object when one or more ACL roles
// have been successfully upserted into state.
type ACLRolesUpsertResponse struct {
WriteMeta
}
// ACLRolesDeleteByIDRequest is the request object to delete one or more ACL
// roles using the role ID.
type ACLRolesDeleteByIDRequest struct {
ACLRoleIDs []string
WriteRequest
}
// ACLRolesDeleteByIDResponse is the response object when performing a deletion
// of one or more ACL roles using the role ID.
type ACLRolesDeleteByIDResponse struct {
WriteMeta
}
package structs
import (
"fmt"
"testing"
"time"
......@@ -285,3 +286,355 @@ func TestACLToken_IsExpired(t *testing.T) {
})
}
}
func TestACLRole_SetHash(t *testing.T) {
testCases := []struct {
name string
inputACLRole *ACLRole
expectedOutput []byte
}{
{
name: "no hash set",
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{},
},
expectedOutput: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
{
name: "hash set with change",
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
137, 147, 2, 29, 53, 94, 78, 13, 45, 51, 127, 193, 21, 248, 230, 126, 34,
106, 216, 73, 248, 219, 209, 146, 204, 107, 185, 2, 89, 255, 198, 5,
},
},
expectedOutput: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
actualOutput := tc.inputACLRole.SetHash()
require.Equal(t, tc.expectedOutput, actualOutput)
require.Equal(t, tc.inputACLRole.Hash, actualOutput)
})
}
}
func TestACLRole_Validate(t *testing.T) {
testCases := []struct {
name string
inputACLRole *ACLRole
expectedError bool
expectedErrorContains string
}{
{
name: "role name too long",
inputACLRole: &ACLRole{
Name: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
},
expectedError: true,
expectedErrorContains: "invalid name",
},
{
name: "role name too short",
inputACLRole: &ACLRole{
Name: "",
},
expectedError: true,
expectedErrorContains: "invalid name",
},
{
name: "role name with invalid characters",
inputACLRole: &ACLRole{
Name: "--#$%$^%_%%_?>",
},
expectedError: true,
expectedErrorContains: "invalid name",
},
{
name: "description too long",
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
},
expectedError: true,
expectedErrorContains: "description longer than",
},
{
name: "no policies",
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "",
},
expectedError: true,
expectedErrorContains: "at least one policy should be specified",
},
{
name: "valid",
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "",
Policies: []*ACLRolePolicyLink{
{Name: "policy-1"},
},
},
expectedError: false,
expectedErrorContains: "",
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
actualOutput := tc.inputACLRole.Validate()
if tc.expectedError {
require.ErrorContains(t, actualOutput, tc.expectedErrorContains)
} else {
require.NoError(t, actualOutput)
}
})
}
}
func TestACLRole_Equals(t *testing.T) {
testCases := []struct {
name string
composedACLRole *ACLRole
inputACLRole *ACLRole
expectedOutput bool
}{
{
name: "equal with hash set",
composedACLRole: &ACLRole{
Name: "acl-role-",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
expectedOutput: true,
},
{
name: "equal without hash set",
composedACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{},
},
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{},
},
expectedOutput: true,
},
{
name: "both nil",
composedACLRole: nil,
inputACLRole: nil,
expectedOutput: true,
},
{
name: "not equal composed nil",
composedACLRole: nil,
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
expectedOutput: false,
},
{
name: "not equal input nil",
composedACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
inputACLRole: nil,
expectedOutput: false,
},
{
name: "not equal with hash set",
composedACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
137, 147, 2, 29, 53, 94, 78, 13, 45, 51, 127, 193, 21, 248, 230, 126, 34,
106, 216, 73, 248, 219, 209, 146, 204, 107, 185, 2, 89, 255, 198, 5,
},
},
expectedOutput: false,
},
{
name: "not equal without hash set",
composedACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{},
},
inputACLRole: &ACLRole{
Name: "acl-role",
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{},
},
expectedOutput: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
actualOutput := tc.composedACLRole.Equals(tc.inputACLRole)
require.Equal(t, tc.expectedOutput, actualOutput)
})
}
}
func TestACLRole_Copy(t *testing.T) {
testCases := []struct {
name string
inputACLRole *ACLRole
}{
{
name: "nil input",
inputACLRole: nil,
},
{
name: "general 1",
inputACLRole: &ACLRole{
Name: fmt.Sprintf("acl-role"),
Description: "mocked-test-acl-role",
Policies: []*ACLRolePolicyLink{
{Name: "mocked-test-policy-1"},
{Name: "mocked-test-policy-2"},
},
CreateIndex: 10,
ModifyIndex: 10,
Hash: []byte{
122, 193, 189, 171, 197, 13, 37, 81, 141, 213, 188, 212, 179, 223, 148, 160,
171, 141, 155, 136, 21, 128, 252, 100, 149, 195, 236, 148, 94, 70, 173, 102,
},
},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
actualOutput := tc.inputACLRole.Copy()
require.Equal(t, tc.inputACLRole, actualOutput)
})
}
}
func Test_ACLRolesUpsertRequest(t *testing.T) {
req := ACLRolesUpsertRequest{}
require.False(t, req.IsRead())
}
func Test_ACLRolesDeleteByIDRequest(t *testing.T) {
req := ACLRolesDeleteByIDRequest{}
require.False(t, req.IsRead())
}
......@@ -113,6 +113,8 @@ const (
SecureVariableDeleteRequestType MessageType = 51
RootKeyMetaUpsertRequestType MessageType = 52
RootKeyMetaDeleteRequestType MessageType = 53
ACLRolesUpsertRequestType MessageType = 54
ACLRolesDeleteByIDRequestType MessageType = 55
// Namespace types were moved from enterprise and therefore start at 64
NamespaceUpsertRequestType MessageType = 64
......
Supports Markdown
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment