Adds a new /v1/acl/bootstrap API (#3349)

This commit is contained in:
James Phillips 2017-08-02 17:05:18 -07:00 committed by GitHub
parent 30bc08143e
commit c31b56a03e
23 changed files with 1267 additions and 259 deletions

View File

@ -20,6 +20,33 @@ func ACLDisabled(resp http.ResponseWriter, req *http.Request) (interface{}, erro
return nil, nil
}
// ACLBootstrap is used to perform a one-time ACL bootstrap operation on
// a cluster to get the first management token.
func (s *HTTPServer) ACLBootstrap(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
if req.Method != "PUT" {
resp.WriteHeader(http.StatusMethodNotAllowed)
return nil, nil
}
args := structs.DCSpecificRequest{
Datacenter: s.agent.config.ACLDatacenter,
}
var out structs.ACL
err := s.agent.RPC("ACL.Bootstrap", &args, &out)
if err != nil {
if strings.Contains(err.Error(), structs.ACLBootstrapNotAllowedErr.Error()) {
resp.WriteHeader(http.StatusForbidden)
fmt.Fprintf(resp, "Permission denied: %v", err)
return nil, nil
} else {
return nil, err
}
}
return aclCreateResponse{out.ID}, nil
}
func (s *HTTPServer) ACLDestroy(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
// Mandate a PUT request
if req.Method != "PUT" {

View File

@ -30,6 +30,52 @@ func makeTestACL(t *testing.T, srv *HTTPServer) string {
return aclResp.ID
}
func TestACL_Bootstrap(t *testing.T) {
t.Parallel()
cfg := TestACLConfig()
cfg.Version = "0.9.1"
cfg.ACLMasterToken = ""
a := NewTestAgent(t.Name(), cfg)
defer a.Shutdown()
tests := []struct {
name string
method string
code int
token bool
}{
{"bad method", "GET", http.StatusMethodNotAllowed, false},
{"bootstrap", "PUT", http.StatusOK, true},
{"not again", "PUT", http.StatusForbidden, false},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
resp := httptest.NewRecorder()
req, _ := http.NewRequest(tt.method, "/v1/acl/bootstrap", nil)
out, err := a.srv.ACLBootstrap(resp, req)
if err != nil {
t.Fatalf("err: %v", err)
}
if got, want := resp.Code, tt.code; got != want {
t.Fatalf("got %d want %d", got, want)
}
if tt.token {
wrap, ok := out.(aclCreateResponse)
if !ok {
t.Fatalf("bad: %T", out)
}
if len(wrap.ID) != len("xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx") {
t.Fatalf("bad: %v", wrap)
}
} else {
if out != nil {
t.Fatalf("bad: %T", out)
}
}
})
}
}
func TestACL_Update(t *testing.T) {
t.Parallel()
a := NewTestAgent(t.Name(), TestACLConfig())

View File

@ -660,12 +660,6 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
if a.config.RaftProtocol != 0 {
base.RaftConfig.ProtocolVersion = raft.ProtocolVersion(a.config.RaftProtocol)
}
if a.config.ACLToken != "" {
base.ACLToken = a.config.ACLToken
}
if a.config.ACLAgentToken != "" {
base.ACLAgentToken = a.config.ACLAgentToken
}
if a.config.ACLMasterToken != "" {
base.ACLMasterToken = a.config.ACLMasterToken
}

View File

@ -743,6 +743,6 @@ func (s *HTTPServer) AgentToken(resp http.ResponseWriter, req *http.Request) (in
return nil, nil
}
s.agent.logger.Printf("[INFO] Updated agent's %q", target)
s.agent.logger.Printf("[INFO] Updated agent's ACL token %q", target)
return nil, nil
}

View File

@ -17,6 +17,69 @@ type ACL struct {
srv *Server
}
// Bootstrap is used to perform a one-time ACL bootstrap operation on
// a cluster to get the first management token.
func (a *ACL) Bootstrap(args *structs.DCSpecificRequest, reply *structs.ACL) error {
if done, err := a.srv.forward("ACL.Bootstrap", args, args, reply); done {
return err
}
// Verify we are allowed to serve this request
if a.srv.config.ACLDatacenter != a.srv.config.Datacenter {
return fmt.Errorf(aclDisabled)
}
// By doing some pre-checks we can head off later bootstrap attempts
// without having to run them through Raft, which should curb abuse.
state := a.srv.fsm.State()
bs, err := state.ACLGetBootstrap()
if err != nil {
return err
}
if bs == nil {
return structs.ACLBootstrapNotInitializedErr
}
if !bs.AllowBootstrap {
return structs.ACLBootstrapNotAllowedErr
}
// Propose a new token.
token, err := uuid.GenerateUUID()
if err != nil {
return fmt.Errorf("failed to make random token: %v", err)
}
// Attempt a bootstrap.
req := structs.ACLRequest{
Datacenter: a.srv.config.ACLDatacenter,
Op: structs.ACLBootstrapNow,
ACL: structs.ACL{
ID: token,
Name: "Bootstrap Token",
Type: structs.ACLTypeManagement,
},
}
resp, err := a.srv.raftApply(structs.ACLRequestType, &req)
if err != nil {
return err
}
switch v := resp.(type) {
case error:
return v
case *structs.ACL:
*reply = *v
default:
// Just log this, since it looks like the bootstrap may have
// completed.
a.srv.logger.Printf("[ERR] consul.acl: Unexpected response during bootstrap: %T", v)
}
a.srv.logger.Printf("[INFO] consul.acl: ACL bootstrap completed")
return nil
}
// aclApplyInternal is used to apply an ACL request after it has been vetted that
// this is a valid operation. It is used when users are updating ACLs, in which
// case we check their token to make sure they have management privileges. It is

View File

@ -13,6 +13,59 @@ import (
"github.com/hashicorp/net-rpc-msgpackrpc"
)
func TestACLEndpoint_Bootstrap(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.Build = "0.8.0" // Too low for auto init of bootstrap.
c.ACLDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Expect an error initially since ACL bootstrap is not initialized.
arg := structs.DCSpecificRequest{
Datacenter: "dc1",
}
var out structs.ACL
err := msgpackrpc.CallWithCodec(codec, "ACL.Bootstrap", &arg, &out)
if err.Error() != structs.ACLBootstrapNotInitializedErr.Error() {
t.Fatalf("err: %v", err)
}
// Manually do an init.
req := structs.ACLRequest{
Datacenter: "dc1",
Op: structs.ACLBootstrapInit,
}
_, err = s1.raftApply(structs.ACLRequestType, &req)
if err != nil {
t.Fatalf("err: %v", err)
}
// Try again, this time it should go through. We can only do some high
// level checks on the ACL since we don't have control over the UUID or
// Raft indexes at this level.
if err := msgpackrpc.CallWithCodec(codec, "ACL.Bootstrap", &arg, &out); err != nil {
t.Fatalf("err: %v", err)
}
if len(out.ID) != len("xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx") ||
out.Name != "Bootstrap Token" ||
out.Type != structs.ACLTypeManagement ||
out.CreateIndex == 0 || out.ModifyIndex == 0 {
t.Fatalf("bad: %#v", out)
}
// Finally, make sure that another attempt is rejected.
err = msgpackrpc.CallWithCodec(codec, "ACL.Bootstrap", &arg, &out)
if err.Error() != structs.ACLBootstrapNotAllowedErr.Error() {
t.Fatalf("err: %v", err)
}
}
func TestACLEndpoint_Apply(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {

View File

@ -191,16 +191,6 @@ type Config struct {
// operators track which versions are actively deployed
Build string
// ACLToken is the default token to use when making a request.
// If not provided, the anonymous token is used. This enables
// backwards compatibility as well.
ACLToken string
// ACLAgentToken is the default token used to make requests for the agent
// itself, such as for registering itself with the catalog. If not
// configured, the ACLToken will be used.
ACLAgentToken string
// ACLMasterToken is used to bootstrap the ACL system. It should be specified
// on the servers in the ACLDatacenter. When the leader comes online, it ensures
// that the Master token is available. This provides the initial token.

View File

@ -15,6 +15,13 @@ import (
"github.com/hashicorp/raft"
)
// TODO (slackpad) - There are two refactors we should do here:
//
// 1. Register the different types from the state store and make the FSM more
// generic, especially around snapshot/restore. Those should really just
// pass the encoder into a WriteSnapshot() kind of method.
// 2. Check all the error return values from all the Write() calls.
// msgpackHandle is a shared handle for encoding/decoding msgpack payloads
var msgpackHandle = &codec.MsgpackHandle{}
@ -231,6 +238,17 @@ func (c *consulFSM) applyACLOperation(buf []byte, index uint64) interface{} {
}
defer metrics.MeasureSince([]string{"consul", "fsm", "acl", string(req.Op)}, time.Now())
switch req.Op {
case structs.ACLBootstrapInit:
enabled, err := c.state.ACLBootstrapInit(index)
if err != nil {
return err
}
return enabled
case structs.ACLBootstrapNow:
if err := c.state.ACLBootstrap(index, &req.ACL); err != nil {
return err
}
return &req.ACL
case structs.ACLForceSet, structs.ACLSet:
if err := c.state.ACLSet(index, &req.ACL); err != nil {
return err
@ -423,6 +441,15 @@ func (c *consulFSM) Restore(old io.ReadCloser) error {
return err
}
case structs.ACLBootstrapRequestType:
var req structs.ACLBootstrap
if err := dec.Decode(&req); err != nil {
return err
}
if err := restore.ACLBootstrap(&req); err != nil {
return err
}
case structs.CoordinateBatchUpdateType:
var req structs.Coordinates
if err := dec.Decode(&req); err != nil {
@ -623,6 +650,18 @@ func (s *consulSnapshot) persistACLs(sink raft.SnapshotSink,
return err
}
}
bs, err := s.state.ACLBootstrap()
if err != nil {
return err
}
if bs != nil {
sink.Write([]byte{byte(structs.ACLBootstrapRequestType)})
if err := encoder.Encode(bs); err != nil {
return err
}
}
return nil
}

View File

@ -6,7 +6,6 @@ import (
"os"
"reflect"
"testing"
"time"
"github.com/hashicorp/consul/agent/consul/state"
@ -16,6 +15,7 @@ import (
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-uuid"
"github.com/hashicorp/raft"
"github.com/pascaldekloe/goe/verify"
)
type MockSink struct {
@ -390,6 +390,9 @@ func TestFSM_SnapshotRestore(t *testing.T) {
fsm.state.SessionCreate(9, session)
acl := &structs.ACL{ID: generateUUID(), Name: "User Token"}
fsm.state.ACLSet(10, acl)
if _, err := fsm.state.ACLBootstrapInit(10); err != nil {
t.Fatalf("err: %v", err)
}
fsm.state.KVSSet(11, &structs.DirEntry{
Key: "/remove",
@ -540,6 +543,18 @@ func TestFSM_SnapshotRestore(t *testing.T) {
if a.ModifyIndex <= 1 {
t.Fatalf("bad index: %d", idx)
}
gotB, err := fsm2.state.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: true,
RaftIndex: structs.RaftIndex{
CreateIndex: 10,
ModifyIndex: 10,
},
}
verify.Values(t, "", gotB, wantB)
// Verify tombstones are restored
func() {
@ -1093,14 +1108,14 @@ func TestFSM_KVSUnlock(t *testing.T) {
}
}
func TestFSM_ACL_Set_Delete(t *testing.T) {
func TestFSM_ACL_CRUD(t *testing.T) {
t.Parallel()
fsm, err := NewFSM(nil, os.Stderr)
if err != nil {
t.Fatalf("err: %v", err)
}
// Create a new ACL
// Create a new ACL.
req := structs.ACLRequest{
Datacenter: "dc1",
Op: structs.ACLSet,
@ -1119,7 +1134,7 @@ func TestFSM_ACL_Set_Delete(t *testing.T) {
t.Fatalf("resp: %v", err)
}
// Get the ACL
// Get the ACL.
id := resp.(string)
_, acl, err := fsm.state.ACLGet(nil, id)
if err != nil {
@ -1129,7 +1144,7 @@ func TestFSM_ACL_Set_Delete(t *testing.T) {
t.Fatalf("missing")
}
// Verify the ACL
// Verify the ACL.
if acl.ID != id {
t.Fatalf("bad: %v", *acl)
}
@ -1140,7 +1155,7 @@ func TestFSM_ACL_Set_Delete(t *testing.T) {
t.Fatalf("bad: %v", *acl)
}
// Try to destroy
// Try to destroy.
destroy := structs.ACLRequest{
Datacenter: "dc1",
Op: structs.ACLDelete,
@ -1164,6 +1179,53 @@ func TestFSM_ACL_Set_Delete(t *testing.T) {
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)
}
gotB, err := fsm.state.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: true,
RaftIndex: gotB.RaftIndex,
}
verify.Values(t, "", gotB, wantB)
// Do a bootstrap.
bootstrap := structs.ACLRequest{
Datacenter: "dc1",
Op: structs.ACLBootstrapNow,
ACL: structs.ACL{
ID: generateUUID(),
Name: "Bootstrap Token",
Type: structs.ACLTypeManagement,
},
}
buf, err = structs.Encode(structs.ACLRequestType, bootstrap)
if err != nil {
t.Fatalf("err: %v", err)
}
resp = fsm.Apply(makeLog(buf))
respACL, ok := resp.(*structs.ACL)
if !ok {
t.Fatalf("resp: %v", resp)
}
bootstrap.ACL.CreateIndex = respACL.CreateIndex
bootstrap.ACL.ModifyIndex = respACL.ModifyIndex
verify.Values(t, "", respACL, &bootstrap.ACL)
}
func TestFSM_PreparedQuery_CRUD(t *testing.T) {

View File

@ -13,6 +13,7 @@ import (
"github.com/hashicorp/consul/agent/consul/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-version"
"github.com/hashicorp/raft"
"github.com/hashicorp/serf/serf"
)
@ -105,7 +106,11 @@ RECONCILE:
goto WAIT
}
establishedLeader = true
defer s.revokeLeadership()
defer func() {
if err := s.revokeLeadership(); err != nil {
s.logger.Printf("[ERR] consul: failed to revoke leadership: %v", err)
}
}()
}
// Reconcile any missing data
@ -144,19 +149,18 @@ WAIT:
// previously inflight transactions have been committed and that our
// state is up-to-date.
func (s *Server) establishLeadership() error {
// This will create the anonymous token and master token (if that is
// configured).
if err := s.initializeACL(); err != nil {
return err
}
// Hint the tombstone expiration timer. When we freshly establish leadership
// we become the authoritative timer, and so we need to start the clock
// on any pending GC events.
s.tombstoneGC.SetEnabled(true)
lastIndex := s.raft.LastIndex()
s.tombstoneGC.Hint(lastIndex)
s.logger.Printf("[DEBUG] consul: reset tombstone GC to index %d", lastIndex)
// Setup ACLs if we are the leader and need to
if err := s.initializeACL(); err != nil {
s.logger.Printf("[ERR] consul: ACL initialization failed: %v", err)
return err
}
// Setup the session timers. This is done both when starting up or when
// a leader fail over happens. Since the timers are maintained by the leader
@ -168,18 +172,12 @@ func (s *Server) establishLeadership() error {
// are available to be initialized. Otherwise initialization may use stale
// data.
if err := s.initializeSessionTimers(); err != nil {
s.logger.Printf("[ERR] consul: Session Timers initialization failed: %v",
err)
return err
}
// Setup autopilot config if we need to
s.getOrCreateAutopilotConfig()
s.startAutopilot()
s.setConsistentReadReady()
return nil
}
@ -192,38 +190,33 @@ func (s *Server) revokeLeadership() error {
// Clear the session timers on either shutdown or step down, since we
// are no longer responsible for session expirations.
if err := s.clearAllSessionTimers(); err != nil {
s.logger.Printf("[ERR] consul: Clearing session timers failed: %v", err)
return err
}
s.resetConsistentReadReady()
s.stopAutopilot()
return nil
}
// initializeACL is used to setup the ACLs if we are the leader
// and need to do this.
func (s *Server) initializeACL() error {
// Bail if not configured or we are not authoritative
// Bail if not configured or we are not authoritative.
authDC := s.config.ACLDatacenter
if len(authDC) == 0 || authDC != s.config.Datacenter {
return nil
}
// Purge the cache, since it could've changed while we
// were not the leader
// Purge the cache, since it could've changed while we were not the
// leader.
s.aclAuthCache.Purge()
// Look for the anonymous token
// Create anonymous token if missing.
state := s.fsm.State()
_, acl, err := state.ACLGet(nil, anonymousToken)
if err != nil {
return fmt.Errorf("failed to get anonymous token: %v", err)
}
// Create anonymous token if missing
if acl == nil {
req := structs.ACLRequest{
Datacenter: authDC,
@ -240,33 +233,69 @@ func (s *Server) initializeACL() error {
}
}
// Check for configured master token
master := s.config.ACLMasterToken
if len(master) == 0 {
return nil
}
// Look for the master token
_, acl, err = state.ACLGet(nil, master)
if err != nil {
return fmt.Errorf("failed to get master token: %v", err)
}
if acl == nil {
req := structs.ACLRequest{
Datacenter: authDC,
Op: structs.ACLSet,
ACL: structs.ACL{
ID: master,
Name: "Master Token",
Type: structs.ACLTypeManagement,
},
}
_, err := s.raftApply(structs.ACLRequestType, &req)
// Check for configured master token.
if master := s.config.ACLMasterToken; len(master) > 0 {
_, acl, err = state.ACLGet(nil, master)
if err != nil {
return fmt.Errorf("failed to create master token: %v", err)
return fmt.Errorf("failed to get master token: %v", err)
}
if acl == nil {
req := structs.ACLRequest{
Datacenter: authDC,
Op: structs.ACLSet,
ACL: structs.ACL{
ID: master,
Name: "Master Token",
Type: structs.ACLTypeManagement,
},
}
_, err := s.raftApply(structs.ACLRequestType, &req)
if err != nil {
return fmt.Errorf("failed to create master token: %v", err)
}
s.logger.Printf("[INFO] consul: 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 ServersMeetMinimumVersion(s.LANMembers(), minVersion) {
bs, err := state.ACLGetBootstrap()
if err != nil {
return fmt.Errorf("failed looking for ACL bootstrap info: %v", err)
}
if bs == nil {
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 error:
return fmt.Errorf("failed to initialize ACL bootstrap: %v", v)
case bool:
if v {
s.logger.Printf("[INFO] consul: ACL bootstrap enabled")
} else {
s.logger.Printf("[INFO] consul: ACL bootstrap disabled, existing management tokens found")
}
default:
return fmt.Errorf("unexpected response trying to initialize ACL bootstrap: %T", v)
}
}
} else {
s.logger.Printf("[WARN] consul: Can't initialize ACL bootstrap until all servers are >= %s", minVersion.String())
}
return nil
}

View File

@ -846,3 +846,70 @@ func TestLeader_ChangeServerID(t *testing.T) {
retry.Run(t, func(r *retry.R) { r.Check(wantPeers(s, 3)) })
}
}
func TestLeader_ACL_Initialization(t *testing.T) {
t.Parallel()
tests := []struct {
name string
build string
master string
init bool
bootstrap bool
}{
{"old version, no master", "0.8.0", "", false, false},
{"old version, master", "0.8.0", "root", false, false},
{"new version, no master", "0.9.1", "", true, true},
{"new version, master", "0.9.1", "root", true, false},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
conf := func(c *Config) {
c.Build = tt.build
c.Bootstrap = true
c.Datacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLMasterToken = tt.master
}
dir1, s1 := testServerWithConfig(t, conf)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
if tt.master != "" {
_, master, err := s1.fsm.State().ACLGet(nil, tt.master)
if err != nil {
t.Fatalf("err: %v", err)
}
if master == nil {
t.Fatalf("master token wasn't created")
}
}
_, anon, err := s1.fsm.State().ACLGet(nil, anonymousToken)
if err != nil {
t.Fatalf("err: %v", err)
}
if anon == nil {
t.Fatalf("anonymous token wasn't created")
}
bs, err := s1.fsm.State().ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
if !tt.init {
if bs != nil {
t.Fatalf("bootstrap should not be initialized")
}
} else {
if bs == nil {
t.Fatalf("bootstrap should be initialized")
}
if got, want := bs.AllowBootstrap, tt.bootstrap; got != want {
t.Fatalf("got %v want %v", got, want)
}
}
})
}
}

View File

@ -7,6 +7,43 @@ import (
"github.com/hashicorp/go-memdb"
)
// aclsTableSchema returns a new table schema used for storing ACL tokens.
func aclsTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acls",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "ID",
Lowercase: false,
},
},
},
}
}
// aclsBootstrapTableSchema returns a new schema used for tracking the ACL
// bootstrap status for a cluster. This is designed to have only a single
// row, so it has a somewhat unusual no-op indexer.
func aclsBootstrapTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acls-bootstrap",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: true,
Unique: true,
Indexer: &memdb.ConditionalIndex{
Conditional: func(obj interface{}) (bool, error) { return true, nil },
},
},
},
}
}
// ACLs is used to pull all the ACLs from the snapshot.
func (s *Snapshot) ACLs() (memdb.ResultIterator, error) {
iter, err := s.tx.Get("acls", "id")
@ -25,10 +62,186 @@ func (s *Restore) ACL(acl *structs.ACL) error {
if err := indexUpdateMaxTxn(s.tx, acl.ModifyIndex, "acls"); err != nil {
return fmt.Errorf("failed updating index: %s", err)
}
return nil
}
// ACLBootstrap is used to pull the ACL bootstrap info from the snapshot. This
// might return nil, in which case nothing should be saved to the snapshot.
func (s *Snapshot) ACLBootstrap() (*structs.ACLBootstrap, error) {
existing, err := s.tx.First("acls-bootstrap", "id")
if err != nil {
return nil, fmt.Errorf("failed acl bootstrap lookup: %s", err)
}
if existing != nil {
return existing.(*structs.ACLBootstrap), nil
}
return nil, nil
}
// ACLBootstrap is used to restore the ACL bootstrap info from the snapshot.
func (s *Restore) ACLBootstrap(bs *structs.ACLBootstrap) error {
if err := s.tx.Insert("acls-bootstrap", bs); err != nil {
return fmt.Errorf("failed updating acl bootstrap: %v", err)
}
return nil
}
// 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. This is because the state store
// snapshots would become incompatible with older agents if we added this on
// the fly, so we rely on the leader to determine a safe time to add this so
// we can start tracking whether bootstrap is enabled. This will return an
// error if bootstrap is already initialized.
//
// This returns a boolean indicating if ACL boostrapping is enabled.
func (s *Store) ACLBootstrapInit(idx uint64) (bool, error) {
tx := s.db.Txn(true)
defer tx.Abort()
// Don't allow this to happen more than once.
existing, err := tx.First("acls-bootstrap", "id")
if err != nil {
return false, fmt.Errorf("failed acl bootstrap lookup: %s", err)
}
if existing != nil {
return false, fmt.Errorf("acl bootstrap init already done")
}
// See if there are any management tokens, which means we shouldn't
// allow bootstrapping.
foundMgmt, err := s.aclHasManagementTokensTxn(tx)
if err != nil {
return false, fmt.Errorf("failed checking for management tokens: %v", err)
}
allowBootstrap := !foundMgmt
// Create a new bootstrap record.
bs := structs.ACLBootstrap{
AllowBootstrap: allowBootstrap,
RaftIndex: structs.RaftIndex{
CreateIndex: idx,
ModifyIndex: idx,
},
}
if err := tx.Insert("acls-bootstrap", &bs); err != nil {
return false, fmt.Errorf("failed creating acl bootstrap: %v", err)
}
tx.Commit()
return allowBootstrap, nil
}
// ACLBootstrap is used to perform a one-time ACL bootstrap operation on a
// cluster to get the first management token.
func (s *Store) ACLBootstrap(idx uint64, acl *structs.ACL) error {
tx := s.db.Txn(true)
defer tx.Abort()
// We must have initialized before this will ever be possible.
existing, err := tx.First("acls-bootstrap", "id")
if err != nil {
return fmt.Errorf("failed acl bootstrap lookup: %s", err)
}
if existing == nil {
return structs.ACLBootstrapNotInitializedErr
}
// See if this cluster has already been bootstrapped.
bs := *existing.(*structs.ACLBootstrap)
if !bs.AllowBootstrap {
return structs.ACLBootstrapNotAllowedErr
}
// This should not be required since we keep the boolean above in sync
// with any new management tokens that are added, but since this is such
// a critical thing for correct operation we perform a sanity check.
foundMgmt, err := s.aclHasManagementTokensTxn(tx)
if err != nil {
return fmt.Errorf("failed checking for management tokens: %v", err)
}
if foundMgmt {
return fmt.Errorf("internal error: acl bootstrap enabled but existing management tokens were found")
}
// Bootstrap and then make sure we disable bootstrapping forever. The
// set will also disable this as a side effect but we want to be super
// explicit here.
if err := s.aclSetTxn(tx, idx, acl); err != nil {
return fmt.Errorf("failed inserting bootstrap token: %v", err)
}
if disabled, err := s.aclDisableBootstrapTxn(tx, idx); err != nil || !disabled {
return fmt.Errorf("failed to disable acl bootstrap (disabled=%v): %v", disabled, err)
}
tx.Commit()
return nil
}
// aclDisableBootstrapTxn will disable ACL bootstrapping if the bootstrap init
// has been completed and bootstrap is currently enabled. This will return true
// if bootstrap is disabled.
func (s *Store) aclDisableBootstrapTxn(tx *memdb.Txn, idx uint64) (bool, error) {
// If the init hasn't been done then we aren't tracking this yet, so we
// can bail out. When the init is done for the first time it will scan
// for management tokens to set the initial state correctly.
existing, err := tx.First("acls-bootstrap", "id")
if err != nil {
return false, fmt.Errorf("failed acl bootstrap lookup: %s", err)
}
if existing == nil {
// Not yet init-ed, nothing to do.
return false, nil
}
// See if bootstrap is already disabled, which is the common case, so we
// can avoid a spurious write. We do a copy here in case we need to write
// down below, though.
bs := *existing.(*structs.ACLBootstrap)
if !bs.AllowBootstrap {
return true, nil
}
// Need to disable bootstrap!
bs.AllowBootstrap = false
bs.ModifyIndex = idx
if err := tx.Insert("acls-bootstrap", &bs); err != nil {
return false, fmt.Errorf("failed updating acl bootstrap: %v", err)
}
return true, nil
}
// aclHasManagementTokensTxn returns true if any management tokens are present
// in the state store.
func (s *Store) aclHasManagementTokensTxn(tx *memdb.Txn) (bool, error) {
iter, err := tx.Get("acls", "id")
if err != nil {
return false, fmt.Errorf("failed acl lookup: %s", err)
}
for acl := iter.Next(); acl != nil; acl = iter.Next() {
if acl.(*structs.ACL).Type == structs.ACLTypeManagement {
return true, nil
}
}
return false, nil
}
// ACLGetBootstrap returns the ACL bootstrap status for the cluster, which might
// be nil if it hasn't yet been initialized.
func (s *Store) ACLGetBootstrap() (*structs.ACLBootstrap, error) {
tx := s.db.Txn(false)
defer tx.Abort()
existing, err := tx.First("acls-bootstrap", "id")
if err != nil {
return nil, fmt.Errorf("failed acl bootstrap lookup: %s", err)
}
if existing != nil {
return existing.(*structs.ACLBootstrap), nil
}
return nil, nil
}
// ACLSet is used to insert an ACL rule into the state store.
func (s *Store) ACLSet(idx uint64, acl *structs.ACL) error {
tx := s.db.Txn(true)
@ -74,6 +287,13 @@ func (s *Store) aclSetTxn(tx *memdb.Txn, idx uint64, acl *structs.ACL) error {
return fmt.Errorf("failed updating index: %s", err)
}
// If this is a management token, make sure bootstrapping gets disabled.
if acl.Type == structs.ACLTypeManagement {
if _, err := s.aclDisableBootstrapTxn(tx, idx); err != nil {
return fmt.Errorf("failed disabling acl bootstrapping: %v", err)
}
}
return nil
}

View File

@ -6,8 +6,295 @@ import (
"github.com/hashicorp/consul/agent/consul/structs"
"github.com/hashicorp/go-memdb"
"github.com/pascaldekloe/goe/verify"
)
func TestStateStore_ACLBootstrap(t *testing.T) {
acl1 := &structs.ACL{
ID: "03f43a07-7e78-1f72-6c72-5a4e3b1ac3df",
Type: structs.ACLTypeManagement,
}
acl2 := &structs.ACL{
ID: "0546a993-aa7a-741e-fb7f-09159ae56ec1",
Type: structs.ACLTypeManagement,
}
setup := func() *Store {
s := testStateStore(t)
// The clean state store should initially have no bootstrap record.
bs, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
if bs != nil {
t.Fatalf("bad: %#v", bs)
}
// Make sure that a bootstrap attempt fails in this state.
if err := s.ACLBootstrap(1, acl1); err != structs.ACLBootstrapNotInitializedErr {
t.Fatalf("err: %v", err)
}
_, gotA, err := s.ACLList(nil)
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", gotA, structs.ACLs{})
// Initialize bootstrapping.
enabled, err := s.ACLBootstrapInit(2)
if err != nil {
t.Fatalf("err: %v", err)
}
if !enabled {
t.Fatalf("bad")
}
// Read it back.
gotB, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: true,
RaftIndex: structs.RaftIndex{
CreateIndex: 2,
ModifyIndex: 2,
},
}
verify.Values(t, "", gotB, wantB)
return s
}
// This is the bootstrap happy path.
t.Run("bootstrap", func(t *testing.T) {
s := setup()
// Perform a regular bootstrap.
if err := s.ACLBootstrap(3, acl1); err != nil {
t.Fatalf("err: %v", err)
}
// Read it back.
gotB, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: false,
RaftIndex: structs.RaftIndex{
CreateIndex: 2,
ModifyIndex: 3,
},
}
verify.Values(t, "", gotB, wantB)
// Make sure another attempt fails.
if err := s.ACLBootstrap(4, acl2); err != structs.ACLBootstrapNotAllowedErr {
t.Fatalf("err: %v", err)
}
// Check that the bootstrap state remains the same.
gotB, err = s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", gotB, wantB)
// Make sure the ACLs are in an expected state.
_, gotA, err := s.ACLList(nil)
if err != nil {
t.Fatalf("err: %v", err)
}
wantA := structs.ACLs{
&structs.ACL{
ID: acl1.ID,
Type: acl1.Type,
RaftIndex: structs.RaftIndex{
CreateIndex: 3,
ModifyIndex: 3,
},
},
}
verify.Values(t, "", gotA, wantA)
})
// This case initialized bootstrap but it gets canceled because a
// management token gets created manually.
t.Run("bootstrap canceled", func(t *testing.T) {
s := setup()
// Make a management token manually.
if err := s.ACLSet(3, acl1); err != nil {
t.Fatalf("err: %v", err)
}
// Bootstrapping should have gotten disabled.
gotB, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: false,
RaftIndex: structs.RaftIndex{
CreateIndex: 2,
ModifyIndex: 3,
},
}
verify.Values(t, "", gotB, wantB)
// Make sure another attempt fails.
if err := s.ACLBootstrap(4, acl2); err != structs.ACLBootstrapNotAllowedErr {
t.Fatalf("err: %v", err)
}
// Check that the bootstrap state remains the same.
gotB, err = s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", gotB, wantB)
// Make sure the ACLs are in an expected state.
_, gotA, err := s.ACLList(nil)
if err != nil {
t.Fatalf("err: %v", err)
}
wantA := structs.ACLs{
&structs.ACL{
ID: acl1.ID,
Type: acl1.Type,
RaftIndex: structs.RaftIndex{
CreateIndex: 3,
ModifyIndex: 3,
},
},
}
verify.Values(t, "", gotA, wantA)
})
}
func TestStateStore_ACLBootstrap_InitialTokens(t *testing.T) {
acl1 := &structs.ACL{
ID: "03f43a07-7e78-1f72-6c72-5a4e3b1ac3df",
Type: structs.ACLTypeManagement,
}
acl2 := &structs.ACL{
ID: "0546a993-aa7a-741e-fb7f-09159ae56ec1",
Type: structs.ACLTypeManagement,
}
s := testStateStore(t)
// Make a management token manually. This also makes sure that it's ok
// to set a token if bootstrap has not been initialized.
if err := s.ACLSet(1, acl1); err != nil {
t.Fatalf("err: %v", err)
}
// Initialize bootstrapping, which should not be enabled since an
// existing token is present.
enabled, err := s.ACLBootstrapInit(2)
if err != nil {
t.Fatalf("err: %v", err)
}
if enabled {
t.Fatalf("bad")
}
// Read it back.
gotB, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: false,
RaftIndex: structs.RaftIndex{
CreateIndex: 2,
ModifyIndex: 2,
},
}
verify.Values(t, "", gotB, wantB)
// Make sure an attempt fails.
if err := s.ACLBootstrap(3, acl2); err != structs.ACLBootstrapNotAllowedErr {
t.Fatalf("err: %v", err)
}
// Check that the bootstrap state remains the same.
gotB, err = s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", gotB, wantB)
// Make sure the ACLs are in an expected state.
_, gotA, err := s.ACLList(nil)
if err != nil {
t.Fatalf("err: %v", err)
}
wantA := structs.ACLs{
&structs.ACL{
ID: acl1.ID,
Type: acl1.Type,
RaftIndex: structs.RaftIndex{
CreateIndex: 1,
ModifyIndex: 1,
},
},
}
verify.Values(t, "", gotA, wantA)
}
func TestStateStore_ACLBootstrap_Snapshot_Restore(t *testing.T) {
s := testStateStore(t)
enabled, err := s.ACLBootstrapInit(1)
if err != nil {
t.Fatalf("err: %v", err)
}
if !enabled {
t.Fatalf("bad")
}
gotB, err := s.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
wantB := &structs.ACLBootstrap{
AllowBootstrap: true,
RaftIndex: structs.RaftIndex{
CreateIndex: 1,
ModifyIndex: 1,
},
}
verify.Values(t, "", gotB, wantB)
snap := s.Snapshot()
defer snap.Close()
bs, err := snap.ACLBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", bs, wantB)
r := testStateStore(t)
restore := r.Restore()
if err := restore.ACLBootstrap(bs); err != nil {
t.Fatalf("err: %v", err)
}
restore.Commit()
gotB, err = r.ACLGetBootstrap()
if err != nil {
t.Fatalf("err: %v", err)
}
verify.Values(t, "", gotB, wantB)
}
func TestStateStore_ACLSet_ACLGet(t *testing.T) {
s := testStateStore(t)

View File

@ -29,6 +29,7 @@ func stateStoreSchema() *memdb.DBSchema {
sessionsTableSchema,
sessionChecksTableSchema,
aclsTableSchema,
aclsBootstrapTableSchema,
coordinatesTableSchema,
preparedQueriesTableSchema,
autopilotConfigTableSchema,
@ -363,25 +364,6 @@ func sessionChecksTableSchema() *memdb.TableSchema {
}
}
// aclsTableSchema returns a new table schema used for
// storing ACL information.
func aclsTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: "acls",
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "ID",
Lowercase: false,
},
},
},
}
}
// coordinatesTableSchema returns a new table schema used for storing
// network coordinates.
func coordinatesTableSchema() *memdb.TableSchema {

155
agent/consul/structs/acl.go Normal file
View File

@ -0,0 +1,155 @@
package structs
import (
"errors"
"time"
"github.com/hashicorp/consul/acl"
)
// ACLOp is used in RPCs to encode ACL operations.
type ACLOp string
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.
ACLBootstrapInit = "bootstrap-init"
// ACLBootstrapNow is used to perform a one-time ACL bootstrap operation on
// a cluster to get the first management token.
ACLBootstrapNow = "bootstrap-now"
// ACLSet creates or updates a token.
ACLSet ACLOp = "set"
// ACLForceSet is deprecated, but left for backwards compatibility.
ACLForceSet = "force-set"
// ACLDelete deletes a token.
ACLDelete = "delete"
)
// ACLBootstrapNotInitializedErr is returned when a bootstrap is attempted but
// we haven't yet initialized ACL bootstrap. It provides some guidance to
// operators on how to proceed.
var ACLBootstrapNotInitializedErr = errors.New("ACL bootstrap not initialized, need to force a leader election and ensure all Consul servers support this feature")
// ACLBootstrapNotAllowedErr is returned once we know that a bootstrap can no
// longer be done since the cluster was bootstrapped, or a management token
// was created manually.
var ACLBootstrapNotAllowedErr = errors.New("ACL bootstrap no longer allowed")
const (
// ACLTypeClient tokens have rules applied
ACLTypeClient = "client"
// ACLTypeManagement tokens have an always allow policy, so they can
// make other tokens and can access all resources.
ACLTypeManagement = "management"
)
// ACL is used to represent a token and its rules
type ACL struct {
ID string
Name string
Type string
Rules string
RaftIndex
}
// ACLs is a slice of ACLs.
type ACLs []*ACL
// IsSame checks if one ACL is the same as another, without looking
// at the Raft information (that's why we didn't call it IsEqual). This is
// useful for seeing if an update would be idempotent for all the functional
// parts of the structure.
func (a *ACL) IsSame(other *ACL) bool {
if a.ID != other.ID ||
a.Name != other.Name ||
a.Type != other.Type ||
a.Rules != other.Rules {
return false
}
return true
}
// ACLBootstrap keeps track of whether bootstrapping ACLs is allowed for a
// cluster.
type ACLBootstrap struct {
// AllowBootstrap will only be true if no existing management tokens
// have been found.
AllowBootstrap bool
RaftIndex
}
// 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
type ACLSpecificRequest struct {
Datacenter string
ACL string
QueryOptions
}
// RequestDatacenter returns the DC this request is targeted to.
func (r *ACLSpecificRequest) RequestDatacenter() string {
return r.Datacenter
}
// ACLPolicyRequest is used to request an ACL by ID, conditionally
// filtering on an ID
type ACLPolicyRequest struct {
Datacenter string
ACL string
ETag string
QueryOptions
}
// RequestDatacenter returns the DC this request is targeted to.
func (r *ACLPolicyRequest) RequestDatacenter() string {
return r.Datacenter
}
// IndexedACLs has tokens along with the Raft metadata about them.
type IndexedACLs struct {
ACLs ACLs
QueryMeta
}
// ACLPolicy is a policy that can be associated with a token.
type ACLPolicy struct {
ETag string
Parent string
Policy *acl.Policy
TTL time.Duration
QueryMeta
}
// ACLReplicationStatus provides information about the health of the ACL
// replication system.
type ACLReplicationStatus struct {
Enabled bool
Running bool
SourceDatacenter string
ReplicatedIndex uint64
LastSuccess time.Time
LastError time.Time
}

View File

@ -0,0 +1,52 @@
package structs
import (
"testing"
)
func TestStructs_ACL_IsSame(t *testing.T) {
acl := &ACL{
ID: "guid",
Name: "An ACL for testing",
Type: "client",
Rules: "service \"\" { policy = \"read\" }",
}
if !acl.IsSame(acl) {
t.Fatalf("should be equal to itself")
}
other := &ACL{
ID: "guid",
Name: "An ACL for testing",
Type: "client",
Rules: "service \"\" { policy = \"read\" }",
RaftIndex: RaftIndex{
CreateIndex: 1,
ModifyIndex: 2,
},
}
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should not care about Raft fields")
}
check := func(twiddle, restore func()) {
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should be the same")
}
twiddle()
if acl.IsSame(other) || other.IsSame(acl) {
t.Fatalf("should not be the same")
}
restore()
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should be the same")
}
}
check(func() { other.ID = "nope" }, func() { other.ID = "guid" })
check(func() { other.Name = "nope" }, func() { other.Name = "An ACL for testing" })
check(func() { other.Type = "management" }, func() { other.Type = "client" })
check(func() { other.Rules = "" }, func() { other.Rules = "service \"\" { policy = \"read\" }" })
}

View File

@ -9,7 +9,6 @@ import (
"strings"
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-msgpack/codec"
@ -32,18 +31,21 @@ type RaftIndex struct {
ModifyIndex uint64
}
// These are serialized between Consul servers and stored in Consul snapshots,
// so entries must only ever be added.
const (
RegisterRequestType MessageType = iota
DeregisterRequestType
KVSRequestType
SessionRequestType
ACLRequestType
TombstoneRequestType
CoordinateBatchUpdateType
PreparedQueryRequestType
TxnRequestType
AutopilotRequestType
AreaRequestType
RegisterRequestType MessageType = 0
DeregisterRequestType = 1
KVSRequestType = 2
SessionRequestType = 3
ACLRequestType = 4
TombstoneRequestType = 5
CoordinateBatchUpdateType = 6
PreparedQueryRequestType = 7
TxnRequestType = 8
AutopilotRequestType = 9
AreaRequestType = 10
ACLBootstrapRequestType = 11 // FSM snapshots only.
)
const (
@ -72,13 +74,6 @@ const (
// metaValueMaxLength is the maximum allowed length of a metadata value
metaValueMaxLength = 512
// Client tokens have rules applied
ACLTypeClient = "client"
// Management tokens have an always allow policy.
// They are used for token management.
ACLTypeManagement = "management"
// MaxLockDelay provides a maximum LockDelay value for
// a session. Any value above this will not be respected.
MaxLockDelay = 60 * time.Second
@ -750,103 +745,6 @@ type IndexedSessions struct {
QueryMeta
}
// ACL is used to represent a token and its rules
type ACL struct {
ID string
Name string
Type string
Rules string
RaftIndex
}
type ACLs []*ACL
type ACLOp string
const (
ACLSet ACLOp = "set"
ACLForceSet = "force-set" // Deprecated, left to backwards compatibility
ACLDelete = "delete"
)
// IsSame checks if one ACL is the same as another, without looking
// at the Raft information (that's why we didn't call it IsEqual). This is
// useful for seeing if an update would be idempotent for all the functional
// parts of the structure.
func (a *ACL) IsSame(other *ACL) bool {
if a.ID != other.ID ||
a.Name != other.Name ||
a.Type != other.Type ||
a.Rules != other.Rules {
return false
}
return true
}
// 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
type ACLSpecificRequest struct {
Datacenter string
ACL string
QueryOptions
}
func (r *ACLSpecificRequest) RequestDatacenter() string {
return r.Datacenter
}
// ACLPolicyRequest is used to request an ACL by ID, conditionally
// filtering on an ID
type ACLPolicyRequest struct {
Datacenter string
ACL string
ETag string
QueryOptions
}
func (r *ACLPolicyRequest) RequestDatacenter() string {
return r.Datacenter
}
type IndexedACLs struct {
ACLs ACLs
QueryMeta
}
type ACLPolicy struct {
ETag string
Parent string
Policy *acl.Policy
TTL time.Duration
QueryMeta
}
// ACLReplicationStatus provides information about the health of the ACL
// replication system.
type ACLReplicationStatus struct {
Enabled bool
Running bool
SourceDatacenter string
ReplicatedIndex uint64
LastSuccess time.Time
LastError time.Time
}
// Coordinate stores a node name with its associated network coordinate.
type Coordinate struct {
Node string

View File

@ -59,53 +59,6 @@ func TestStructs_Implements(t *testing.T) {
)
}
func TestStructs_ACL_IsSame(t *testing.T) {
acl := &ACL{
ID: "guid",
Name: "An ACL for testing",
Type: "client",
Rules: "service \"\" { policy = \"read\" }",
}
if !acl.IsSame(acl) {
t.Fatalf("should be equal to itself")
}
other := &ACL{
ID: "guid",
Name: "An ACL for testing",
Type: "client",
Rules: "service \"\" { policy = \"read\" }",
RaftIndex: RaftIndex{
CreateIndex: 1,
ModifyIndex: 2,
},
}
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should not care about Raft fields")
}
check := func(twiddle, restore func()) {
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should be the same")
}
twiddle()
if acl.IsSame(other) || other.IsSame(acl) {
t.Fatalf("should not be the same")
}
restore()
if !acl.IsSame(other) || !other.IsSame(acl) {
t.Fatalf("should be the same")
}
}
check(func() { other.ID = "nope" }, func() { other.ID = "guid" })
check(func() { other.Name = "nope" }, func() { other.Name = "An ACL for testing" })
check(func() { other.Type = "management" }, func() { other.Type = "client" })
check(func() { other.Rules = "" }, func() { other.Rules = "service \"\" { policy = \"read\" }" })
}
func TestStructs_RegisterRequest_ChangesNode(t *testing.T) {
req := &RegisterRequest{
ID: types.NodeID("40e4a748-2192-161a-0510-9bf59fe950b5"),

View File

@ -73,6 +73,7 @@ func (s *HTTPServer) handler(enableDebug bool) http.Handler {
// API V1.
if s.agent.config.ACLDatacenter != "" {
handleFuncMetrics("/v1/acl/bootstrap", s.wrap(s.ACLBootstrap))
handleFuncMetrics("/v1/acl/create", s.wrap(s.ACLCreate))
handleFuncMetrics("/v1/acl/update", s.wrap(s.ACLUpdate))
handleFuncMetrics("/v1/acl/destroy/", s.wrap(s.ACLDestroy))
@ -82,6 +83,7 @@ func (s *HTTPServer) handler(enableDebug bool) http.Handler {
handleFuncMetrics("/v1/acl/replication", s.wrap(s.ACLReplicationStatus))
handleFuncMetrics("/v1/agent/token/", s.wrap(s.AgentToken))
} else {
handleFuncMetrics("/v1/acl/bootstrap", s.wrap(ACLDisabled))
handleFuncMetrics("/v1/acl/create", s.wrap(ACLDisabled))
handleFuncMetrics("/v1/acl/update", s.wrap(ACLDisabled))
handleFuncMetrics("/v1/acl/destroy/", s.wrap(ACLDisabled))

View File

@ -42,6 +42,24 @@ func (c *Client) ACL() *ACL {
return &ACL{c}
}
// Bootstrap is used to perform a one-time ACL bootstrap operation on a cluster
// to get the first management token.
func (a *ACL) Bootstrap() (string, *WriteMeta, error) {
r := a.c.newRequest("PUT", "/v1/acl/bootstrap")
rtt, resp, err := requireOK(a.c.doRequest(r))
if err != nil {
return "", nil, err
}
defer resp.Body.Close()
wm := &WriteMeta{RequestTime: rtt}
var out struct{ ID string }
if err := decodeBody(resp, &out); err != nil {
return "", nil, err
}
return out.ID, wm, nil
}
// Create is used to generate a new token with the given parameters
func (a *ACL) Create(acl *ACLEntry, q *WriteOptions) (string, *WriteMeta, error) {
r := a.c.newRequest("PUT", "/v1/acl/create")

View File

@ -4,6 +4,13 @@ import (
"testing"
)
func TestAPI_ACLBootstrap(t *testing.T) {
// TODO (slackpad) We currently can't inject the version, and the
// version in the binary depends on Git tags, so we can't reliably
// test this until we are just running an agent in-process here and
// have full control over the config.
}
func TestAPI_ACLCreateDestroy(t *testing.T) {
t.Parallel()
c, s := makeACLClient(t)

View File

@ -10,6 +10,54 @@ description: |-
The `/acl` endpoints create, update, destroy, and query ACL tokens in Consul. For more information about ACLs, please see the [ACL Guide](/docs/guides/acl.html).
## Bootstrap ACLs
This endpoint does a special one-time bootstrap of the ACL system, making the first
management token if the [`acl_master_token`](/docs/agent/options.html#acl_master_token)
is not specified in the Consul server configuration, and if the cluster has not been
bootstrapped previously. This is available in Consul 0.9.1 and later, and requires all
Consul servers to be upgraded in order to operate.
This provides a mechanism to bootstrap ACLs without having any secrets present in Consul's
configuration files.
| Method | Path | Produces |
| ------ | ---------------------------- | -------------------------- |
| `PUT` | `/acl/bootstrap` | `application/json` |
The table below shows this endpoint's support for
[blocking queries](/api/index.html#blocking-queries),
[consistency modes](/api/index.html#consistency-modes), and
[required ACLs](/api/index.html#acls).
| Blocking Queries | Consistency Modes | ACL Required |
| ---------------- | ----------------- | ------------ |
| `NO` | `none` | `none` |
### Sample Request
```text
$ curl \
--request PUT \
https://consul.rocks/v1/acl/bootstrap
```
### Sample Response
```json
{
"ID": "adf4238a-882b-9ddc-4a9d-5b6758e4159e"
}
```
You can detect if something has interfered with the ACL bootstrapping process by
checking the response code. A 200 response means that the bootstrap was a success, and
a 403 means that the cluster has already been bootstrapped, at which point you should
consider the cluster in a potentially compromised state.
The returned token will be a management token which can be used to further configure the
ACL system. Please see the [ACL Guide](/docs/guides/acl.html) for more details.
## Create ACL Token
This endpoint makes a new ACL token.

View File

@ -190,7 +190,7 @@ The first step for bootstrapping ACLs is to enable ACLs on the Consul servers in
datacenter. In this example, we are configuring the following:
1. An ACL datacenter of "dc1", which is where these servers are
2. An ACL master token of "b1gs33cr3t"
2. An ACL master token of "b1gs33cr3t"; see below for an alternative using the [/v1/acl/bootstrap API](/api/acl.html#bootstrap-acls)
3. A default policy of "deny" which means we are in whitelist mode
4. A down policy of "extend-cache" which means that we will ignore token TTLs during an
outage
@ -218,6 +218,22 @@ a server acquires cluster leadership. If you would like to install or change the
[`acl_master_token`](/docs/agent/options.html#acl_master_token) in the configuration
for all servers. Once this is done, restart the current leader to force a leader election.
In Consul 0.9.1 and later, you can use the [/v1/acl/bootstrap API](/api/acl.html#bootstrap-acls)
to make the initial master token, so a token never needs to be placed into a configuration
file. To use this approach, omit `acl_master_token` from the above config and then call the API:
```text
$ curl \
--request PUT \
http://127.0.0.1:8500/v1/acl/bootstrap
{"ID":"fe3b8d40-0ee0-8783-6cc2-ab1aa9bb16c1"}
```
The returned token is the initial management token, which is randomly generated by Consul.
It's only possible to bootstrap one time, and bootstrapping will be disabled if a master
token was configured and created.
Once the ACL system is bootstrapped, ACL tokens can be managed through the
[ACL API](/api/acl.html).
@ -237,7 +253,7 @@ own internal operations like updating its node information in the catalog and pe
[anti-entropy](/docs/internals/anti-entropy.html) syncing. We can create a token using the
ACL API, and the ACL master token we set in the previous step:
```
```text
$ curl \
--request PUT \
--header "X-Consul-Token: b1gs33cr3t" \
@ -267,7 +283,7 @@ configuration and restart the servers once more to apply it:
In Consul 0.9.1 and later you can also introduce the agent token using an API,
so it doesn't need to be set in the configuration file:
```
```text
$ curl \
--request PUT \
--header "X-Consul-Token: b1gs33cr3t" \
@ -302,7 +318,7 @@ with a configuration file that enables ACLs:
Similar to the previous example, in Consul 0.9.1 and later you can also introduce the
agent token using an API, so it doesn't need to be set in the configuration file:
```
```text
$ curl \
--request PUT \
--header "X-Consul-Token: b1gs33cr3t" \
@ -354,7 +370,7 @@ configure Consul's behavior when no token is supplied. The anonymous token is ma
like any other ACL token, except that `anonymous` is used for the ID. In this example
we will give the anonymous token read privileges for all nodes:
```
```text
$ curl \
--request PUT \
--header "X-Consul-Token: b1gs33cr3t" \
@ -407,7 +423,7 @@ consul. 0 IN SOA ns.consul. postmaster.consul. 14
Now we get an `NXDOMAIN` error because the anonymous token doesn't have access to the
"consul" service. Let's add that to the anonymous token's policy:
```
```text
$ curl \
--request PUT \
--header "X-Consul-Token: b1gs33cr3t" \