connect: intentions are now managed as a new config entry kind "service-intentions" (#8834)

- Upgrade the ConfigEntry.ListAll RPC to be kind-aware so that older
copies of consul will not see new config entries it doesn't understand
replicate down.

- Add shim conversion code so that the old API/CLI method of interacting
with intentions will continue to work so long as none of these are
edited via config entry endpoints. Almost all of the read-only APIs will
continue to function indefinitely.

- Add new APIs that operate on individual intentions without IDs so that
the UI doesn't need to implement CAS operations.

- Add a new serf feature flag indicating support for
intentions-as-config-entries.

- The old line-item intentions way of interacting with the state store
will transparently flip between the legacy memdb table and the config
entry representations so that readers will never see a hiccup during
migration where the results are incomplete. It uses a piece of system
metadata to control the flip.

- The primary datacenter will begin migrating intentions into config
entries on startup once all servers in the datacenter are on a version
of Consul with the intentions-as-config-entries feature flag. When it is
complete the old state store representations will be cleared. We also
record a piece of system metadata indicating this has occurred. We use
this metadata to skip ALL of this code the next time the leader starts
up.

- The secondary datacenters continue to run the old intentions
replicator until all servers in the secondary DC and primary DC support
intentions-as-config-entries (via serf flag). Once this condition it met
the old intentions replicator ceases.

- The secondary datacenters replicate the new config entries as they are
migrated in the primary. When they detect that the primary has zeroed
it's old state store table it waits until all config entries up to that
point are replicated and then zeroes its own copy of the old state store
table. We also record a piece of system metadata indicating this has
occurred. We use this metadata to skip ALL of this code the next time
the leader starts up.
This commit is contained in:
R.B. Boyer 2020-10-06 13:24:05 -05:00 committed by GitHub
parent e657341cd8
commit a2a8e9c783
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
71 changed files with 7403 additions and 1970 deletions

6
.changelog/8834.txt Normal file
View File

@ -0,0 +1,6 @@
```release-note:feature
connect: intentions are now managed as a new config entry kind "service-intentions"
```
```release-note:breaking-change
connect: intention destinations can no longer be renamed
```

View File

@ -39,6 +39,7 @@ type configTest struct {
privatev4 func() ([]*net.IPAddr, error)
publicv6 func() ([]*net.IPAddr, error)
patch func(rt *RuntimeConfig)
patchActual func(rt *RuntimeConfig)
err string
warns []string
hostname func() (string, error)
@ -3814,6 +3815,192 @@ func TestBuilder_BuildAndValide_ConfigFlagsAndEdgecases(t *testing.T) {
},
// TODO(rb): add in missing tests for ingress-gateway (snake + camel)
// TODO(rb): add in missing tests for terminating-gateway (snake + camel)
{
desc: "ConfigEntry bootstrap service-intentions (snake-case)",
args: []string{`-data-dir=` + dataDir},
json: []string{`{
"config_entries": {
"bootstrap": [
{
"kind": "service-intentions",
"name": "web",
"meta" : {
"foo": "bar",
"gir": "zim"
},
"sources": [
{
"name": "foo",
"action": "deny",
"type": "consul",
"description": "foo desc"
},
{
"name": "bar",
"action": "allow",
"description": "bar desc"
},
{
"name": "*",
"action": "deny",
"description": "wild desc"
}
]
}
]
}
}`,
},
hcl: []string{`
config_entries {
bootstrap {
kind = "service-intentions"
name = "web"
meta {
"foo" = "bar"
"gir" = "zim"
}
sources = [
{
name = "foo"
action = "deny"
type = "consul"
description = "foo desc"
},
{
name = "bar"
action = "allow"
description = "bar desc"
}
]
sources {
name = "*"
action = "deny"
description = "wild desc"
}
}
}
`,
},
patchActual: func(rt *RuntimeConfig) {
// Wipe the time tracking fields to make comparison easier.
for _, raw := range rt.ConfigEntryBootstrap {
if entry, ok := raw.(*structs.ServiceIntentionsConfigEntry); ok {
for _, src := range entry.Sources {
src.LegacyCreateTime = nil
src.LegacyUpdateTime = nil
}
}
}
},
patch: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
rt.ConfigEntryBootstrap = []structs.ConfigEntry{
&structs.ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "web",
Meta: map[string]string{
"foo": "bar",
"gir": "zim",
},
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "foo",
Action: "deny",
Type: "consul",
Description: "foo desc",
Precedence: 9,
EnterpriseMeta: *defaultEntMeta,
},
{
Name: "bar",
Action: "allow",
Type: "consul",
Description: "bar desc",
Precedence: 9,
EnterpriseMeta: *defaultEntMeta,
},
{
Name: "*",
Action: "deny",
Type: "consul",
Description: "wild desc",
Precedence: 8,
EnterpriseMeta: *defaultEntMeta,
},
},
},
}
},
},
{
desc: "ConfigEntry bootstrap service-intentions wildcard destination (snake-case)",
args: []string{`-data-dir=` + dataDir},
json: []string{`{
"config_entries": {
"bootstrap": [
{
"kind": "service-intentions",
"name": "*",
"sources": [
{
"name": "foo",
"action": "deny",
"precedence": 6
}
]
}
]
}
}`,
},
hcl: []string{`
config_entries {
bootstrap {
kind = "service-intentions"
name = "*"
sources {
name = "foo"
action = "deny"
# should be parsed, but we'll ignore it later
precedence = 6
}
}
}
`,
},
patchActual: func(rt *RuntimeConfig) {
// Wipe the time tracking fields to make comparison easier.
for _, raw := range rt.ConfigEntryBootstrap {
if entry, ok := raw.(*structs.ServiceIntentionsConfigEntry); ok {
for _, src := range entry.Sources {
src.LegacyCreateTime = nil
src.LegacyUpdateTime = nil
}
}
}
},
patch: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
rt.ConfigEntryBootstrap = []structs.ConfigEntry{
&structs.ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "*",
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "foo",
Action: "deny",
Type: "consul",
Precedence: 6,
EnterpriseMeta: *defaultEntMeta,
},
},
},
}
},
},
///////////////////////////////////
// Defaults sanity checks
@ -4557,6 +4744,9 @@ func testConfig(t *testing.T, tests []configTest, dataDir string) {
require.Equal(t, actual.DataDir, actual.ACLTokens.DataDir)
expected.ACLTokens.DataDir = actual.ACLTokens.DataDir
if tt.patchActual != nil {
tt.patchActual(&actual)
}
require.Equal(t, expected, actual)
})
}

View File

@ -36,13 +36,14 @@ func (s *HTTPHandlers) configGet(resp http.ResponseWriter, req *http.Request) (i
switch len(pathArgs) {
case 2:
if err := s.parseEntMetaNoWildcard(req, &args.EnterpriseMeta); err != nil {
return nil, err
}
// Both kind/name provided.
args.Kind = pathArgs[0]
args.Name = pathArgs[1]
if err := s.parseEntMetaForConfigEntryKind(args.Kind, req, &args.EnterpriseMeta); err != nil {
return nil, err
}
var reply structs.ConfigEntryResponse
if err := s.agent.RPC("ConfigEntry.Get", &args, &reply); err != nil {
return nil, err
@ -95,7 +96,8 @@ func (s *HTTPHandlers) configDelete(resp http.ResponseWriter, req *http.Request)
args.Entry = entry
// Parse enterprise meta.
meta := args.Entry.GetEnterpriseMeta()
if err := s.parseEntMetaNoWildcard(req, meta); err != nil {
if err := s.parseEntMetaForConfigEntryKind(entry.GetKind(), req, meta); err != nil {
return nil, err
}
@ -128,7 +130,7 @@ func (s *HTTPHandlers) ConfigApply(resp http.ResponseWriter, req *http.Request)
// Parse enterprise meta.
var meta structs.EnterpriseMeta
if err := s.parseEntMetaNoWildcard(req, &meta); err != nil {
if err := s.parseEntMetaForConfigEntryKind(args.Entry.GetKind(), req, &meta); err != nil {
return nil, err
}
args.Entry.GetEnterpriseMeta().Merge(&meta)
@ -150,3 +152,10 @@ func (s *HTTPHandlers) ConfigApply(resp http.ResponseWriter, req *http.Request)
return reply, nil
}
func (s *HTTPHandlers) parseEntMetaForConfigEntryKind(kind string, req *http.Request, entMeta *structs.EnterpriseMeta) error {
if kind == structs.ServiceIntentions {
return s.parseEntMeta(req, entMeta)
}
return s.parseEntMetaNoWildcard(req, entMeta)
}

View File

@ -458,6 +458,11 @@ type Config struct {
// disable a background routine.
DisableFederationStateAntiEntropy bool
// OverrideInitialSerfTags solely exists for use in unit tests to ensure
// that a serf tag is initially set to a known value, rather than the
// default to test some consul upgrade scenarios with fewer races.
OverrideInitialSerfTags func(tags map[string]string)
// CAConfig is used to apply the initial Connect CA configuration when
// bootstrapping.
CAConfig *structs.CAConfiguration

View File

@ -19,6 +19,10 @@ type ConfigEntry struct {
// Apply does an upsert of the given config entry.
func (c *ConfigEntry) Apply(args *structs.ConfigEntryRequest, reply *bool) error {
return c.applyInternal(args, reply, nil)
}
func (c *ConfigEntry) applyInternal(args *structs.ConfigEntryRequest, reply *bool, normalizeAndValidateFn func(structs.ConfigEntry) error) error {
if err := c.srv.validateEnterpriseRequest(args.Entry.GetEnterpriseMeta(), true); err != nil {
return err
}
@ -38,12 +42,22 @@ func (c *ConfigEntry) Apply(args *structs.ConfigEntryRequest, reply *bool) error
return err
}
// Normalize and validate the incoming config entry.
if err := args.Entry.Normalize(); err != nil {
if err := c.preflightCheck(args.Entry.GetKind()); err != nil {
return err
}
if err := args.Entry.Validate(); err != nil {
return err
// Normalize and validate the incoming config entry as if it came from a user.
if normalizeAndValidateFn == nil {
if err := args.Entry.Normalize(); err != nil {
return err
}
if err := args.Entry.Validate(); err != nil {
return err
}
} else {
if err := normalizeAndValidateFn(args.Entry); err != nil {
return err
}
}
if authz != nil && !args.Entry.CanWrite(authz) {
@ -159,8 +173,18 @@ func (c *ConfigEntry) List(args *structs.ConfigEntryQuery, reply *structs.Indexe
})
}
var configEntryKindsFromConsul_1_8_0 = []string{
structs.ServiceDefaults,
structs.ProxyDefaults,
structs.ServiceRouter,
structs.ServiceSplitter,
structs.ServiceResolver,
structs.IngressGateway,
structs.TerminatingGateway,
}
// ListAll returns all the known configuration entries
func (c *ConfigEntry) ListAll(args *structs.DCSpecificRequest, reply *structs.IndexedGenericConfigEntries) error {
func (c *ConfigEntry) ListAll(args *structs.ConfigEntryListAllRequest, reply *structs.IndexedGenericConfigEntries) error {
if err := c.srv.validateEnterpriseRequest(&args.EnterpriseMeta, false); err != nil {
return err
}
@ -175,6 +199,15 @@ func (c *ConfigEntry) ListAll(args *structs.DCSpecificRequest, reply *structs.In
return err
}
if len(args.Kinds) == 0 {
args.Kinds = configEntryKindsFromConsul_1_8_0
}
kindMap := make(map[string]struct{})
for _, kind := range args.Kinds {
kindMap[kind] = struct{}{}
}
return c.srv.blockingQuery(
&args.QueryOptions,
&reply.QueryMeta,
@ -184,12 +217,21 @@ func (c *ConfigEntry) ListAll(args *structs.DCSpecificRequest, reply *structs.In
return err
}
// Filter the entries returned by ACL permissions.
// Filter the entries returned by ACL permissions or by the provided kinds.
filteredEntries := make([]structs.ConfigEntry, 0, len(entries))
for _, entry := range entries {
if authz != nil && !entry.CanRead(authz) {
continue
}
// Doing this filter outside of memdb isn't terribly
// performant. This kind filter is currently only used across
// version upgrades, so in the common case we are going to
// always return all of the data anyway, so it should be fine.
// If that changes at some point, then we should move this down
// into memdb.
if _, ok := kindMap[entry.GetKind()]; !ok {
continue
}
filteredEntries = append(filteredEntries, entry)
}
@ -219,6 +261,10 @@ func (c *ConfigEntry) Delete(args *structs.ConfigEntryRequest, reply *struct{})
return err
}
if err := c.preflightCheck(args.Entry.GetKind()); err != nil {
return err
}
// Normalize the incoming entry.
if err := args.Entry.Normalize(); err != nil {
return err
@ -401,3 +447,21 @@ func (c *ConfigEntry) ResolveServiceConfig(args *structs.ServiceConfigRequest, r
return nil
})
}
// preflightCheck is meant to have kind-specific system validation outside of
// content validation. The initial use case is restricting the ability to do
// writes of service-intentions until the system is finished migration.
func (c *ConfigEntry) preflightCheck(kind string) error {
switch kind {
case structs.ServiceIntentions:
usingConfigEntries, err := c.srv.fsm.State().AreIntentionsInConfigEntries()
if err != nil {
return fmt.Errorf("system metadata lookup failed: %v", err)
}
if !usingConfigEntries {
return ErrIntentionsNotUpgradedYet
}
}
return nil
}

View File

@ -368,8 +368,6 @@ func TestConfigEntry_List(t *testing.T) {
func TestConfigEntry_ListAll(t *testing.T) {
t.Parallel()
require := require.New(t)
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
@ -378,34 +376,82 @@ func TestConfigEntry_ListAll(t *testing.T) {
// Create some dummy services in the state store to look up.
state := s1.fsm.State()
expected := structs.IndexedGenericConfigEntries{
Entries: []structs.ConfigEntry{
&structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: "global",
},
&structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "bar",
},
&structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "foo",
entries := []structs.ConfigEntry{
&structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: "global",
},
&structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "bar",
},
&structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "foo",
},
&structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "api",
Sources: []*structs.SourceIntention{
{
Name: "web",
Action: structs.IntentionActionAllow,
},
},
},
}
require.NoError(state.EnsureConfigEntry(1, expected.Entries[0], nil))
require.NoError(state.EnsureConfigEntry(2, expected.Entries[1], nil))
require.NoError(state.EnsureConfigEntry(3, expected.Entries[2], nil))
require.NoError(t, state.EnsureConfigEntry(1, entries[0], nil))
require.NoError(t, state.EnsureConfigEntry(2, entries[1], nil))
require.NoError(t, state.EnsureConfigEntry(3, entries[2], nil))
require.NoError(t, state.EnsureConfigEntry(4, entries[3], nil))
args := structs.DCSpecificRequest{
Datacenter: "dc1",
}
var out structs.IndexedGenericConfigEntries
require.NoError(msgpackrpc.CallWithCodec(codec, "ConfigEntry.ListAll", &args, &out))
t.Run("all kinds", func(t *testing.T) {
args := structs.ConfigEntryListAllRequest{
Datacenter: "dc1",
Kinds: structs.AllConfigEntryKinds,
}
var out structs.IndexedGenericConfigEntries
require.NoError(t, msgpackrpc.CallWithCodec(codec, "ConfigEntry.ListAll", &args, &out))
expected.QueryMeta = out.QueryMeta
require.Equal(expected, out)
expected := structs.IndexedGenericConfigEntries{
Entries: entries[:],
QueryMeta: out.QueryMeta,
}
require.Equal(t, expected, out)
})
t.Run("all kinds pre 1.9.0", func(t *testing.T) {
args := structs.ConfigEntryListAllRequest{
Datacenter: "dc1",
Kinds: nil, // let it default
}
var out structs.IndexedGenericConfigEntries
require.NoError(t, msgpackrpc.CallWithCodec(codec, "ConfigEntry.ListAll", &args, &out))
expected := structs.IndexedGenericConfigEntries{
Entries: entries[0:3],
QueryMeta: out.QueryMeta,
}
require.Equal(t, expected, out)
})
t.Run("omit service defaults", func(t *testing.T) {
args := structs.ConfigEntryListAllRequest{
Datacenter: "dc1",
Kinds: []string{
structs.ProxyDefaults,
},
}
var out structs.IndexedGenericConfigEntries
require.NoError(t, msgpackrpc.CallWithCodec(codec, "ConfigEntry.ListAll", &args, &out))
expected := structs.IndexedGenericConfigEntries{
Entries: entries[0:1],
QueryMeta: out.QueryMeta,
}
require.Equal(t, expected, out)
})
}
func TestConfigEntry_List_ACLDeny(t *testing.T) {
@ -543,8 +589,9 @@ operator = "read"
}, nil))
// This should filter out the "db" service since we don't have permissions for it.
args := structs.ConfigEntryQuery{
args := structs.ConfigEntryListAllRequest{
Datacenter: s1.config.Datacenter,
Kinds: structs.AllConfigEntryKinds,
QueryOptions: structs.QueryOptions{Token: id},
}
var out structs.IndexedGenericConfigEntries

View File

@ -100,8 +100,9 @@ func (s *Server) reconcileLocalConfig(ctx context.Context, configs []structs.Con
func (s *Server) fetchConfigEntries(lastRemoteIndex uint64) (*structs.IndexedGenericConfigEntries, error) {
defer metrics.MeasureSince([]string{"leader", "replication", "config-entries", "fetch"}, time.Now())
req := structs.DCSpecificRequest{
req := structs.ConfigEntryListAllRequest{
Datacenter: s.config.PrimaryDatacenter,
Kinds: structs.AllConfigEntryKinds,
QueryOptions: structs.QueryOptions{
AllowStale: true,
MinQueryIndex: lastRemoteIndex,

View File

@ -293,9 +293,15 @@ func (c *FSM) applyIntentionOperation(buf []byte, index uint64) interface{} {
[]metrics.Label{{Name: "op", Value: string(req.Op)}})
switch req.Op {
case structs.IntentionOpCreate, structs.IntentionOpUpdate:
return c.state.IntentionSet(index, req.Intention)
//nolint:staticcheck
return c.state.LegacyIntentionSet(index, req.Intention)
case structs.IntentionOpDelete:
return c.state.IntentionDelete(index, req.Intention.ID)
//nolint:staticcheck
return c.state.LegacyIntentionDelete(index, req.Intention.ID)
case structs.IntentionOpDeleteAll:
return c.state.LegacyIntentionDeleteAll(index)
case structs.IntentionOpUpsert:
fallthrough // unsupported
default:
c.logger.Warn("Invalid Intention operation", "operation", req.Op)
return fmt.Errorf("Invalid Intention operation '%s'", req.Op)

View File

@ -1230,6 +1230,7 @@ func TestFSM_Intention_CRUD(t *testing.T) {
Intention: structs.TestIntention(t),
}
ixn.Intention.ID = generateUUID()
//nolint:staticcheck
ixn.Intention.UpdatePrecedence()
{
@ -1240,7 +1241,7 @@ func TestFSM_Intention_CRUD(t *testing.T) {
// Verify it's in the state store.
{
_, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
_, _, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
assert.Nil(err)
actual.CreateIndex, actual.ModifyIndex = 0, 0
@ -1260,7 +1261,7 @@ func TestFSM_Intention_CRUD(t *testing.T) {
// Verify the update.
{
_, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
_, _, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
assert.Nil(err)
actual.CreateIndex, actual.ModifyIndex = 0, 0
@ -1279,7 +1280,7 @@ func TestFSM_Intention_CRUD(t *testing.T) {
// Make sure it's gone.
{
_, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
_, _, actual, err := fsm.state.IntentionGet(nil, ixn.Intention.ID)
assert.Nil(err)
assert.Nil(actual)
}

View File

@ -20,7 +20,7 @@ func init() {
registerRestorer(structs.CoordinateBatchUpdateType, restoreCoordinates)
registerRestorer(structs.PreparedQueryRequestType, restorePreparedQuery)
registerRestorer(structs.AutopilotRequestType, restoreAutopilot)
registerRestorer(structs.IntentionRequestType, restoreIntention)
registerRestorer(structs.IntentionRequestType, restoreLegacyIntention)
registerRestorer(structs.ConnectCARequestType, restoreConnectCA)
registerRestorer(structs.ConnectCAProviderStateType, restoreConnectCAProviderState)
registerRestorer(structs.ConnectCAConfigType, restoreConnectCAConfig)
@ -57,7 +57,7 @@ func persistOSS(s *snapshot, sink raft.SnapshotSink, encoder *codec.Encoder) err
if err := s.persistAutopilot(sink, encoder); err != nil {
return err
}
if err := s.persistIntentions(sink, encoder); err != nil {
if err := s.persistLegacyIntentions(sink, encoder); err != nil {
return err
}
if err := s.persistConnectCA(sink, encoder); err != nil {
@ -402,9 +402,10 @@ func (s *snapshot) persistConnectCAProviderState(sink raft.SnapshotSink,
return nil
}
func (s *snapshot) persistIntentions(sink raft.SnapshotSink,
func (s *snapshot) persistLegacyIntentions(sink raft.SnapshotSink,
encoder *codec.Encoder) error {
ixns, err := s.state.Intentions()
//nolint:staticcheck
ixns, err := s.state.LegacyIntentions()
if err != nil {
return err
}
@ -614,12 +615,13 @@ func restoreAutopilot(header *snapshotHeader, restore *state.Restore, decoder *c
return nil
}
func restoreIntention(header *snapshotHeader, restore *state.Restore, decoder *codec.Decoder) error {
func restoreLegacyIntention(header *snapshotHeader, restore *state.Restore, decoder *codec.Decoder) error {
var req structs.Intention
if err := decoder.Decode(&req); err != nil {
return err
}
if err := restore.Intention(&req); err != nil {
//nolint:staticcheck
if err := restore.LegacyIntention(&req); err != nil {
return err
}
return nil

View File

@ -175,14 +175,15 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
}
require.NoError(t, fsm.state.AutopilotSetConfig(15, autopilotConf))
// Intentions
// Legacy Intentions
ixn := structs.TestIntention(t)
ixn.ID = generateUUID()
ixn.RaftIndex = structs.RaftIndex{
CreateIndex: 14,
ModifyIndex: 14,
}
require.NoError(t, fsm.state.IntentionSet(14, ixn))
//nolint:staticcheck
require.NoError(t, fsm.state.LegacyIntentionSet(14, ixn))
// CA Roots
roots := []*structs.CARoot{
@ -405,6 +406,19 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
}
require.NoError(t, fsm.state.SystemMetadataSet(25, systemMetadataEntry))
// service-intentions
serviceIxn := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "foo",
Sources: []*structs.SourceIntention{
{
Name: "bar",
Action: structs.IntentionActionAllow,
},
},
}
require.NoError(t, fsm.state.EnsureConfigEntry(26, serviceIxn, structs.DefaultEnterpriseMeta()))
// Snapshot
snap, err := fsm.Snapshot()
require.NoError(t, err)
@ -609,8 +623,8 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
require.NoError(t, err)
require.Equal(t, autopilotConf, restoredConf)
// Verify intentions are restored.
_, ixns, err := fsm2.state.Intentions(nil, structs.WildcardEnterpriseMeta())
// Verify legacy intentions are restored.
_, ixns, err := fsm2.state.LegacyIntentions(nil, structs.WildcardEnterpriseMeta())
require.NoError(t, err)
require.Len(t, ixns, 1)
require.Equal(t, ixn, ixns[0])
@ -672,6 +686,11 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
require.Len(t, systemMetadataLoaded, 1)
require.Equal(t, systemMetadataEntry, systemMetadataLoaded[0])
// Verify service-intentions is restored
_, serviceIxnEntry, err := fsm2.state.ConfigEntry(nil, structs.ServiceIntentions, "foo", structs.DefaultEnterpriseMeta())
require.NoError(t, err)
require.Equal(t, serviceIxn, serviceIxnEntry)
// Snapshot
snap, err = fsm2.Snapshot()
require.NoError(t, err)

View File

@ -21,16 +21,27 @@ var (
ErrIntentionNotFound = errors.New("Intention not found")
)
// NewIntentionEndpoint returns a new Intention endpoint.
func NewIntentionEndpoint(srv *Server, logger hclog.Logger) *Intention {
return &Intention{
srv: srv,
logger: logger,
configEntryEndpoint: &ConfigEntry{srv},
}
}
// Intention manages the Connect intentions.
type Intention struct {
// srv is a pointer back to the server.
srv *Server
logger hclog.Logger
configEntryEndpoint *ConfigEntry
}
func (s *Intention) checkIntentionID(id string) (bool, error) {
state := s.srv.fsm.State()
if _, ixn, err := state.IntentionGet(nil, id); err != nil {
if _, _, ixn, err := state.IntentionGet(nil, id); err != nil {
return false, err
} else if ixn != nil {
return false, nil
@ -39,9 +50,18 @@ func (s *Intention) checkIntentionID(id string) (bool, error) {
return true, nil
}
// prepareApplyCreate validates that the requester has permissions to create the new intention,
// generates a new uuid for the intention and generally validates that the request is well-formed
func (s *Intention) prepareApplyCreate(ident structs.ACLIdentity, authz acl.Authorizer, entMeta *structs.EnterpriseMeta, args *structs.IntentionRequest) error {
// prepareApplyCreate validates that the requester has permissions to create
// the new intention, generates a new uuid for the intention and generally
// validates that the request is well-formed
//
// Returns an existing service-intentions config entry for this destination if
// one exists.
func (s *Intention) prepareApplyCreate(
ident structs.ACLIdentity,
authz acl.Authorizer,
entMeta *structs.EnterpriseMeta,
args *structs.IntentionRequest,
) (*structs.ServiceIntentionsConfigEntry, error) {
if !args.Intention.CanWrite(authz) {
var accessorID string
if ident != nil {
@ -49,7 +69,7 @@ func (s *Intention) prepareApplyCreate(ident structs.ACLIdentity, authz acl.Auth
}
// todo(kit) Migrate intention access denial logging over to audit logging when we implement it
s.logger.Warn("Intention creation denied due to ACLs", "intention", args.Intention.ID, "accessorID", accessorID)
return acl.ErrPermissionDenied
return nil, acl.ErrPermissionDenied
}
// If no ID is provided, generate a new ID. This must be done prior to
@ -57,13 +77,13 @@ func (s *Intention) prepareApplyCreate(ident structs.ACLIdentity, authz acl.Auth
// the entry is in the log, the state update MUST be deterministic or
// the followers will not converge.
if args.Intention.ID != "" {
return fmt.Errorf("ID must be empty when creating a new intention")
return nil, fmt.Errorf("ID must be empty when creating a new intention")
}
var err error
args.Intention.ID, err = lib.GenerateUUID(s.checkIntentionID)
if err != nil {
return err
return nil, err
}
// Set the created at
args.Intention.CreatedAt = time.Now().UTC()
@ -77,27 +97,35 @@ func (s *Intention) prepareApplyCreate(ident structs.ACLIdentity, authz acl.Auth
args.Intention.DefaultNamespaces(entMeta)
if err := s.validateEnterpriseIntention(args.Intention); err != nil {
return err
return nil, err
}
// Validate. We do not validate on delete since it is valid to only
// send an ID in that case.
// Set the precedence
args.Intention.UpdatePrecedence()
//nolint:staticcheck
if err := args.Intention.Validate(); err != nil {
return err
return nil, err
}
// make sure we set the hash prior to raft application
args.Intention.SetHash()
_, configEntry, err := s.srv.fsm.State().ConfigEntry(nil, structs.ServiceIntentions, args.Intention.DestinationName, args.Intention.DestinationEnterpriseMeta())
if err != nil {
return nil, fmt.Errorf("service-intentions config entry lookup failed: %v", err)
} else if configEntry == nil {
return nil, nil
}
return nil
return configEntry.(*structs.ServiceIntentionsConfigEntry), nil
}
// prepareApplyUpdate validates that the requester has permissions on both the updated and existing
// prepareApplyUpdateLegacy validates that the requester has permissions on both the updated and existing
// intention as well as generally validating that the request is well-formed
func (s *Intention) prepareApplyUpdate(ident structs.ACLIdentity, authz acl.Authorizer, entMeta *structs.EnterpriseMeta, args *structs.IntentionRequest) error {
//
// Returns an existing service-intentions config entry for this destination if
// one exists.
func (s *Intention) prepareApplyUpdateLegacy(
ident structs.ACLIdentity,
authz acl.Authorizer,
entMeta *structs.EnterpriseMeta,
args *structs.IntentionRequest,
) (*structs.ServiceIntentionsConfigEntry, error) {
if !args.Intention.CanWrite(authz) {
var accessorID string
if ident != nil {
@ -105,15 +133,15 @@ func (s *Intention) prepareApplyUpdate(ident structs.ACLIdentity, authz acl.Auth
}
// todo(kit) Migrate intention access denial logging over to audit logging when we implement it
s.logger.Warn("Update operation on intention denied due to ACLs", "intention", args.Intention.ID, "accessorID", accessorID)
return acl.ErrPermissionDenied
return nil, acl.ErrPermissionDenied
}
_, ixn, err := s.srv.fsm.State().IntentionGet(nil, args.Intention.ID)
_, configEntry, ixn, err := s.srv.fsm.State().IntentionGet(nil, args.Intention.ID)
if err != nil {
return fmt.Errorf("Intention lookup failed: %v", err)
return nil, fmt.Errorf("Intention lookup failed: %v", err)
}
if ixn == nil {
return fmt.Errorf("Cannot modify non-existent intention: '%s'", args.Intention.ID)
if ixn == nil || configEntry == nil {
return nil, fmt.Errorf("Cannot modify non-existent intention: '%s'", args.Intention.ID)
}
// Perform the ACL check that we have write to the old intention too,
@ -126,7 +154,13 @@ func (s *Intention) prepareApplyUpdate(ident structs.ACLIdentity, authz acl.Auth
}
// todo(kit) Migrate intention access denial logging over to audit logging when we implement it
s.logger.Warn("Update operation on intention denied due to ACLs", "intention", args.Intention.ID, "accessorID", accessorID)
return acl.ErrPermissionDenied
return nil, acl.ErrPermissionDenied
}
// Prior to v1.9.0 renames of the destination side of an intention were
// allowed, but that behavior doesn't work anymore.
if ixn.DestinationServiceName() != args.Intention.DestinationServiceName() {
return nil, fmt.Errorf("Cannot modify DestinationNS or DestinationName for an intention once it exists.")
}
// We always update the updatedat field.
@ -140,40 +174,41 @@ func (s *Intention) prepareApplyUpdate(ident structs.ACLIdentity, authz acl.Auth
args.Intention.DefaultNamespaces(entMeta)
if err := s.validateEnterpriseIntention(args.Intention); err != nil {
return err
return nil, err
}
// Set the precedence
args.Intention.UpdatePrecedence()
// Validate. We do not validate on delete since it is valid to only
// send an ID in that case.
//nolint:staticcheck
if err := args.Intention.Validate(); err != nil {
return err
return nil, err
}
// make sure we set the hash prior to raft application
args.Intention.SetHash()
return nil
return configEntry, nil
}
// prepareApplyDelete ensures that the intention specified by the ID in the request exists
// prepareApplyDeleteLegacy ensures that the intention specified by the ID in the request exists
// and that the requester is authorized to delete it
func (s *Intention) prepareApplyDelete(ident structs.ACLIdentity, authz acl.Authorizer, args *structs.IntentionRequest) error {
//
// Returns an existing service-intentions config entry for this destination if
// one exists.
func (s *Intention) prepareApplyDeleteLegacy(
ident structs.ACLIdentity,
authz acl.Authorizer,
args *structs.IntentionRequest,
) (*structs.ServiceIntentionsConfigEntry, error) {
// If this is not a create, then we have to verify the ID.
state := s.srv.fsm.State()
_, ixn, err := state.IntentionGet(nil, args.Intention.ID)
_, configEntry, ixn, err := s.srv.fsm.State().IntentionGet(nil, args.Intention.ID)
if err != nil {
return fmt.Errorf("Intention lookup failed: %v", err)
return nil, fmt.Errorf("Intention lookup failed: %v", err)
}
if ixn == nil {
return fmt.Errorf("Cannot delete non-existent intention: '%s'", args.Intention.ID)
if ixn == nil || configEntry == nil {
return nil, fmt.Errorf("Cannot delete non-existent intention: '%s'", args.Intention.ID)
}
// Perform the ACL check that we have write to the old intention too,
// which must be true to perform any rename. This is the only ACL enforcement
// done for deletions and a secondary enforcement for updates.
// Perform the ACL check that we have write to the old intention. This is
// the only ACL enforcement done for deletions and a secondary enforcement
// for updates.
if !ixn.CanWrite(authz) {
var accessorID string
if ident != nil {
@ -181,9 +216,27 @@ func (s *Intention) prepareApplyDelete(ident structs.ACLIdentity, authz acl.Auth
}
// todo(kit) Migrate intention access denial logging over to audit logging when we implement it
s.logger.Warn("Deletion operation on intention denied due to ACLs", "intention", args.Intention.ID, "accessorID", accessorID)
return acl.ErrPermissionDenied
return nil, acl.ErrPermissionDenied
}
return configEntry, nil
}
var ErrIntentionsNotUpgradedYet = errors.New("Intentions are read only while being upgraded to config entries")
// legacyUpgradeCheck fast fails a write request using the legacy intention
// RPCs if the system is known to be mid-upgrade. This is purely a perf
// optimization and the actual real enforcement happens in the FSM. It would be
// wasteful to round trip all the way through raft to have it fail for
// known-up-front reasons, hence why we check it twice.
func (s *Intention) legacyUpgradeCheck() error {
usingConfigEntries, err := s.srv.fsm.State().AreIntentionsInConfigEntries()
if err != nil {
return fmt.Errorf("system metadata lookup failed: %v", err)
}
if !usingConfigEntries {
return ErrIntentionsNotUpgradedYet
}
return nil
}
@ -192,10 +245,9 @@ func (s *Intention) Apply(
args *structs.IntentionRequest,
reply *string) error {
// Forward this request to the primary DC if we're a secondary that's replicating intentions.
if s.srv.intentionReplicationEnabled() {
args.Datacenter = s.srv.config.PrimaryDatacenter
}
// Ensure that all service-intentions config entry writes go to the primary
// datacenter. These will then be replicated to all the other datacenters.
args.Datacenter = s.srv.config.PrimaryDatacenter
if done, err := s.srv.ForwardRPC("Intention.Apply", args, args, reply); done {
return err
@ -203,6 +255,10 @@ func (s *Intention) Apply(
defer metrics.MeasureSince([]string{"consul", "intention", "apply"}, time.Now())
defer metrics.MeasureSince([]string{"intention", "apply"}, time.Now())
if err := s.legacyUpgradeCheck(); err != nil {
return err
}
// Always set a non-nil intention to avoid nil-access below
if args.Intention == nil {
args.Intention = &structs.Intention{}
@ -215,37 +271,236 @@ func (s *Intention) Apply(
return err
}
var (
prevEntry *structs.ServiceIntentionsConfigEntry
upsertEntry *structs.ServiceIntentionsConfigEntry
legacyWrite bool
noop bool
)
switch args.Op {
case structs.IntentionOpCreate:
if err := s.prepareApplyCreate(ident, authz, &entMeta, args); err != nil {
legacyWrite = true
// This variant is just for legacy UUID-based intentions.
prevEntry, err = s.prepareApplyCreate(ident, authz, &entMeta, args)
if err != nil {
return err
}
if prevEntry == nil {
upsertEntry = args.Intention.ToConfigEntry()
} else {
upsertEntry = prevEntry.Clone()
upsertEntry.Sources = append(upsertEntry.Sources, args.Intention.ToSourceIntention())
}
case structs.IntentionOpUpdate:
if err := s.prepareApplyUpdate(ident, authz, &entMeta, args); err != nil {
// This variant is just for legacy UUID-based intentions.
legacyWrite = true
prevEntry, err = s.prepareApplyUpdateLegacy(ident, authz, &entMeta, args)
if err != nil {
return err
}
upsertEntry = prevEntry.Clone()
for i, src := range upsertEntry.Sources {
if src.LegacyID == args.Intention.ID {
upsertEntry.Sources[i] = args.Intention.ToSourceIntention()
break
}
}
case structs.IntentionOpUpsert:
// This variant is just for config-entry based intentions.
legacyWrite = false
if args.Intention.ID != "" {
// This is a new-style only endpoint
return fmt.Errorf("ID must not be specified")
}
args.Intention.DefaultNamespaces(&entMeta)
prevEntry, err = s.getServiceIntentionsConfigEntry(args.Intention.DestinationName, args.Intention.DestinationEnterpriseMeta())
if err != nil {
return err
}
sn := args.Intention.SourceServiceName()
// TODO(intentions): have service-intentions validation functions
// return structured errors so that we can rewrite the field prefix
// here so that the validation errors are not misleading.
if prevEntry == nil {
// Meta is NOT permitted here, as it would need to be persisted on
// the enclosing config entry.
if len(args.Intention.Meta) > 0 {
return fmt.Errorf("Meta must not be specified")
}
upsertEntry = args.Intention.ToConfigEntry()
} else {
upsertEntry = prevEntry.Clone()
if len(args.Intention.Meta) > 0 {
// Meta is NOT permitted here, but there is one exception. If
// you are updating a previous record, but that record lives
// within a config entry that itself has Meta, then you may
// incidentally ship the Meta right back to consul.
//
// In that case if Meta is provided, it has to be a perfect
// match for what is already on the enclosing config entry so
// it's safe to discard.
if !equalStringMaps(upsertEntry.Meta, args.Intention.Meta) {
return fmt.Errorf("Meta must not be specified, or should be unchanged during an update.")
}
// Now it is safe to discard
args.Intention.Meta = nil
}
found := false
for i, src := range upsertEntry.Sources {
if src.SourceServiceName() == sn {
upsertEntry.Sources[i] = args.Intention.ToSourceIntention()
found = true
break
}
}
if !found {
upsertEntry.Sources = append(upsertEntry.Sources, args.Intention.ToSourceIntention())
}
}
case structs.IntentionOpDelete:
if err := s.prepareApplyDelete(ident, authz, args); err != nil {
return err
// There are two ways to get this request:
//
// 1) legacy: the ID field is populated
// 2) config-entry: the ID field is NOT populated
if args.Intention.ID == "" {
// config-entry style: no LegacyID
legacyWrite = false
args.Intention.DefaultNamespaces(&entMeta)
prevEntry, err = s.getServiceIntentionsConfigEntry(args.Intention.DestinationName, args.Intention.DestinationEnterpriseMeta())
if err != nil {
return err
}
// NOTE: validation errors may be misleading!
noop = true
if prevEntry != nil {
sn := args.Intention.SourceServiceName()
upsertEntry = prevEntry.Clone()
for i, src := range upsertEntry.Sources {
if src.SourceServiceName() == sn {
// Delete slice element: https://github.com/golang/go/wiki/SliceTricks#delete
// a = append(a[:i], a[i+1:]...)
upsertEntry.Sources = append(upsertEntry.Sources[:i], upsertEntry.Sources[i+1:]...)
if len(upsertEntry.Sources) == 0 {
upsertEntry.Sources = nil
}
noop = false
break
}
}
}
} else {
// legacy style: LegacyID required
legacyWrite = true
prevEntry, err = s.prepareApplyDeleteLegacy(ident, authz, args)
if err != nil {
return err
}
upsertEntry = prevEntry.Clone()
for i, src := range upsertEntry.Sources {
if src.LegacyID == args.Intention.ID {
// Delete slice element: https://github.com/golang/go/wiki/SliceTricks#delete
// a = append(a[:i], a[i+1:]...)
upsertEntry.Sources = append(upsertEntry.Sources[:i], upsertEntry.Sources[i+1:]...)
if len(upsertEntry.Sources) == 0 {
upsertEntry.Sources = nil
}
break
}
}
}
case structs.IntentionOpDeleteAll:
// This is an internal operation initiated by the leader and is not
// exposed for general RPC use.
fallthrough
default:
return fmt.Errorf("Invalid Intention operation: %v", args.Op)
}
// setup the reply which will have been filled in by one of the 3 preparedApply* funcs
*reply = args.Intention.ID
// Commit
resp, err := s.srv.raftApply(structs.IntentionRequestType, args)
if err != nil {
s.logger.Error("Raft apply failed", "error", err)
return err
}
if respErr, ok := resp.(error); ok {
return respErr
if !noop && prevEntry != nil && legacyWrite && !prevEntry.LegacyIDFieldsAreAllSet() {
sn := prevEntry.DestinationServiceName()
return fmt.Errorf("cannot use legacy intention API to edit intentions with a destination of %q after editing them via a service-intentions config entry", sn.String())
}
return nil
// setup the reply which will have been filled in by one of the preparedApply* funcs
if legacyWrite {
*reply = args.Intention.ID
} else {
*reply = ""
}
if noop {
return nil
}
// Commit indirectly by invoking the other RPC handler directly.
configReq := &structs.ConfigEntryRequest{
Datacenter: args.Datacenter,
WriteRequest: args.WriteRequest,
}
if upsertEntry == nil || len(upsertEntry.Sources) == 0 {
configReq.Op = structs.ConfigEntryDelete
configReq.Entry = &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: prevEntry.Name,
EnterpriseMeta: prevEntry.EnterpriseMeta,
}
var ignored struct{}
return s.configEntryEndpoint.Delete(configReq, &ignored)
} else {
// Update config entry CAS
configReq.Op = structs.ConfigEntryUpsertCAS
configReq.Entry = upsertEntry
var normalizeAndValidateFn func(raw structs.ConfigEntry) error
if legacyWrite {
normalizeAndValidateFn = func(raw structs.ConfigEntry) error {
entry := raw.(*structs.ServiceIntentionsConfigEntry)
if err := entry.LegacyNormalize(); err != nil {
return err
}
return entry.LegacyValidate()
}
}
var applied bool
err := s.configEntryEndpoint.applyInternal(configReq, &applied, normalizeAndValidateFn)
if err != nil {
return err
}
if !applied {
return fmt.Errorf("config entry failed to persist due to CAS failure: kind=%q, name=%q", upsertEntry.Kind, upsertEntry.Name)
}
return nil
}
}
// Get returns a single intention by ID.
@ -290,9 +545,9 @@ func (s *Intention) Get(
err error
)
if args.IntentionID != "" {
index, ixn, err = state.IntentionGet(ws, args.IntentionID)
index, _, ixn, err = state.IntentionGet(ws, args.IntentionID)
} else if args.Exact != nil {
index, ixn, err = state.IntentionGetExact(ws, args.Exact)
index, _, ixn, err = state.IntentionGetExact(ws, args.Exact)
}
if err != nil {
@ -325,7 +580,7 @@ func (s *Intention) Get(
// List returns all the intentions.
func (s *Intention) List(
args *structs.DCSpecificRequest,
args *structs.IntentionListRequest,
reply *structs.IndexedIntentions) error {
// Forward if necessary
if done, err := s.srv.ForwardRPC("Intention.List", args, args, reply); done {
@ -349,7 +604,17 @@ func (s *Intention) List(
return s.srv.blockingQuery(
&args.QueryOptions, &reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, ixns, err := state.Intentions(ws, &args.EnterpriseMeta)
var (
index uint64
ixns structs.Intentions
fromConfig bool
err error
)
if args.Legacy {
index, ixns, err = state.LegacyIntentions(ws, &args.EnterpriseMeta)
} else {
index, ixns, fromConfig, err = state.Intentions(ws, &args.EnterpriseMeta)
}
if err != nil {
return err
}
@ -359,6 +624,12 @@ func (s *Intention) List(
reply.Intentions = make(structs.Intentions, 0)
}
if fromConfig {
reply.DataOrigin = structs.IntentionDataOriginConfigEntries
} else {
reply.DataOrigin = structs.IntentionDataOriginLegacy
}
if err := s.srv.filterACL(args.Token, reply); err != nil {
return err
}
@ -588,3 +859,35 @@ func (s *Intention) validateEnterpriseIntention(ixn *structs.Intention) error {
}
return nil
}
func (s *Intention) getServiceIntentionsConfigEntry(name string, entMeta *structs.EnterpriseMeta) (*structs.ServiceIntentionsConfigEntry, error) {
_, raw, err := s.srv.fsm.State().ConfigEntry(nil, structs.ServiceIntentions, name, entMeta)
if err != nil {
return nil, fmt.Errorf("Intention lookup failed: %v", err)
}
if raw == nil {
return nil, nil
}
configEntry, ok := raw.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return nil, fmt.Errorf("invalid service config type %T", raw)
}
return configEntry, nil
}
func equalStringMaps(a, b map[string]string) bool {
if len(a) != len(b) {
return false
}
for k := range a {
v, ok := b[k]
if !ok || a[k] != v {
return false
}
}
return true
}

View File

@ -1,12 +1,14 @@
package consul
import (
"fmt"
"os"
"testing"
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/require"
)
@ -15,7 +17,6 @@ import (
func TestIntentionApply_new(t *testing.T) {
t.Parallel()
require := require.New(t)
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
@ -44,8 +45,8 @@ func TestIntentionApply_new(t *testing.T) {
now := time.Now()
// Create
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
require.NotEmpty(reply)
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
require.NotEmpty(t, reply)
// Read
ixn.Intention.ID = reply
@ -55,20 +56,44 @@ func TestIntentionApply_new(t *testing.T) {
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(resp.Intentions, 1)
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
require.Equal(resp.Index, actual.ModifyIndex)
require.WithinDuration(now, actual.CreatedAt, 5*time.Second)
require.WithinDuration(now, actual.UpdatedAt, 5*time.Second)
require.Equal(t, resp.Index, actual.ModifyIndex)
require.WithinDuration(t, now, actual.CreatedAt, 5*time.Second)
require.WithinDuration(t, now, actual.UpdatedAt, 5*time.Second)
actual.CreateIndex, actual.ModifyIndex = 0, 0
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
actual.Hash = ixn.Intention.Hash
//nolint:staticcheck
ixn.Intention.UpdatePrecedence()
require.Equal(ixn.Intention, actual)
require.Equal(t, ixn.Intention, actual)
}
// Rename should fail
t.Run("renaming the destination should fail", func(t *testing.T) {
// Setup a basic record to create
ixn2 := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpUpdate,
Intention: &structs.Intention{
ID: ixn.Intention.ID,
SourceNS: structs.IntentionDefaultNamespace,
SourceName: "test",
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: "test-updated",
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
},
}
var reply string
err := msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn2, &reply)
testutil.RequireErrorContains(t, err, "Cannot modify DestinationNS or DestinationName for an intention once it exists.")
})
}
// Test the source type defaults
@ -151,7 +176,6 @@ func TestIntentionApply_createWithID(t *testing.T) {
func TestIntentionApply_updateGood(t *testing.T) {
t.Parallel()
require := require.New(t)
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
@ -177,8 +201,8 @@ func TestIntentionApply_updateGood(t *testing.T) {
var reply string
// Create
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
require.NotEmpty(reply)
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
require.NotEmpty(t, reply)
// Read CreatedAt
var createdAt time.Time
@ -189,8 +213,8 @@ func TestIntentionApply_updateGood(t *testing.T) {
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(resp.Intentions, 1)
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
createdAt = actual.CreatedAt
}
@ -201,8 +225,8 @@ func TestIntentionApply_updateGood(t *testing.T) {
// Update
ixn.Op = structs.IntentionOpUpdate
ixn.Intention.ID = reply
ixn.Intention.SourceName = "*"
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
ixn.Intention.Description = "updated"
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Apply", &ixn, &reply))
// Read
ixn.Intention.ID = reply
@ -212,18 +236,19 @@ func TestIntentionApply_updateGood(t *testing.T) {
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
require.Nil(msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(resp.Intentions, 1)
require.NoError(t, msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
require.Equal(createdAt, actual.CreatedAt)
require.WithinDuration(time.Now(), actual.UpdatedAt, 5*time.Second)
require.Equal(t, createdAt, actual.CreatedAt)
require.WithinDuration(t, time.Now(), actual.UpdatedAt, 5*time.Second)
actual.CreateIndex, actual.ModifyIndex = 0, 0
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
actual.Hash = ixn.Intention.Hash
//nolint:staticcheck
ixn.Intention.UpdatePrecedence()
require.Equal(ixn.Intention, actual)
require.Equal(t, ixn.Intention, actual)
}
}
@ -305,6 +330,424 @@ func TestIntentionApply_deleteGood(t *testing.T) {
}
}
func TestIntentionApply_WithoutIDs(t *testing.T) {
t.Parallel()
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
waitForLeaderEstablishment(t, s1)
defaultEntMeta := structs.DefaultEnterpriseMeta()
opApply := func(req *structs.IntentionRequest) error {
req.Datacenter = "dc1"
var ignored string
return msgpackrpc.CallWithCodec(codec, "Intention.Apply", &req, &ignored)
}
opGet := func(req *structs.IntentionQueryRequest) (*structs.IndexedIntentions, error) {
req.Datacenter = "dc1"
var resp structs.IndexedIntentions
if err := msgpackrpc.CallWithCodec(codec, "Intention.Get", req, &resp); err != nil {
return nil, err
}
return &resp, nil
}
opList := func() (*structs.IndexedIntentions, error) {
req := &structs.IntentionListRequest{
Datacenter: "dc1",
EnterpriseMeta: *structs.WildcardEnterpriseMeta(),
}
var resp structs.IndexedIntentions
if err := msgpackrpc.CallWithCodec(codec, "Intention.List", req, &resp); err != nil {
return nil, err
}
return &resp, nil
}
configEntryUpsert := func(entry *structs.ServiceIntentionsConfigEntry) error {
req := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Op: structs.ConfigEntryUpsert,
Entry: entry,
}
var ignored bool
return msgpackrpc.CallWithCodec(codec, "ConfigEntry.Apply", req, &ignored)
}
getConfigEntry := func(kind, name string) (*structs.ServiceIntentionsConfigEntry, error) {
state := s1.fsm.State()
_, entry, err := state.ConfigEntry(nil, kind, name, defaultEntMeta)
if err != nil {
return nil, err
}
ixn, ok := entry.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return nil, fmt.Errorf("unexpected type: %T", entry)
}
return ixn, nil
}
// Setup a basic record to create
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "test",
DestinationName: "test",
Action: structs.IntentionActionAllow,
Description: "original",
},
}))
// Read it back.
{
resp, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "test",
DestinationName: "test",
},
})
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
got := resp.Intentions[0]
require.Equal(t, "original", got.Description)
// Verify it is in the new-style.
require.Empty(t, got.ID)
require.True(t, got.CreatedAt.IsZero())
require.True(t, got.UpdatedAt.IsZero())
}
// Double check that there's only 1.
{
resp, err := opList()
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
}
// Verify the config entry structure is expected.
{
entry, err := getConfigEntry(structs.ServiceIntentions, "test")
require.NoError(t, err)
require.NotNil(t, entry)
expect := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionAllow,
Description: "original",
Precedence: 9,
Type: structs.IntentionSourceConsul,
},
},
RaftIndex: entry.RaftIndex,
}
require.Equal(t, expect, entry)
}
// Update in place.
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "test",
DestinationName: "test",
Action: structs.IntentionActionAllow,
Description: "updated",
},
}))
// Read it back.
{
resp, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "test",
DestinationName: "test",
},
})
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
got := resp.Intentions[0]
require.Equal(t, "updated", got.Description)
// Verify it is in the new-style.
require.Empty(t, got.ID)
require.True(t, got.CreatedAt.IsZero())
require.True(t, got.UpdatedAt.IsZero())
}
// Double check that there's only 1.
{
resp, err := opList()
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
}
// Create a second one sharing the same destination
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "assay",
DestinationName: "test",
Action: structs.IntentionActionDeny,
Description: "original-2",
},
}))
// Read it back.
{
resp, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "assay",
DestinationName: "test",
},
})
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
got := resp.Intentions[0]
require.Equal(t, "original-2", got.Description)
// Verify it is in the new-style.
require.Empty(t, got.ID)
require.True(t, got.CreatedAt.IsZero())
require.True(t, got.UpdatedAt.IsZero())
}
// Double check that there's 2 now.
{
resp, err := opList()
require.NoError(t, err)
require.Len(t, resp.Intentions, 2)
}
// Verify the config entry structure is expected.
{
entry, err := getConfigEntry(structs.ServiceIntentions, "test")
require.NoError(t, err)
require.NotNil(t, entry)
expect := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionAllow,
Description: "updated",
Precedence: 9,
Type: structs.IntentionSourceConsul,
},
{
Name: "assay",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionDeny,
Description: "original-2",
Precedence: 9,
Type: structs.IntentionSourceConsul,
},
},
RaftIndex: entry.RaftIndex,
}
require.Equal(t, expect, entry)
}
// Delete the original
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpDelete,
Intention: &structs.Intention{
SourceName: "test",
DestinationName: "test",
},
}))
// Read it back (not found)
{
_, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "test",
DestinationName: "test",
},
})
testutil.RequireErrorContains(t, err, ErrIntentionNotFound.Error())
}
// Double check that there's 1 again.
{
resp, err := opList()
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
}
// Verify the config entry structure is expected.
{
entry, err := getConfigEntry(structs.ServiceIntentions, "test")
require.NoError(t, err)
require.NotNil(t, entry)
expect := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "assay",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionDeny,
Description: "original-2",
Precedence: 9,
Type: structs.IntentionSourceConsul,
},
},
RaftIndex: entry.RaftIndex,
}
require.Equal(t, expect, entry)
}
// Set metadata on the config entry directly.
{
require.NoError(t, configEntryUpsert(&structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "test",
EnterpriseMeta: *defaultEntMeta,
Meta: map[string]string{
"foo": "bar",
"zim": "gir",
},
Sources: []*structs.SourceIntention{
{
Name: "assay",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionDeny,
Description: "original-2",
Precedence: 9,
Type: structs.IntentionSourceConsul,
},
},
}))
}
// Attempt to create a new intention and set the metadata.
{
err := opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "foo",
DestinationName: "bar",
Action: structs.IntentionActionDeny,
Meta: map[string]string{"horseshoe": "crab"},
},
})
testutil.RequireErrorContains(t, err, "Meta must not be specified")
}
// Attempt to update an intention and change the metadata.
{
err := opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "assay",
DestinationName: "test",
Action: structs.IntentionActionDeny,
Description: "original-3",
Meta: map[string]string{"horseshoe": "crab"},
},
})
testutil.RequireErrorContains(t, err, "Meta must not be specified, or should be unchanged during an update.")
}
// Try again with the same metadata.
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "assay",
DestinationName: "test",
Action: structs.IntentionActionDeny,
Description: "original-3",
Meta: map[string]string{
"foo": "bar",
"zim": "gir",
},
},
}))
// Read it back.
{
resp, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "assay",
DestinationName: "test",
},
})
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
got := resp.Intentions[0]
require.Equal(t, "original-3", got.Description)
require.Equal(t, map[string]string{
"foo": "bar",
"zim": "gir",
}, got.Meta)
// Verify it is in the new-style.
require.Empty(t, got.ID)
require.True(t, got.CreatedAt.IsZero())
require.True(t, got.UpdatedAt.IsZero())
}
// Try again with NO metadata.
require.NoError(t, opApply(&structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
Intention: &structs.Intention{
SourceName: "assay",
DestinationName: "test",
Action: structs.IntentionActionDeny,
Description: "original-4",
},
}))
// Read it back.
{
resp, err := opGet(&structs.IntentionQueryRequest{
Exact: &structs.IntentionQueryExact{
SourceName: "assay",
DestinationName: "test",
},
})
require.NoError(t, err)
require.Len(t, resp.Intentions, 1)
got := resp.Intentions[0]
require.Equal(t, "original-4", got.Description)
require.Equal(t, map[string]string{
"foo": "bar",
"zim": "gir",
}, got.Meta)
// Verify it is in the new-style.
require.Empty(t, got.ID)
require.True(t, got.CreatedAt.IsZero())
require.True(t, got.UpdatedAt.IsZero())
}
}
// Test apply with a deny ACL
func TestIntentionApply_aclDeny(t *testing.T) {
t.Parallel()
@ -380,6 +823,7 @@ service "foo" {
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
actual.Hash = ixn.Intention.Hash
//nolint:staticcheck
ixn.Intention.UpdatePrecedence()
require.Equal(ixn.Intention, actual)
}
@ -406,7 +850,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
denyToken, err := upsertTestTokenWithPolicyRules(codec, TestDefaultMasterToken, "dc1", `service_prefix "" { policy = "deny" intentions = "deny" }`)
require.NoError(t, err)
doIntentionCreate := func(t *testing.T, token string, deny bool) string {
doIntentionCreate := func(t *testing.T, token string, dest string, deny bool) string {
t.Helper()
ixn := structs.IntentionRequest{
Datacenter: "dc1",
@ -415,7 +859,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
SourceNS: "default",
SourceName: "*",
DestinationNS: "default",
DestinationName: "*",
DestinationName: dest,
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
},
@ -437,7 +881,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
t.Run("deny-write-for-read-token", func(t *testing.T) {
// This tests ensures that tokens with only read access to all intentions
// cannot create a wildcard intention
doIntentionCreate(t, readToken.SecretID, true)
doIntentionCreate(t, readToken.SecretID, "*", true)
})
t.Run("deny-write-for-exact-wildcard-rule", func(t *testing.T) {
@ -446,7 +890,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
// intentions = "write"
// }
// will not actually allow creating an intention with a wildcard service name
doIntentionCreate(t, exactToken.SecretID, true)
doIntentionCreate(t, exactToken.SecretID, "*", true)
})
t.Run("deny-write-for-prefix-wildcard-rule", func(t *testing.T) {
@ -455,14 +899,14 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
// intentions = "write"
// }
// will not actually allow creating an intention with a wildcard service name
doIntentionCreate(t, wildcardPrefixToken.SecretID, true)
doIntentionCreate(t, wildcardPrefixToken.SecretID, "*", true)
})
var intentionID string
allowWriteOk := t.Run("allow-write", func(t *testing.T) {
// tests that a token with all the required privileges can create
// intentions with a wildcard destination
intentionID = doIntentionCreate(t, writeToken.SecretID, false)
intentionID = doIntentionCreate(t, writeToken.SecretID, "*", false)
})
requireAllowWrite := func(t *testing.T) {
@ -522,7 +966,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
doIntentionList := func(t *testing.T, token string, deny bool) {
t.Helper()
requireAllowWrite(t)
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: token},
}
@ -609,7 +1053,11 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
doIntentionMatch(t, denyToken.SecretID, true)
})
doIntentionUpdate := func(t *testing.T, token string, dest string, deny bool) {
// Since we can't rename the destination, create a new intention for the rest of this test.
wildIntentionID := intentionID
fooIntentionID := doIntentionCreate(t, writeToken.SecretID, "foo", false)
doIntentionUpdate := func(t *testing.T, token string, intentionID, dest, description string, deny bool) {
t.Helper()
requireAllowWrite(t)
ixn := structs.IntentionRequest{
@ -621,6 +1069,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
SourceName: "*",
DestinationNS: "default",
DestinationName: dest,
Description: description,
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
},
@ -637,19 +1086,16 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
}
t.Run("deny-update-for-foo-token", func(t *testing.T) {
doIntentionUpdate(t, fooToken.SecretID, "foo", true)
doIntentionUpdate(t, fooToken.SecretID, wildIntentionID, "*", "wild-desc", true)
})
t.Run("allow-update-for-prefix-token", func(t *testing.T) {
// this tests that regardless of going from a wildcard intention
// to a non-wildcard or the opposite direction that the permissions
// are checked correctly. This also happens to leave the intention
// in a state ready for verifying similar things with deletion
doIntentionUpdate(t, writeToken.SecretID, "foo", false)
doIntentionUpdate(t, writeToken.SecretID, "*", false)
// This tests that the prefix token can edit wildcard intentions and regular intentions.
doIntentionUpdate(t, writeToken.SecretID, fooIntentionID, "foo", "foo-desc-two", false)
doIntentionUpdate(t, writeToken.SecretID, wildIntentionID, "*", "wild-desc-two", false)
})
doIntentionDelete := func(t *testing.T, token string, deny bool) {
doIntentionDelete := func(t *testing.T, token string, intentionID string, deny bool) {
t.Helper()
requireAllowWrite(t)
ixn := structs.IntentionRequest{
@ -671,7 +1117,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
}
t.Run("deny-delete-for-read-token", func(t *testing.T) {
doIntentionDelete(t, readToken.SecretID, true)
doIntentionDelete(t, readToken.SecretID, fooIntentionID, true)
})
t.Run("deny-delete-for-exact-wildcard-rule", func(t *testing.T) {
@ -680,7 +1126,7 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
// intentions = "write"
// }
// will not actually allow deleting an intention with a wildcard service name
doIntentionDelete(t, exactToken.SecretID, true)
doIntentionDelete(t, exactToken.SecretID, fooIntentionID, true)
})
t.Run("deny-delete-for-prefix-wildcard-rule", func(t *testing.T) {
@ -689,13 +1135,13 @@ func TestIntention_WildcardACLEnforcement(t *testing.T) {
// intentions = "write"
// }
// will not actually allow creating an intention with a wildcard service name
doIntentionDelete(t, wildcardPrefixToken.SecretID, true)
doIntentionDelete(t, wildcardPrefixToken.SecretID, fooIntentionID, true)
})
t.Run("allow-delete", func(t *testing.T) {
// tests that a token with all the required privileges can delete
// intentions with a wildcard destination
doIntentionDelete(t, writeToken.SecretID, false)
doIntentionDelete(t, writeToken.SecretID, fooIntentionID, false)
})
}
@ -1056,7 +1502,7 @@ func TestIntentionList(t *testing.T) {
// Test with no intentions inserted yet
{
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
}
var resp structs.IndexedIntentions
@ -1101,7 +1547,7 @@ func TestIntentionList_acl(t *testing.T) {
// Test with no token
t.Run("no-token", func(t *testing.T) {
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
}
var resp structs.IndexedIntentions
@ -1111,7 +1557,7 @@ func TestIntentionList_acl(t *testing.T) {
// Test with management token
t.Run("master-token", func(t *testing.T) {
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: TestDefaultMasterToken},
}
@ -1122,7 +1568,7 @@ func TestIntentionList_acl(t *testing.T) {
// Test with user token
t.Run("user-token", func(t *testing.T) {
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: token.SecretID},
}
@ -1132,7 +1578,7 @@ func TestIntentionList_acl(t *testing.T) {
})
t.Run("filtered", func(t *testing.T) {
req := &structs.DCSpecificRequest{
req := &structs.IntentionListRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{
Token: TestDefaultMasterToken,
@ -1512,3 +1958,44 @@ func TestIntentionCheck_match(t *testing.T) {
require.False(t, resp.Allowed)
}
}
func TestEqualStringMaps(t *testing.T) {
m1 := map[string]string{
"foo": "a",
}
m2 := map[string]string{
"foo": "a",
"bar": "b",
}
var m3 map[string]string
m4 := map[string]string{
"dog": "",
}
m5 := map[string]string{
"cat": "",
}
tests := []struct {
a map[string]string
b map[string]string
result bool
}{
{m1, m1, true},
{m2, m2, true},
{m1, m2, false},
{m2, m1, false},
{m2, m2, true},
{m3, m1, false},
{m3, m3, true},
{m4, m5, false},
}
for i, test := range tests {
actual := equalStringMaps(test.a, test.b)
if actual != test.result {
t.Fatalf("case %d, expected %v, got %v", i, test.result, actual)
}
}
}

View File

@ -290,6 +290,7 @@ WAIT:
// previously inflight transactions have been committed and that our
// state is up-to-date.
func (s *Server) establishLeadership() error {
start := time.Now()
// check for the upgrade here - this helps us transition to new ACLs much
// quicker if this is a new cluster or this is a test agent
if canUpgrade := s.canUpgradeToNewACLs(true); canUpgrade {
@ -326,11 +327,6 @@ func (s *Server) establishLeadership() error {
return err
}
// attempt to bootstrap config entries
if err := s.bootstrapConfigEntries(s.config.ConfigEntryBootstrap); err != nil {
return err
}
s.getOrCreateAutopilotConfig()
s.autopilot.Start()
@ -345,9 +341,20 @@ func (s *Server) establishLeadership() error {
s.startFederationStateAntiEntropy()
s.startConnectLeader()
if err := s.startConnectLeader(); err != nil {
return err
}
// Attempt to bootstrap config entries. We wait until after starting the
// Connect leader tasks so we hopefully have transitioned to supporting
// service-intentions.
if err := s.bootstrapConfigEntries(s.config.ConfigEntryBootstrap); err != nil {
return err
}
s.setConsistentReadReady()
s.logger.Debug("successfully established leadership", "duration", time.Since(start))
return nil
}
@ -1017,6 +1024,23 @@ func (s *Server) bootstrapConfigEntries(entries []structs.ConfigEntry) error {
}
state := s.fsm.State()
// Do a quick preflight check to see if someone is trying to upgrade from
// an older pre-1.9.0 version of consul with intentions AND are trying to
// bootstrap a service-intentions config entry at the same time.
usingConfigEntries, err := s.fsm.State().AreIntentionsInConfigEntries()
if err != nil {
return fmt.Errorf("Failed to determine if we are migrating intentions yet: %v", err)
}
if !usingConfigEntries {
for _, entry := range entries {
if entry.GetKind() == structs.ServiceIntentions {
return fmt.Errorf("Refusing to apply configuration entry %q / %q because intentions are still being migrated to config entries: %v",
entry.GetKind(), entry.GetName(), err)
}
}
}
for _, entry := range entries {
// avoid a round trip through Raft if we know the CAS is going to fail
_, existing, err := state.ConfigEntry(nil, entry.GetKind(), entry.GetName(), entry.GetEnterpriseMeta())
@ -1544,7 +1568,7 @@ func (s *Server) DatacenterSupportsFederationStates() bool {
found: false,
}
// check if they are supported in the primary dc
// if we are in a secondary, check if they are supported in the primary dc
if s.config.PrimaryDatacenter != s.config.Datacenter {
s.router.CheckServers(s.config.PrimaryDatacenter, state.update)
@ -1594,3 +1618,68 @@ func (s *serversFederationStatesInfo) update(srv *metadata.Server) bool {
// prevent continuing server evaluation
return false
}
func (s *Server) setDatacenterSupportsIntentionsAsConfigEntries() {
atomic.StoreInt32(&s.dcSupportsIntentionsAsConfigEntries, 1)
}
func (s *Server) DatacenterSupportsIntentionsAsConfigEntries() bool {
if atomic.LoadInt32(&s.dcSupportsIntentionsAsConfigEntries) != 0 {
return true
}
state := serversIntentionsAsConfigEntriesInfo{
supported: true,
found: false,
}
// if we are in a secondary, check if they are supported in the primary dc
if s.config.PrimaryDatacenter != s.config.Datacenter {
s.router.CheckServers(s.config.PrimaryDatacenter, state.update)
if !state.supported || !state.found {
s.logger.Debug("intentions have not been migrated to config entries in the primary dc yet")
return false
}
}
// check the servers in the local DC
s.router.CheckServers(s.config.Datacenter, state.update)
if state.supported && state.found {
s.setDatacenterSupportsIntentionsAsConfigEntries()
return true
}
s.logger.Debug("intentions cannot be migrated to config entries in this datacenter", "datacenter", s.config.Datacenter)
return false
}
type serversIntentionsAsConfigEntriesInfo struct {
// supported indicates whether every processed server supports intentions as config entries
supported bool
// found indicates that at least one server was processed
found bool
}
func (s *serversIntentionsAsConfigEntriesInfo) update(srv *metadata.Server) bool {
if srv.Status != serf.StatusAlive && srv.Status != serf.StatusFailed {
// they are left or something so regardless we treat these servers as meeting
// the version requirement
return true
}
// mark that we processed at least one server
s.found = true
if supported, ok := srv.FeatureFlags["si"]; ok && supported == 1 {
return true
}
// mark that at least one server does not support service-intentions
s.supported = false
// prevent continuing server evaluation
return false
}

View File

@ -1,7 +1,6 @@
package consul
import (
"bytes"
"context"
"fmt"
"reflect"
@ -25,10 +24,6 @@ const (
// retryBucketSize is the maximum number of stored rate limit attempts for looped
// blocking query operations.
retryBucketSize = 5
// maxIntentionTxnSize is the maximum size (in bytes) of a transaction used during
// Intention replication.
maxIntentionTxnSize = raftWarnSize / 4
)
var (
@ -553,24 +548,26 @@ func (s *Server) generateCASignRequest(csr string) *structs.CASignRequest {
}
// startConnectLeader starts multi-dc connect leader routines.
func (s *Server) startConnectLeader() {
func (s *Server) startConnectLeader() error {
if !s.config.ConnectEnabled {
return nil
}
// Start the Connect secondary DC actions if enabled.
if s.config.ConnectEnabled && s.config.Datacenter != s.config.PrimaryDatacenter {
if s.config.Datacenter != s.config.PrimaryDatacenter {
s.leaderRoutineManager.Start(secondaryCARootWatchRoutineName, s.secondaryCARootWatch)
s.leaderRoutineManager.Start(intentionReplicationRoutineName, s.replicateIntentions)
s.leaderRoutineManager.Start(secondaryCertRenewWatchRoutineName, s.secondaryIntermediateCertRenewalWatch)
s.startConnectLeaderEnterprise()
}
s.leaderRoutineManager.Start(caRootPruningRoutineName, s.runCARootPruning)
return s.startIntentionConfigEntryMigration()
}
// stopConnectLeader stops connect specific leader functions.
func (s *Server) stopConnectLeader() {
s.leaderRoutineManager.Stop(intentionMigrationRoutineName)
s.leaderRoutineManager.Stop(secondaryCARootWatchRoutineName)
s.leaderRoutineManager.Stop(intentionReplicationRoutineName)
s.leaderRoutineManager.Stop(caRootPruningRoutineName)
s.stopConnectLeaderEnterprise()
// If the provider implements NeedsStop, we call Stop to perform any shutdown actions.
s.caProviderReconfigurationLock.Lock()
@ -782,66 +779,6 @@ func (s *Server) secondaryCARootWatch(ctx context.Context) error {
return nil
}
// replicateIntentions executes a blocking query to the primary datacenter to replicate
// the intentions there to the local state.
func (s *Server) replicateIntentions(ctx context.Context) error {
connectLogger := s.loggers.Named(logging.Connect)
args := structs.DCSpecificRequest{
Datacenter: s.config.PrimaryDatacenter,
}
connectLogger.Debug("starting Connect intention replication from primary datacenter", "primary", s.config.PrimaryDatacenter)
retryLoopBackoff(ctx, func() error {
// Always use the latest replication token value in case it changed while looping.
args.QueryOptions.Token = s.tokens.ReplicationToken()
var remote structs.IndexedIntentions
if err := s.forwardDC("Intention.List", s.config.PrimaryDatacenter, &args, &remote); err != nil {
return err
}
_, local, err := s.fsm.State().Intentions(nil, s.replicationEnterpriseMeta())
if err != nil {
return err
}
// Compute the diff between the remote and local intentions.
deletes, updates := diffIntentions(local, remote.Intentions)
txnOpSets := batchIntentionUpdates(deletes, updates)
// Apply batched updates to the state store.
for _, ops := range txnOpSets {
txnReq := structs.TxnRequest{Ops: ops}
resp, err := s.raftApply(structs.TxnRequestType, &txnReq)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
if txnResp, ok := resp.(structs.TxnResponse); ok {
if len(txnResp.Errors) > 0 {
return txnResp.Error()
}
} else {
return fmt.Errorf("unexpected return type %T", resp)
}
}
args.QueryOptions.MinQueryIndex = nextIndexVal(args.QueryOptions.MinQueryIndex, remote.QueryMeta.Index)
return nil
}, func(err error) {
connectLogger.Error("error replicating intentions",
"routine", intentionReplicationRoutineName,
"error", err,
)
})
return nil
}
// retryLoopBackoff loops a given function indefinitely, backing off exponentially
// upon errors up to a maximum of maxRetryBackoff seconds.
func retryLoopBackoff(ctx context.Context, loopFn func() error, errFn func(error)) {
@ -888,79 +825,6 @@ func retryLoopBackoffHandleSuccess(ctx context.Context, loopFn func() error, err
}
}
// diffIntentions computes the difference between the local and remote intentions
// and returns lists of deletes and updates.
func diffIntentions(local, remote structs.Intentions) (structs.Intentions, structs.Intentions) {
localIdx := make(map[string][]byte, len(local))
remoteIdx := make(map[string]struct{}, len(remote))
var deletes structs.Intentions
var updates structs.Intentions
for _, intention := range local {
localIdx[intention.ID] = intention.Hash
}
for _, intention := range remote {
remoteIdx[intention.ID] = struct{}{}
}
for _, intention := range local {
if _, ok := remoteIdx[intention.ID]; !ok {
deletes = append(deletes, intention)
}
}
for _, intention := range remote {
existingHash, ok := localIdx[intention.ID]
if !ok {
updates = append(updates, intention)
} else if bytes.Compare(existingHash, intention.Hash) != 0 {
updates = append(updates, intention)
}
}
return deletes, updates
}
// batchIntentionUpdates breaks up the given updates into sets of TxnOps based
// on the estimated size of the operations.
func batchIntentionUpdates(deletes, updates structs.Intentions) []structs.TxnOps {
var txnOps structs.TxnOps
for _, delete := range deletes {
deleteOp := &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: delete,
}
txnOps = append(txnOps, &structs.TxnOp{Intention: deleteOp})
}
for _, update := range updates {
updateOp := &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: update,
}
txnOps = append(txnOps, &structs.TxnOp{Intention: updateOp})
}
// Divide the operations into chunks according to maxIntentionTxnSize.
var batchedOps []structs.TxnOps
for batchStart := 0; batchStart < len(txnOps); {
// inner loop finds the last element to include in this batch.
batchSize := 0
batchEnd := batchStart
for ; batchEnd < len(txnOps) && batchSize < maxIntentionTxnSize; batchEnd += 1 {
batchSize += txnOps[batchEnd].Intention.Intention.EstimateSize()
}
batchedOps = append(batchedOps, txnOps[batchStart:batchEnd])
// txnOps[batchEnd] wasn't included as the slicing doesn't include the element at the stop index
batchStart = batchEnd
}
return batchedOps
}
// nextIndexVal computes the next index value to query for, resetting to zero
// if the index went backward.
func nextIndexVal(prevIdx, idx uint64) uint64 {

View File

@ -16,7 +16,6 @@ import (
ca "github.com/hashicorp/consul/agent/connect/ca"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/token"
tokenStore "github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
uuid "github.com/hashicorp/go-uuid"
@ -793,422 +792,6 @@ func getTestRoots(s *Server, datacenter string) (*structs.IndexedCARoots, *struc
return &rootList, active, nil
}
func TestLeader_ReplicateIntentions(t *testing.T) {
t.Parallel()
assert := assert.New(t)
require := require.New(t)
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
// set the build to ensure all the version checks pass and enable all the connect features that operate cross-dc
c.Build = "1.6.0"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
s1.tokens.UpdateAgentToken("root", tokenStore.TokenSourceConfig)
replicationRules := `acl = "read" service_prefix "" { policy = "read" intentions = "read" } operator = "write" `
// create some tokens
replToken1, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", replicationRules)
require.NoError(err)
replToken2, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", replicationRules)
require.NoError(err)
// dc2 as a secondary DC
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLDefaultPolicy = "deny"
c.ACLTokenReplication = false
c.Build = "1.6.0"
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
s2.tokens.UpdateAgentToken("root", tokenStore.TokenSourceConfig)
// start out with one token
s2.tokens.UpdateReplicationToken(replToken1.SecretID, tokenStore.TokenSourceConfig)
// Create the WAN link
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Create an intention in dc1
ixn := structs.IntentionRequest{
Datacenter: "dc1",
WriteRequest: structs.WriteRequest{Token: "root"},
Op: structs.IntentionOpCreate,
Intention: &structs.Intention{
SourceNS: structs.IntentionDefaultNamespace,
SourceName: "test",
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: "test",
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
},
}
var reply string
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
require.NotEmpty(reply)
// Wait for it to get replicated to dc2
var createdAt time.Time
ixn.Intention.ID = reply
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
r.Check(s2.RPC("Intention.Get", req, &resp))
if len(resp.Intentions) != 1 {
r.Fatalf("bad: %v", resp.Intentions)
}
actual := resp.Intentions[0]
createdAt = actual.CreatedAt
})
// Sleep a bit so that the UpdatedAt field will definitely be different
time.Sleep(1 * time.Millisecond)
// delete underlying acl token being used for replication
require.NoError(deleteTestToken(codec, "root", "dc1", replToken1.AccessorID))
// switch to the other token
s2.tokens.UpdateReplicationToken(replToken2.SecretID, tokenStore.TokenSourceConfig)
// Update the intention in dc1
ixn.Op = structs.IntentionOpUpdate
ixn.Intention.ID = reply
ixn.Intention.SourceName = "*"
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
// Wait for dc2 to get the update
ixn.Intention.ID = reply
var resp structs.IndexedIntentions
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
r.Check(s2.RPC("Intention.Get", req, &resp))
if len(resp.Intentions) != 1 {
r.Fatalf("bad: %v", resp.Intentions)
}
if resp.Intentions[0].SourceName != "*" {
r.Fatalf("bad: %v", resp.Intentions[0])
}
})
actual := resp.Intentions[0]
assert.Equal(createdAt, actual.CreatedAt)
assert.WithinDuration(time.Now(), actual.UpdatedAt, 5*time.Second)
actual.CreateIndex, actual.ModifyIndex = 0, 0
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
ixn.Intention.UpdatePrecedence()
assert.Equal(ixn.Intention, actual)
// Delete
ixn.Op = structs.IntentionOpDelete
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
// Wait for the delete to be replicated
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
err := s2.RPC("Intention.Get", req, &resp)
if err == nil || !strings.Contains(err.Error(), ErrIntentionNotFound.Error()) {
r.Fatalf("expected intention not found")
}
})
}
func TestLeader_ReplicateIntentions_forwardToPrimary(t *testing.T) {
t.Parallel()
assert := assert.New(t)
require := require.New(t)
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// dc2 as a secondary DC
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Create the WAN link
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Create an intention in dc2
ixn := structs.IntentionRequest{
Datacenter: "dc2",
Op: structs.IntentionOpCreate,
Intention: &structs.Intention{
SourceNS: structs.IntentionDefaultNamespace,
SourceName: "test",
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: "test",
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
},
}
var reply string
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
require.NotEmpty(reply)
// Make sure it exists in both DCs
var createdAt time.Time
ixn.Intention.ID = reply
retry.Run(t, func(r *retry.R) {
for _, server := range []*Server{s1, s2} {
req := &structs.IntentionQueryRequest{
Datacenter: server.config.Datacenter,
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
r.Check(server.RPC("Intention.Get", req, &resp))
if len(resp.Intentions) != 1 {
r.Fatalf("bad: %v", resp.Intentions)
}
actual := resp.Intentions[0]
createdAt = actual.CreatedAt
}
})
// Sleep a bit so that the UpdatedAt field will definitely be different
time.Sleep(1 * time.Millisecond)
// Update the intention in dc1
ixn.Op = structs.IntentionOpUpdate
ixn.Intention.ID = reply
ixn.Intention.SourceName = "*"
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
// Wait for dc2 to get the update
ixn.Intention.ID = reply
var resp structs.IndexedIntentions
retry.Run(t, func(r *retry.R) {
for _, server := range []*Server{s1, s2} {
req := &structs.IntentionQueryRequest{
Datacenter: server.config.Datacenter,
IntentionID: ixn.Intention.ID,
}
r.Check(server.RPC("Intention.Get", req, &resp))
if len(resp.Intentions) != 1 {
r.Fatalf("bad: %v", resp.Intentions)
}
if resp.Intentions[0].SourceName != "*" {
r.Fatalf("bad: %v", resp.Intentions[0])
}
}
})
actual := resp.Intentions[0]
assert.Equal(createdAt, actual.CreatedAt)
assert.WithinDuration(time.Now(), actual.UpdatedAt, 5*time.Second)
actual.CreateIndex, actual.ModifyIndex = 0, 0
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
actual.Hash = ixn.Intention.Hash
ixn.Intention.UpdatePrecedence()
assert.Equal(ixn.Intention, actual)
// Delete
ixn.Op = structs.IntentionOpDelete
require.NoError(s1.RPC("Intention.Apply", &ixn, &reply))
// Wait for the delete to be replicated
retry.Run(t, func(r *retry.R) {
for _, server := range []*Server{s1, s2} {
req := &structs.IntentionQueryRequest{
Datacenter: server.config.Datacenter,
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
err := server.RPC("Intention.Get", req, &resp)
if err == nil || !strings.Contains(err.Error(), ErrIntentionNotFound.Error()) {
r.Fatalf("expected intention not found")
}
}
})
}
func TestLeader_batchIntentionUpdates(t *testing.T) {
t.Parallel()
assert := assert.New(t)
ixn1 := structs.TestIntention(t)
ixn1.ID = "ixn1"
ixn2 := structs.TestIntention(t)
ixn2.ID = "ixn2"
ixnLarge := structs.TestIntention(t)
ixnLarge.ID = "ixnLarge"
ixnLarge.Description = strings.Repeat("x", maxIntentionTxnSize-1)
cases := []struct {
deletes structs.Intentions
updates structs.Intentions
expected []structs.TxnOps
}{
// 1 deletes, 0 updates
{
deletes: structs.Intentions{ixn1},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
},
},
},
// 0 deletes, 1 updates
{
updates: structs.Intentions{ixn1},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn1,
},
},
},
},
},
// 1 deletes, 1 updates
{
deletes: structs.Intentions{ixn1},
updates: structs.Intentions{ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
// 1 large intention update
{
updates: structs.Intentions{ixnLarge},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixnLarge,
},
},
},
},
},
// 2 deletes (w/ a large intention), 1 updates
{
deletes: structs.Intentions{ixn1, ixnLarge},
updates: structs.Intentions{ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixnLarge,
},
},
},
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
// 1 deletes , 2 updates (w/ a large intention)
{
deletes: structs.Intentions{ixn1},
updates: structs.Intentions{ixnLarge, ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixnLarge,
},
},
},
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
}
for _, tc := range cases {
actual := batchIntentionUpdates(tc.deletes, tc.updates)
assert.Equal(tc.expected, actual)
}
}
func TestLeader_GenerateCASignRequest(t *testing.T) {
csr := "A"
s := Server{config: &Config{PrimaryDatacenter: "east"}, tokens: new(token.Store)}

View File

@ -0,0 +1,487 @@
package consul
import (
"bytes"
"context"
"fmt"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/logging"
)
const (
// maxIntentionTxnSize is the maximum size (in bytes) of a transaction used during
// Intention replication.
maxIntentionTxnSize = raftWarnSize / 4
)
func (s *Server) startIntentionConfigEntryMigration() error {
if !s.config.ConnectEnabled {
return nil
}
// Check for the system metadata first, as that's the most trustworthy in
// both the primary and secondaries.
intentionFormat, err := s.getSystemMetadata(structs.SystemMetadataIntentionFormatKey)
if err != nil {
return err
}
if intentionFormat == structs.SystemMetadataIntentionFormatConfigValue {
// Bypass the serf component and jump right to the final state.
s.setDatacenterSupportsIntentionsAsConfigEntries()
return nil // nothing to migrate
}
if s.config.PrimaryDatacenter == s.config.Datacenter {
// Do a quick legacy intentions check to see if it's even worth
// spinning up the routine at all. This only applies if the primary
// datacenter is composed entirely of compatible servers and there are
// no more legacy intentions.
if s.DatacenterSupportsIntentionsAsConfigEntries() {
_, ixns, err := s.fsm.State().LegacyIntentions(nil, structs.WildcardEnterpriseMeta())
if err != nil {
return err
}
if len(ixns) == 0 {
// Though there's nothing to migrate, still trigger the special
// delete-all operation which should update various indexes and
// drop some system metadata so we can skip all of this next
// time.
//
// This is done inline with leader election so that new
// clusters on 1.9.0 with no legacy intentions will immediately
// transition to intentions-as-config-entries mode.
return s.legacyIntentionsMigrationCleanupPhase(true)
}
}
// When running in the primary we do all of the real work.
s.leaderRoutineManager.Start(intentionMigrationRoutineName, s.legacyIntentionMigration)
} else {
// When running in the secondary we mostly just wait until the
// primary finishes, and then wait until we're pretty sure the main
// config entry replication thread has seen all of the
// migration-related config entry edits before zeroing OUR copy of
// the old intentions table.
s.leaderRoutineManager.Start(intentionMigrationRoutineName, s.legacyIntentionMigrationInSecondaryDC)
}
return nil
}
// This function is only intended to be run as a managed go routine, it will block until
// the context passed in indicates that it should exit.
func (s *Server) legacyIntentionMigration(ctx context.Context) error {
if s.config.PrimaryDatacenter != s.config.Datacenter {
return nil
}
connectLogger := s.loggers.Named(logging.Connect)
loopCtx, loopCancel := context.WithCancel(ctx)
defer loopCancel()
retryLoopBackoff(loopCtx, func() error {
// We have to wait until all of our sibling servers are upgraded.
if !s.DatacenterSupportsIntentionsAsConfigEntries() {
return nil
}
state := s.fsm.State()
_, ixns, err := state.LegacyIntentions(nil, structs.WildcardEnterpriseMeta())
if err != nil {
return err
}
// NOTE: do not early abort here if the list is empty, let it run to completion.
entries, err := convertLegacyIntentionsToConfigEntries(ixns)
if err != nil {
return err
}
// Totally cheat and repurpose one part of config entry replication
// here so we automatically get our writes rate limited.
_, err = s.reconcileLocalConfig(ctx, entries, structs.ConfigEntryUpsert)
if err != nil {
return err
}
// Wrap up
if err := s.legacyIntentionsMigrationCleanupPhase(false); err != nil {
return err
}
loopCancel()
connectLogger.Info("intention migration complete")
return nil
}, func(err error) {
connectLogger.Error(
"error migrating intentions to config entries, will retry",
"routine", intentionMigrationRoutineName,
"error", err,
)
})
return nil
}
func convertLegacyIntentionsToConfigEntries(ixns structs.Intentions) ([]structs.ConfigEntry, error) {
entries := migrateIntentionsToConfigEntries(ixns)
genericEntries := make([]structs.ConfigEntry, 0, len(entries))
for _, entry := range entries {
if err := entry.LegacyNormalize(); err != nil {
return nil, err
}
if err := entry.LegacyValidate(); err != nil {
return nil, err
}
genericEntries = append(genericEntries, entry)
}
return genericEntries, nil
}
// legacyIntentionsMigrationCleanupPhase will delete all legacy intentions and
// also record a piece of system metadata indicating that the migration has
// been completed.
func (s *Server) legacyIntentionsMigrationCleanupPhase(quiet bool) error {
if !quiet {
s.loggers.Named(logging.Connect).
Info("finishing up intention migration by clearing the legacy store")
}
// This is a special intention op that ensures we bind the raft indexes
// associated with both the legacy table and the config entry table.
//
// We also update a piece of system metadata to reflect that we are
// definitely in a post-migration world.
req := structs.IntentionRequest{
Op: structs.IntentionOpDeleteAll,
}
if resp, err := s.raftApply(structs.IntentionRequestType, req); err != nil {
return err
} else if respErr, ok := resp.(error); ok {
return respErr
}
// Bypass the serf component and jump right to the final state.
s.setDatacenterSupportsIntentionsAsConfigEntries()
return nil
}
func (s *Server) legacyIntentionMigrationInSecondaryDC(ctx context.Context) error {
if s.config.PrimaryDatacenter == s.config.Datacenter {
return nil
}
const (
stateReplicateLegacy = iota
stateWaitForPrimary
stateWaitForConfigReplication
stateDoCleanup
)
var (
connectLogger = s.loggers.Named(logging.Connect)
currentState = stateReplicateLegacy
lastLegacyReplicationFetchIndex uint64
legacyReplicationDisabled bool
lastLegacyOnlyFetchIndex uint64
)
// This loop does several things:
//
// (1) Until we know for certain that the all of the servers in the primary
// DC and all of the servers in our DC are running a Consul version that
// can support intentions as config entries we have to continue to do
// legacy intention replication.
//
// (2) Once we know all versions of Consul are compatible, we cease to
// replicate legacy intentions as that table is frozen in the primary DC.
// We do a special blocking query back to exclusively the legacy intentions
// table in the primary to detect when it is zeroed out. We capture the max
// raft index of this zeroing.
//
// (3) We wait until our own config entry replication crosses the primary
// index from (2) so we know that we have replicated all of the new forms
// of the existing intentions.
// (1) Legacy intention replication. A blocking query back to the primary
// asking for intentions to replicate is both needed if the primary is OLD
// since we still need to replicate new writes, but also if the primary is
// NEW to know when the migration code in the primary has completed and
// zeroed the legacy memdb table.
//
// (2) If the primary has finished migration, we have to wait until our own
// config entry replication catches up.
//
// (3) After config entry replication catches up we should zero out own own
// legacy intentions memdb table.
loopCtx, loopCancel := context.WithCancel(ctx)
defer loopCancel()
retryLoopBackoff(loopCtx, func() error {
// This for loop only exists to avoid backoff every state transition.
// Only trigger the loop if the state changes, otherwise return a nil
// error.
for {
// Check for the system metadata first, as that's the most trustworthy.
intentionFormat, err := s.getSystemMetadata(structs.SystemMetadataIntentionFormatKey)
if err != nil {
return err
}
if intentionFormat == structs.SystemMetadataIntentionFormatConfigValue {
// Bypass the serf component and jump right to the final state.
s.setDatacenterSupportsIntentionsAsConfigEntries()
loopCancel()
return nil // nothing to migrate
}
switch currentState {
case stateReplicateLegacy:
if s.DatacenterSupportsIntentionsAsConfigEntries() {
// Now all nodes in this datacenter and the primary are totally
// ready for intentions as config entries, so disable legacy
// replication and transition to the next phase.
currentState = stateWaitForPrimary
// Explicitly zero these out as they are now unused but could
// be at worst misleading.
lastLegacyReplicationFetchIndex = 0
legacyReplicationDisabled = false
} else if !legacyReplicationDisabled {
// This is the embedded legacy intention replication.
index, outOfLegacyMode, err := s.replicateLegacyIntentionsOnce(ctx, lastLegacyReplicationFetchIndex)
if err != nil {
return err
} else if outOfLegacyMode {
// We chill out and wait until all of the nodes in this
// datacenter are ready for intentions as config entries.
//
// It's odd that we get this to happen before serf gives us
// the feature flag, but gossip isn't immediate so it's
// technically possible.
legacyReplicationDisabled = true
} else {
lastLegacyReplicationFetchIndex = nextIndexVal(lastLegacyReplicationFetchIndex, index)
return nil
}
}
case stateWaitForPrimary:
// Loop until we see the primary has finished migrating to config entries.
index, numIxns, err := s.fetchLegacyIntentionsSummary(ctx, lastLegacyOnlyFetchIndex)
if err != nil {
return err
}
lastLegacyOnlyFetchIndex = nextIndexVal(lastLegacyOnlyFetchIndex, index)
if numIxns == 0 {
connectLogger.Debug("intention migration in secondary status", "last_primary_index", lastLegacyOnlyFetchIndex)
currentState = stateWaitForConfigReplication
// do not clear lastLegacyOnlyFetchIndex!
} else {
return nil
}
case stateWaitForConfigReplication:
// manually list replicated config entries by kind
// lastLegacyOnlyFetchIndex is now the raft commit index that
// zeroed out the intentions memdb table.
//
// We compare that with the last raft commit index we have replicated
// config entries for and use that to determine if we have caught up.
lastReplicatedConfigIndex := s.configReplicator.Index()
connectLogger.Debug(
"intention migration in secondary status",
"last_primary_intention_index", lastLegacyOnlyFetchIndex,
"last_primary_replicated_config_index", lastReplicatedConfigIndex,
)
if lastReplicatedConfigIndex >= lastLegacyOnlyFetchIndex {
currentState = stateDoCleanup
} else {
return nil
}
case stateDoCleanup:
if err := s.legacyIntentionsMigrationCleanupPhase(false); err != nil {
return err
}
loopCancel()
return nil
default:
return fmt.Errorf("impossible state: %v", currentState)
}
}
}, func(err error) {
connectLogger.Error(
"error performing intention migration in secondary datacenter, will retry",
"routine", intentionMigrationRoutineName,
"error", err,
)
})
return nil
}
func (s *Server) fetchLegacyIntentionsSummary(_ context.Context, lastFetchIndex uint64) (uint64, int, error) {
args := structs.IntentionListRequest{
Datacenter: s.config.PrimaryDatacenter,
Legacy: true,
QueryOptions: structs.QueryOptions{
MinQueryIndex: lastFetchIndex,
Token: s.tokens.ReplicationToken(),
},
}
var remote structs.IndexedIntentions
if err := s.forwardDC("Intention.List", s.config.PrimaryDatacenter, &args, &remote); err != nil {
return 0, 0, err
}
return remote.Index, len(remote.Intentions), nil
}
// replicateLegacyIntentionsOnce executes a blocking query to the primary
// datacenter to replicate the intentions there to the local state one time.
func (s *Server) replicateLegacyIntentionsOnce(ctx context.Context, lastFetchIndex uint64) (uint64, bool, error) {
args := structs.DCSpecificRequest{
Datacenter: s.config.PrimaryDatacenter,
EnterpriseMeta: *s.replicationEnterpriseMeta(),
QueryOptions: structs.QueryOptions{
MinQueryIndex: lastFetchIndex,
Token: s.tokens.ReplicationToken(),
},
}
var remote structs.IndexedIntentions
if err := s.forwardDC("Intention.List", s.config.PrimaryDatacenter, &args, &remote); err != nil {
return 0, false, err
}
select {
case <-ctx.Done():
return 0, false, ctx.Err()
default:
}
if remote.DataOrigin == structs.IntentionDataOriginConfigEntries {
return 0, true, nil
}
_, local, err := s.fsm.State().LegacyIntentions(nil, s.replicationEnterpriseMeta())
if err != nil {
return 0, false, err
}
// Compute the diff between the remote and local intentions.
deletes, updates := diffIntentions(local, remote.Intentions)
txnOpSets := batchLegacyIntentionUpdates(deletes, updates)
// Apply batched updates to the state store.
for _, ops := range txnOpSets {
txnReq := structs.TxnRequest{Ops: ops}
resp, err := s.raftApply(structs.TxnRequestType, &txnReq)
if err != nil {
return 0, false, err
}
if respErr, ok := resp.(error); ok {
return 0, false, respErr
}
if txnResp, ok := resp.(structs.TxnResponse); ok {
if len(txnResp.Errors) > 0 {
return 0, false, txnResp.Error()
}
} else {
return 0, false, fmt.Errorf("unexpected return type %T", resp)
}
}
return remote.QueryMeta.Index, false, nil
}
// diffIntentions computes the difference between the local and remote intentions
// and returns lists of deletes and updates.
func diffIntentions(local, remote structs.Intentions) (structs.Intentions, structs.Intentions) {
localIdx := make(map[string][]byte, len(local))
remoteIdx := make(map[string]struct{}, len(remote))
var deletes structs.Intentions
var updates structs.Intentions
for _, intention := range local {
localIdx[intention.ID] = intention.Hash
}
for _, intention := range remote {
remoteIdx[intention.ID] = struct{}{}
}
for _, intention := range local {
if _, ok := remoteIdx[intention.ID]; !ok {
deletes = append(deletes, intention)
}
}
for _, intention := range remote {
existingHash, ok := localIdx[intention.ID]
if !ok {
updates = append(updates, intention)
} else if bytes.Compare(existingHash, intention.Hash) != 0 {
updates = append(updates, intention)
}
}
return deletes, updates
}
// batchLegacyIntentionUpdates breaks up the given updates into sets of TxnOps based
// on the estimated size of the operations.
//
//nolint:staticcheck
func batchLegacyIntentionUpdates(deletes, updates structs.Intentions) []structs.TxnOps {
var txnOps structs.TxnOps
for _, delete := range deletes {
deleteOp := &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: delete,
}
txnOps = append(txnOps, &structs.TxnOp{Intention: deleteOp})
}
for _, update := range updates {
updateOp := &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: update,
}
txnOps = append(txnOps, &structs.TxnOp{Intention: updateOp})
}
// Divide the operations into chunks according to maxIntentionTxnSize.
var batchedOps []structs.TxnOps
for batchStart := 0; batchStart < len(txnOps); {
// inner loop finds the last element to include in this batch.
batchSize := 0
batchEnd := batchStart
for ; batchEnd < len(txnOps) && batchSize < maxIntentionTxnSize; batchEnd += 1 {
batchSize += txnOps[batchEnd].Intention.Intention.LegacyEstimateSize()
}
batchedOps = append(batchedOps, txnOps[batchStart:batchEnd])
// txnOps[batchEnd] wasn't included as the slicing doesn't include the element at the stop index
batchStart = batchEnd
}
return batchedOps
}

View File

@ -3,42 +3,18 @@
package consul
import (
"context"
"fmt"
"strings"
"time"
"github.com/hashicorp/consul/agent/structs"
)
const intentionUpgradeCleanupRoutineName = "intention cleanup"
func (s *Server) startConnectLeaderEnterprise() {
if s.config.PrimaryDatacenter != s.config.Datacenter {
// intention cleanup should only run in the primary
return
}
s.leaderRoutineManager.Start(intentionUpgradeCleanupRoutineName, s.runIntentionUpgradeCleanup)
}
func (s *Server) stopConnectLeaderEnterprise() {
// will be a no-op when not started
s.leaderRoutineManager.Stop(intentionUpgradeCleanupRoutineName)
}
func (s *Server) runIntentionUpgradeCleanup(ctx context.Context) error {
// TODO(rb): handle retry?
func migrateIntentionsToConfigEntries(ixns structs.Intentions) []*structs.ServiceIntentionsConfigEntry {
// Remove any intention in OSS that happened to have used a non-default
// namespace.
//
// The one exception is that if we find wildcards namespaces we "upgrade"
// them to "default" if there isn't already an existing intention.
_, ixns, err := s.fsm.State().Intentions(nil, structs.WildcardEnterpriseMeta())
if err != nil {
return fmt.Errorf("failed to list intentions: %v", err)
}
//
// default/<foo> => default/<foo> || OK
// default/* => default/<foo> || OK
// */* => default/<foo> || becomes: default/* => default/<foo>
@ -57,7 +33,7 @@ func (s *Server) runIntentionUpgradeCleanup(ctx context.Context) error {
var (
retained = make(map[intentionName]struct{})
tryUpgrades = make(map[intentionName]*structs.Intention)
removeIDs []string
output structs.Intentions
)
for _, ixn := range ixns {
srcNS := strings.ToLower(ixn.SourceNS)
@ -75,6 +51,7 @@ func (s *Server) runIntentionUpgradeCleanup(ctx context.Context) error {
dstNS, ixn.DestinationName,
}
retained[name] = struct{}{}
output = append(output, ixn)
continue // a-ok for OSS
}
@ -88,54 +65,22 @@ func (s *Server) runIntentionUpgradeCleanup(ctx context.Context) error {
updated.DestinationNS = structs.IntentionDefaultNamespace
}
// Run parts of the checks in Intention.prepareApplyUpdate.
// We always update the updatedat field.
updated.UpdatedAt = time.Now().UTC()
// Set the precedence
updated.UpdatePrecedence()
// make sure we set the hash prior to raft application
updated.SetHash()
name := intentionName{
updated.SourceNS, updated.SourceName,
updated.DestinationNS, updated.DestinationName,
}
tryUpgrades[name] = updated
} else {
removeIDs = append(removeIDs, ixn.ID)
}
}
for name, updated := range tryUpgrades {
if _, collision := retained[name]; collision {
// The update we wanted to do would collide with an existing intention
// so delete our original wildcard intention instead.
removeIDs = append(removeIDs, updated.ID)
} else {
req := structs.IntentionRequest{
Op: structs.IntentionOpUpdate,
Intention: updated,
}
if _, err := s.raftApply(structs.IntentionRequestType, &req); err != nil {
return fmt.Errorf("failed to remove wildcard namespaces from intention %q: %v", updated.ID, err)
}
// Check to see if the update we wanted to do would collide with an
// existing intention. If so, we delete our original wildcard intention
// via simply omitting it from migration.
if _, collision := retained[name]; !collision {
output = append(output, updated)
}
}
for _, id := range removeIDs {
req := structs.IntentionRequest{
Op: structs.IntentionOpDelete,
Intention: &structs.Intention{
ID: id,
},
}
if _, err := s.raftApply(structs.IntentionRequestType, &req); err != nil {
return fmt.Errorf("failed to remove intention with invalid namespace %q: %v", id, err)
}
}
return nil // transition complete
return structs.MigrateIntentions(output)
}

View File

@ -3,63 +3,20 @@
package consul
import (
"context"
"os"
"sort"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
tokenStore "github.com/hashicorp/consul/agent/token"
"github.com/stretchr/testify/require"
)
func TestLeader_OSS_IntentionUpgradeCleanup(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
// set the build to ensure all the version checks pass and enable all the connect features that operate cross-dc
c.Build = "1.6.0"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
waitForLeaderEstablishment(t, s1)
s1.tokens.UpdateAgentToken("root", tokenStore.TokenSourceConfig)
lastIndex := uint64(0)
nextIndex := func() uint64 {
lastIndex++
return lastIndex
}
wildEntMeta := structs.WildcardEnterpriseMeta()
resetIntentions := func(t *testing.T) {
func TestMigrateIntentionsToConfigEntries(t *testing.T) {
compare := func(t *testing.T, got structs.Intentions, expect [][]string) {
t.Helper()
_, ixns, err := s1.fsm.State().Intentions(nil, wildEntMeta)
require.NoError(t, err)
for _, ixn := range ixns {
require.NoError(t, s1.fsm.State().IntentionDelete(nextIndex(), ixn.ID))
}
}
compare := func(t *testing.T, expect [][]string) {
t.Helper()
_, ixns, err := s1.fsm.State().Intentions(nil, wildEntMeta)
require.NoError(t, err)
var actual [][]string
for _, ixn := range ixns {
for _, ixn := range got {
actual = append(actual, []string{
ixn.SourceNS,
ixn.SourceName,
@ -151,9 +108,9 @@ func TestLeader_OSS_IntentionUpgradeCleanup(t *testing.T) {
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
resetIntentions(t)
// Do something super evil and directly reach into the FSM to seed it with "bad" data.
var ixns structs.Intentions
for _, elem := range tc.insert {
require.Len(t, elem, 4)
ixn := structs.TestIntention(t)
@ -164,17 +121,23 @@ func TestLeader_OSS_IntentionUpgradeCleanup(t *testing.T) {
ixn.DestinationName = elem[3]
ixn.CreatedAt = time.Now().UTC()
ixn.UpdatedAt = ixn.CreatedAt
require.NoError(t, s1.fsm.State().IntentionSet(nextIndex(), ixn))
ixns = append(ixns, ixn)
}
// Sleep a bit so that the UpdatedAt field will definitely be different
time.Sleep(1 * time.Millisecond)
// TODO: figure out how to test this properly during leader startup
got := migrateIntentionsToConfigEntries(ixns)
require.NoError(t, s1.runIntentionUpgradeCleanup(context.Background()))
// Convert them back to the line-item version.
var gotIxns structs.Intentions
for _, entry := range got {
gotIxns = append(gotIxns, entry.ToIntentions()...)
}
sort.Sort(structs.IntentionPrecedenceSorter(gotIxns))
compare(t, tc.expect)
compare(t, gotIxns, tc.expect)
})
}
}

View File

@ -0,0 +1,588 @@
package consul
import (
"os"
"strings"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
tokenStore "github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestLeader_ReplicateIntentions(t *testing.T) {
// This setup is a little hacky, but should work. We spin up BOTH servers with
// no intentions and force them to think they're not eligible for intentions
// config entries yet by overriding serf tags.
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
c.Build = "1.6.0"
c.OverrideInitialSerfTags = func(tags map[string]string) {
tags["ft_si"] = "0"
}
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
waitForLeaderEstablishment(t, s1)
retry.Run(t, func(r *retry.R) {
if s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 shouldn't activate service-intentions")
}
})
s1.tokens.UpdateAgentToken("root", tokenStore.TokenSourceConfig)
replicationRules := `acl = "read" service_prefix "" { policy = "read" intentions = "read" } operator = "write" `
// create some tokens
replToken1, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", replicationRules)
require.NoError(t, err)
replToken2, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", replicationRules)
require.NoError(t, err)
// dc2 as a secondary DC
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLDefaultPolicy = "deny"
c.ACLTokenReplication = false
c.Build = "1.6.0"
c.OverrideInitialSerfTags = func(tags map[string]string) {
tags["ft_si"] = "0"
}
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
s2.tokens.UpdateAgentToken("root", tokenStore.TokenSourceConfig)
// start out with one token
s2.tokens.UpdateReplicationToken(replToken1.SecretID, tokenStore.TokenSourceConfig)
// Create the WAN link
joinWAN(t, s2, s1)
waitForLeaderEstablishment(t, s2)
retry.Run(t, func(r *retry.R) {
if s2.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 2 shouldn't activate service-intentions")
}
})
legacyApply := func(s *Server, req *structs.IntentionRequest) error {
if req.Op != structs.IntentionOpDelete {
// Do these directly on the inputs so it's corrected for future
// equality checks.
req.Intention.CreatedAt = time.Now().UTC()
req.Intention.UpdatedAt = req.Intention.CreatedAt
//nolint:staticcheck
req.Intention.UpdatePrecedence()
//nolint:staticcheck
require.NoError(t, req.Intention.Validate())
//nolint:staticcheck
req.Intention.SetHash()
}
req2 := *req
req2.Intention = req.Intention.Clone()
if req.Op != structs.IntentionOpDelete {
req2.Intention.Hash = req.Intention.Hash // not part of Clone
}
resp, err := s.raftApply(structs.IntentionRequestType, req2)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
return nil
}
// Directly insert legacy intentions into raft in dc1.
id := generateUUID()
ixn := structs.IntentionRequest{
Datacenter: "dc1",
WriteRequest: structs.WriteRequest{Token: "root"},
Op: structs.IntentionOpCreate,
Intention: &structs.Intention{
ID: id,
SourceNS: structs.IntentionDefaultNamespace,
SourceName: "test",
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: "test",
Action: structs.IntentionActionAllow,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
},
}
require.NoError(t, legacyApply(s1, &ixn))
// Wait for it to get replicated to dc2
var createdAt time.Time
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
require.NoError(r, s2.RPC("Intention.Get", req, &resp), "ID=%q", ixn.Intention.ID)
require.Len(r, resp.Intentions, 1)
actual := resp.Intentions[0]
createdAt = actual.CreatedAt
})
// Sleep a bit so that the UpdatedAt field will definitely be different
time.Sleep(1 * time.Millisecond)
// delete underlying acl token being used for replication
require.NoError(t, deleteTestToken(codec, "root", "dc1", replToken1.AccessorID))
// switch to the other token
s2.tokens.UpdateReplicationToken(replToken2.SecretID, tokenStore.TokenSourceConfig)
// Update the intention in dc1
ixn.Op = structs.IntentionOpUpdate
ixn.Intention.ID = id
ixn.Intention.SourceName = "*"
require.NoError(t, legacyApply(s1, &ixn))
// Wait for dc2 to get the update
var resp structs.IndexedIntentions
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
require.NoError(r, s2.RPC("Intention.Get", req, &resp), "ID=%q", ixn.Intention.ID)
require.Len(r, resp.Intentions, 1)
require.Equal(r, "*", resp.Intentions[0].SourceName)
})
actual := resp.Intentions[0]
require.Equal(t, createdAt, actual.CreatedAt)
require.WithinDuration(t, time.Now(), actual.UpdatedAt, 5*time.Second)
actual.CreateIndex, actual.ModifyIndex = 0, 0
actual.CreatedAt = ixn.Intention.CreatedAt
actual.UpdatedAt = ixn.Intention.UpdatedAt
//nolint:staticcheck
ixn.Intention.UpdatePrecedence()
require.Equal(t, ixn.Intention, actual)
// Delete
require.NoError(t, legacyApply(s1, &structs.IntentionRequest{
Datacenter: "dc1",
WriteRequest: structs.WriteRequest{Token: "root"},
Op: structs.IntentionOpDelete,
Intention: &structs.Intention{
ID: ixn.Intention.ID,
},
}))
// Wait for the delete to be replicated
retry.Run(t, func(r *retry.R) {
req := &structs.IntentionQueryRequest{
Datacenter: "dc2",
QueryOptions: structs.QueryOptions{Token: "root"},
IntentionID: ixn.Intention.ID,
}
var resp structs.IndexedIntentions
err := s2.RPC("Intention.Get", req, &resp)
require.Error(r, err)
if !strings.Contains(err.Error(), ErrIntentionNotFound.Error()) {
r.Fatalf("expected intention not found, got: %v", err)
}
})
}
//nolint:staticcheck
func TestLeader_batchLegacyIntentionUpdates(t *testing.T) {
t.Parallel()
assert := assert.New(t)
ixn1 := structs.TestIntention(t)
ixn1.ID = "ixn1"
ixn2 := structs.TestIntention(t)
ixn2.ID = "ixn2"
ixnLarge := structs.TestIntention(t)
ixnLarge.ID = "ixnLarge"
ixnLarge.Description = strings.Repeat("x", maxIntentionTxnSize-1)
cases := []struct {
deletes structs.Intentions
updates structs.Intentions
expected []structs.TxnOps
}{
// 1 deletes, 0 updates
{
deletes: structs.Intentions{ixn1},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
},
},
},
// 0 deletes, 1 updates
{
updates: structs.Intentions{ixn1},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn1,
},
},
},
},
},
// 1 deletes, 1 updates
{
deletes: structs.Intentions{ixn1},
updates: structs.Intentions{ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
// 1 large intention update
{
updates: structs.Intentions{ixnLarge},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixnLarge,
},
},
},
},
},
// 2 deletes (w/ a large intention), 1 updates
{
deletes: structs.Intentions{ixn1, ixnLarge},
updates: structs.Intentions{ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixnLarge,
},
},
},
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
// 1 deletes , 2 updates (w/ a large intention)
{
deletes: structs.Intentions{ixn1},
updates: structs.Intentions{ixnLarge, ixn2},
expected: []structs.TxnOps{
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpDelete,
Intention: ixn1,
},
},
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixnLarge,
},
},
},
{
&structs.TxnOp{
Intention: &structs.TxnIntentionOp{
Op: structs.IntentionOpUpdate,
Intention: ixn2,
},
},
},
},
},
}
for _, tc := range cases {
actual := batchLegacyIntentionUpdates(tc.deletes, tc.updates)
assert.Equal(tc.expected, actual)
}
}
func TestLeader_LegacyIntentionMigration(t *testing.T) {
// This setup is a little hacky, but should work. We spin up a server with
// no intentions and force it to think it's not eligible for intentions
// config entries yet by overriding serf tags.
//
// Then we directly write legacy intentions into raft. This is mimicking
// what a service-intentions aware server might do if an older copy of
// consul was still leader.
//
// This lets us generate a snapshot+raft state containing legacy intentions
// without having to spin up an old version of consul for the test.
//
// Then we shut it down and bring up a new copy on that datadir which
// should then trigger migration code.
dir1pre, s1pre := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc1"
c.Build = "1.6.0"
c.OverrideInitialSerfTags = func(tags map[string]string) {
tags["ft_si"] = "0"
}
})
defer os.RemoveAll(dir1pre)
defer s1pre.Shutdown()
testrpc.WaitForLeader(t, s1pre.RPC, "dc1")
retry.Run(t, func(r *retry.R) {
if s1pre.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 shouldn't activate service-intentions")
}
})
// Insert a bunch of legacy intentions.
makeIxn := func(src, dest string, allow bool) *structs.Intention {
ixn := &structs.Intention{
ID: generateUUID(),
SourceNS: structs.IntentionDefaultNamespace,
SourceName: src,
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: dest,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
}
if allow {
ixn.Action = structs.IntentionActionAllow
} else {
ixn.Action = structs.IntentionActionDeny
}
//nolint:staticcheck
ixn.UpdatePrecedence()
//nolint:staticcheck
ixn.SetHash()
return ixn
}
ixns := []*structs.Intention{
makeIxn("api", "db", true),
makeIxn("web", "db", false),
makeIxn("*", "web", true),
makeIxn("*", "api", false),
makeIxn("intern", "*", false),
makeIxn("contractor", "*", false),
makeIxn("*", "*", true),
}
for _, ixn := range ixns {
ixn2 := *ixn
resp, err := s1pre.raftApply(structs.IntentionRequestType, &structs.IntentionRequest{
Op: structs.IntentionOpCreate,
Intention: &ixn2,
})
require.NoError(t, err)
if respErr, ok := resp.(error); ok {
t.Fatalf("respErr: %v", respErr)
}
}
mapify := func(ixns []*structs.Intention) map[string]*structs.Intention {
m := make(map[string]*structs.Intention)
for _, ixn := range ixns {
m[ixn.ID] = ixn
}
return m
}
checkIntentions := func(t *testing.T, srv *Server, legacyOnly bool, expect map[string]*structs.Intention) {
t.Helper()
wildMeta := structs.WildcardEnterpriseMeta()
retry.Run(t, func(r *retry.R) {
var (
got structs.Intentions
err error
)
if legacyOnly {
_, got, err = srv.fsm.State().LegacyIntentions(nil, wildMeta)
} else {
_, got, _, err = srv.fsm.State().Intentions(nil, wildMeta)
}
require.NoError(r, err)
gotM := mapify(got)
assert.Len(r, gotM, len(expect))
for k, expectV := range expect {
gotV, ok := gotM[k]
if !ok {
r.Errorf("results are missing key %q", k)
continue
}
assert.Equal(r, expectV.ID, gotV.ID)
assert.Equal(r, expectV.SourceNS, gotV.SourceNS)
assert.Equal(r, expectV.SourceName, gotV.SourceName)
assert.Equal(r, expectV.DestinationNS, gotV.DestinationNS)
assert.Equal(r, expectV.DestinationName, gotV.DestinationName)
assert.Equal(r, expectV.Action, gotV.Action)
assert.Equal(r, expectV.Meta, gotV.Meta)
assert.Equal(r, expectV.Precedence, gotV.Precedence)
assert.Equal(r, expectV.SourceType, gotV.SourceType)
}
})
}
expectM := mapify(ixns)
checkIntentions(t, s1pre, false, expectM)
checkIntentions(t, s1pre, true, expectM)
// Shutdown s1pre and restart it to trigger migration.
s1pre.Shutdown()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DataDir = s1pre.config.DataDir
c.Datacenter = "dc1"
c.NodeName = s1pre.config.NodeName
c.NodeID = s1pre.config.NodeID
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// check that all 7 intentions are present before migration
checkIntentions(t, s1, false, expectM)
// Wait until the migration routine is complete.
retry.Run(t, func(r *retry.R) {
intentionFormat, err := s1.getSystemMetadata(structs.SystemMetadataIntentionFormatKey)
require.NoError(r, err)
if intentionFormat != structs.SystemMetadataIntentionFormatConfigValue {
r.Fatal("intention migration is not yet complete")
}
})
// check that all 7 intentions are present the general way after migration
checkIntentions(t, s1, false, expectM)
// check that no intentions exist in the legacy table
checkIntentions(t, s1, true, map[string]*structs.Intention{})
mapifyConfigs := func(entries interface{}) map[structs.ConfigEntryKindName]*structs.ServiceIntentionsConfigEntry {
m := make(map[structs.ConfigEntryKindName]*structs.ServiceIntentionsConfigEntry)
switch v := entries.(type) {
case []*structs.ServiceIntentionsConfigEntry:
for _, entry := range v {
kn := structs.NewConfigEntryKindName(entry.Kind, entry.Name, &entry.EnterpriseMeta)
m[kn] = entry
}
case []structs.ConfigEntry:
for _, entry := range v {
kn := structs.NewConfigEntryKindName(entry.GetKind(), entry.GetName(), entry.GetEnterpriseMeta())
m[kn] = entry.(*structs.ServiceIntentionsConfigEntry)
}
default:
t.Fatalf("bad type: %T", entries)
}
return m
}
// also check config entries
_, gotConfigs, err := s1.fsm.State().ConfigEntriesByKind(nil, structs.ServiceIntentions, structs.WildcardEnterpriseMeta())
require.NoError(t, err)
gotConfigsM := mapifyConfigs(gotConfigs)
expectConfigs := structs.MigrateIntentions(ixns)
for _, entry := range expectConfigs {
require.NoError(t, entry.LegacyNormalize()) // tidy them up the same way the write would
}
expectConfigsM := mapifyConfigs(expectConfigs)
assert.Len(t, gotConfigsM, len(expectConfigsM))
for kn, expectV := range expectConfigsM {
gotV, ok := gotConfigsM[kn]
if !ok {
t.Errorf("results are missing key %q", kn)
continue
}
// Migrated intentions won't have toplevel Meta.
assert.Nil(t, gotV.Meta)
require.Len(t, gotV.Sources, len(expectV.Sources))
expSrcMap := make(map[string]*structs.SourceIntention)
for i, src := range expectV.Sources {
require.NotEmpty(t, src.LegacyID, "index[%d] missing LegacyID", i)
// Do a shallow copy and strip the times from the copy
src2 := *src
src2.LegacyCreateTime = nil
src2.LegacyUpdateTime = nil
expSrcMap[src2.LegacyID] = &src2
}
for i, got := range gotV.Sources {
require.NotEmpty(t, got.LegacyID, "index[%d] missing LegacyID", i)
// Do a shallow copy and strip the times from the copy
got2 := *got
got2.LegacyCreateTime = nil
got2.LegacyUpdateTime = nil
cmp, ok := expSrcMap[got2.LegacyID]
require.True(t, ok, "missing %q", got2.LegacyID)
assert.Equal(t, cmp, &got2, "index[%d]", i)
}
}
}

View File

@ -1271,7 +1271,7 @@ func TestLeader_ConfigEntryBootstrap_Fail(t *testing.T) {
ch <- ""
return
}
if strings.Contains(line, "initialized primary datacenter") {
if strings.Contains(line, "successfully established leadership") {
ch <- "leadership should not have gotten here if config entries properly failed"
return
}
@ -1632,3 +1632,351 @@ func TestDatacenterSupportsFederationStates(t *testing.T) {
})
})
}
func TestDatacenterSupportsIntentionsAsConfigEntries(t *testing.T) {
addLegacyIntention := func(srv *Server, dc, src, dest string, allow bool) error {
ixn := &structs.Intention{
SourceNS: structs.IntentionDefaultNamespace,
SourceName: src,
DestinationNS: structs.IntentionDefaultNamespace,
DestinationName: dest,
SourceType: structs.IntentionSourceConsul,
Meta: map[string]string{},
}
if allow {
ixn.Action = structs.IntentionActionAllow
} else {
ixn.Action = structs.IntentionActionDeny
}
//nolint:staticcheck
ixn.UpdatePrecedence()
//nolint:staticcheck
ixn.SetHash()
arg := structs.IntentionRequest{
Datacenter: dc,
Op: structs.IntentionOpCreate,
Intention: ixn,
}
var id string
return srv.RPC("Intention.Apply", &arg, &id)
}
getConfigEntry := func(srv *Server, dc, kind, name string) (structs.ConfigEntry, error) {
arg := structs.ConfigEntryQuery{
Datacenter: dc,
Kind: kind,
Name: name,
}
var reply structs.ConfigEntryResponse
if err := srv.RPC("ConfigEntry.Get", &arg, &reply); err != nil {
return nil, err
}
return reply.Entry, nil
}
disableServiceIntentions := func(tags map[string]string) {
tags["ft_si"] = "0"
}
defaultEntMeta := structs.DefaultEnterpriseMeta()
t.Run("one node primary with old version", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.OverrideInitialSerfTags = disableServiceIntentions
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
retry.Run(t, func(r *retry.R) {
if s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 shouldn't activate service-intentions")
}
})
testutil.RequireErrorContains(t,
addLegacyIntention(s1, "dc1", "web", "api", true),
ErrIntentionsNotUpgradedYet.Error(),
)
})
t.Run("one node primary with new version", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
retry.Run(t, func(r *retry.R) {
if !s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 didn't activate service-intentions")
}
})
// try to write a using the legacy API and it should work
require.NoError(t, addLegacyIntention(s1, "dc1", "web", "api", true))
// read it back as a config entry and that should work too
raw, err := getConfigEntry(s1, "dc1", structs.ServiceIntentions, "api")
require.NoError(t, err)
require.NotNil(t, raw)
got, ok := raw.(*structs.ServiceIntentionsConfigEntry)
require.True(t, ok)
require.Len(t, got.Sources, 1)
expect := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "api",
EnterpriseMeta: *defaultEntMeta,
Sources: []*structs.SourceIntention{
{
Name: "web",
EnterpriseMeta: *defaultEntMeta,
Action: structs.IntentionActionAllow,
Type: structs.IntentionSourceConsul,
Precedence: 9,
LegacyMeta: map[string]string{},
LegacyID: got.Sources[0].LegacyID,
// steal
LegacyCreateTime: got.Sources[0].LegacyCreateTime,
LegacyUpdateTime: got.Sources[0].LegacyUpdateTime,
},
},
RaftIndex: got.RaftIndex,
}
require.Equal(t, expect, got)
})
t.Run("two node primary with mixed versions", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.OverrideInitialSerfTags = disableServiceIntentions
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node2"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.Bootstrap = false
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Put s1 last so we don't trigger a leader election.
servers := []*Server{s2, s1}
// Try to join
joinLAN(t, s2, s1)
for _, s := range servers {
retry.Run(t, func(r *retry.R) { r.Check(wantPeers(s, 2)) })
}
waitForLeaderEstablishment(t, s1)
retry.Run(t, func(r *retry.R) {
if s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 shouldn't activate service-intentions")
}
})
retry.Run(t, func(r *retry.R) {
if s2.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 2 shouldn't activate service-intentions")
}
})
testutil.RequireErrorContains(t,
addLegacyIntention(s1, "dc1", "web", "api", true),
ErrIntentionsNotUpgradedYet.Error(),
)
testutil.RequireErrorContains(t,
addLegacyIntention(s2, "dc1", "web", "api", true),
ErrIntentionsNotUpgradedYet.Error(),
)
})
t.Run("two node primary with new version", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node2"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.Bootstrap = false
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Put s1 last so we don't trigger a leader election.
servers := []*Server{s2, s1}
// Try to join
joinLAN(t, s2, s1)
for _, s := range servers {
retry.Run(t, func(r *retry.R) { r.Check(wantPeers(s, 2)) })
}
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s2.RPC, "dc1")
retry.Run(t, func(r *retry.R) {
if !s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 didn't activate service-intentions")
}
})
retry.Run(t, func(r *retry.R) {
if !s2.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 2 didn't activate service-intentions")
}
})
// try to write a using the legacy API and it should work from both sides
require.NoError(t, addLegacyIntention(s1, "dc1", "web", "api", true))
require.NoError(t, addLegacyIntention(s2, "dc1", "web2", "api", true))
// read it back as a config entry and that should work too
raw, err := getConfigEntry(s1, "dc1", structs.ServiceIntentions, "api")
require.NoError(t, err)
require.NotNil(t, raw)
raw, err = getConfigEntry(s2, "dc1", structs.ServiceIntentions, "api")
require.NoError(t, err)
require.NotNil(t, raw)
})
t.Run("primary and secondary with new version", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node2"
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.ConfigReplicationRate = 100
c.ConfigReplicationBurst = 100
c.ConfigReplicationApplyLimit = 1000000
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
waitForLeaderEstablishment(t, s2)
// Try to join
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
retry.Run(t, func(r *retry.R) {
if !s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 didn't activate service-intentions")
}
})
retry.Run(t, func(r *retry.R) {
if !s2.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 2 didn't activate service-intentions")
}
})
// try to write a using the legacy API
require.NoError(t, addLegacyIntention(s1, "dc1", "web", "api", true))
// read it back as a config entry and that should work too
raw, err := getConfigEntry(s1, "dc1", structs.ServiceIntentions, "api")
require.NoError(t, err)
require.NotNil(t, raw)
// Wait until after replication runs for the secondary.
retry.Run(t, func(r *retry.R) {
raw, err = getConfigEntry(s2, "dc1", structs.ServiceIntentions, "api")
require.NoError(r, err)
require.NotNil(r, raw)
})
})
t.Run("primary and secondary with mixed versions", func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node1"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.OverrideInitialSerfTags = disableServiceIntentions
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
waitForLeaderEstablishment(t, s1)
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "node2"
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.ConfigReplicationRate = 100
c.ConfigReplicationBurst = 100
c.ConfigReplicationApplyLimit = 1000000
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
waitForLeaderEstablishment(t, s2)
// Try to join
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
retry.Run(t, func(r *retry.R) {
if s1.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 1 shouldn't activate service-intentions")
}
})
retry.Run(t, func(r *retry.R) {
if s2.DatacenterSupportsIntentionsAsConfigEntries() {
r.Fatal("server 2 shouldn't activate service-intentions")
}
})
testutil.RequireErrorContains(t,
addLegacyIntention(s1, "dc1", "web", "api", true),
ErrIntentionsNotUpgradedYet.Error(),
)
testutil.RequireErrorContains(t,
addLegacyIntention(s2, "dc1", "web", "api", true),
ErrIntentionsNotUpgradedYet.Error(),
)
})
}

View File

@ -100,7 +100,7 @@ const (
federationStateReplicationRoutineName = "federation state replication"
federationStateAntiEntropyRoutineName = "federation state anti-entropy"
federationStatePruningRoutineName = "federation state pruning"
intentionReplicationRoutineName = "intention replication"
intentionMigrationRoutineName = "intention config entry migration"
secondaryCARootWatchRoutineName = "secondary CA roots watch"
secondaryCertRenewWatchRoutineName = "secondary cert renew watch"
)
@ -301,6 +301,12 @@ type Server struct {
actingSecondaryCA bool
actingSecondaryLock sync.RWMutex
// dcSupportsIntentionsAsConfigEntries is used to determine whether we can
// migrate old intentions into service-intentions config entries. All
// servers in the local DC must be on a version of Consul supporting
// service-intentions before this will get enabled.
dcSupportsIntentionsAsConfigEntries int32
// Manager to handle starting/stopping go routines when establishing/revoking raft leadership
leaderRoutineManager *LeaderRoutineManager
@ -1425,10 +1431,6 @@ func (s *Server) isReadyForConsistentReads() bool {
return atomic.LoadInt32(&s.readyForConsistentReads) == 1
}
func (s *Server) intentionReplicationEnabled() bool {
return s.config.ConnectEnabled && s.config.Datacenter != s.config.PrimaryDatacenter
}
// CreateACLToken will create an ACL token from the given template
func (s *Server) CreateACLToken(template *structs.ACLToken) (*structs.ACLToken, error) {
// we have to require local tokens or else it would require having these servers use a token with acl:write to make a

View File

@ -13,7 +13,7 @@ func init() {
registerEndpoint(func(s *Server) interface{} { return &FederationState{s} })
registerEndpoint(func(s *Server) interface{} { return &DiscoveryChain{s} })
registerEndpoint(func(s *Server) interface{} { return &Health{s} })
registerEndpoint(func(s *Server) interface{} { return &Intention{s, s.loggers.Named(logging.Intentions)} })
registerEndpoint(func(s *Server) interface{} { return NewIntentionEndpoint(s, s.loggers.Named(logging.Intentions)) })
registerEndpoint(func(s *Server) interface{} { return &Internal{s, s.loggers.Named(logging.Internal)} })
registerEndpoint(func(s *Server) interface{} { return &KVS{s, s.loggers.Named(logging.KV)} })
registerEndpoint(func(s *Server) interface{} { return &Operator{s, s.loggers.Named(logging.Operator)} })

View File

@ -77,6 +77,9 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string, w
// feature flag: advertise support for federation states
conf.Tags["ft_fs"] = "1"
// feature flag: advertise support for service-intentions
conf.Tags["ft_si"] = "1"
var subLoggerName string
if wan {
subLoggerName = logging.WAN
@ -167,6 +170,10 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string, w
s.addEnterpriseSerfTags(conf.Tags)
if s.config.OverrideInitialSerfTags != nil {
s.config.OverrideInitialSerfTags(conf.Tags)
}
return serf.Create(conf)
}

View File

@ -144,8 +144,8 @@ func (s *Store) ConfigEntriesByKind(ws memdb.WatchSet, kind string, entMeta *str
}
func configEntriesByKindTxn(tx ReadTxn, ws memdb.WatchSet, kind string, entMeta *structs.EnterpriseMeta) (uint64, []structs.ConfigEntry, error) {
// Get the index
idx := maxIndexTxn(tx, configTableName)
// Get the index and watch for updates
idx := maxIndexWatchTxn(tx, ws, configTableName)
// Lookup by kind, or all if kind is empty
var iter memdb.ResultIterator
@ -191,7 +191,13 @@ func ensureConfigEntryTxn(tx *txn, idx uint64, conf structs.ConfigEntry, entMeta
if existing != nil {
existingIdx := existing.(structs.ConfigEntry).GetRaftIndex()
raftIndex.CreateIndex = existingIdx.CreateIndex
raftIndex.ModifyIndex = existingIdx.ModifyIndex
// Handle optional upsert logic.
if updatableConf, ok := conf.(structs.UpdatableConfigEntry); ok {
if err := updatableConf.UpdateOver(existing.(structs.ConfigEntry)); err != nil {
return err
}
}
} else {
raftIndex.CreateIndex = idx
}
@ -275,7 +281,7 @@ func (s *Store) DeleteConfigEntry(idx uint64, kind, name string, entMeta *struct
// Delete the config entry from the DB and update the index.
if err := tx.Delete(configTableName, existing); err != nil {
return fmt.Errorf("failed removing check: %s", err)
return fmt.Errorf("failed removing config entry: %s", err)
}
if err := tx.Insert("index", &IndexEntry{configTableName, idx}); err != nil {
return fmt.Errorf("failed updating index: %s", err)
@ -321,7 +327,6 @@ func validateProposedConfigEntryInGraph(
next structs.ConfigEntry,
entMeta *structs.EnterpriseMeta,
) error {
validateAllChains := false
switch kind {
@ -344,6 +349,10 @@ func validateProposedConfigEntryInGraph(
if err != nil {
return err
}
case structs.ServiceIntentions:
// TODO(rb): should this validate protocols?
return nil
default:
return fmt.Errorf("unhandled kind %q during validation of %q", kind, name)
}
@ -1043,6 +1052,31 @@ func getResolverConfigEntryTxn(
return idx, resolver, nil
}
// getServiceIntentionsConfigEntryTxn is a convenience method for fetching a
// service-intentions kind of config entry.
//
// If an override is returned the index returned will be 0.
func getServiceIntentionsConfigEntryTxn(
tx ReadTxn,
ws memdb.WatchSet,
name string,
overrides map[structs.ConfigEntryKindName]structs.ConfigEntry,
entMeta *structs.EnterpriseMeta,
) (uint64, *structs.ServiceIntentionsConfigEntry, error) {
idx, entry, err := configEntryWithOverridesTxn(tx, ws, structs.ServiceIntentions, name, overrides, entMeta)
if err != nil {
return 0, nil, err
} else if entry == nil {
return idx, nil, nil
}
ixn, ok := entry.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return 0, nil, fmt.Errorf("invalid service config type %T", entry)
}
return idx, ixn, nil
}
func configEntryWithOverridesTxn(
tx ReadTxn,
ws memdb.WatchSet,

View File

@ -0,0 +1,310 @@
package state
import (
"encoding/json"
"fmt"
"sort"
"github.com/hashicorp/consul/agent/structs"
memdb "github.com/hashicorp/go-memdb"
)
type ServiceIntentionLegacyIDIndex struct {
uuidFieldIndex memdb.UUIDFieldIndex // for helper code
}
func (s *ServiceIntentionLegacyIDIndex) FromObject(obj interface{}) (bool, [][]byte, error) {
entry, ok := obj.(structs.ConfigEntry)
if !ok {
return false, nil, fmt.Errorf("object is not a ConfigEntry")
}
if entry.GetKind() != structs.ServiceIntentions {
return false, nil, nil
}
ixnEntry, ok := entry.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return false, nil, nil
}
// We don't pre-size this slice because it will only be populated
// for legacy data, which should reduce over time.
var vals [][]byte
for _, src := range ixnEntry.Sources {
if src.LegacyID != "" {
arg, err := s.FromArgs(src.LegacyID)
if err != nil {
return false, nil, err
}
vals = append(vals, arg)
}
}
if len(vals) == 0 {
return false, nil, nil
}
return true, vals, nil
}
func (s *ServiceIntentionLegacyIDIndex) FromArgs(args ...interface{}) ([]byte, error) {
arg, err := s.uuidFieldIndex.FromArgs(args...)
if err != nil {
return nil, err
}
// Add the null character as a terminator
b := make([]byte, 0, len(arg)+1)
b = append(b, arg...)
b = append(b, '\x00')
return b, nil
}
func (s *ServiceIntentionLegacyIDIndex) PrefixFromArgs(args ...interface{}) ([]byte, error) {
val, err := s.FromArgs(args...)
if err != nil {
return nil, err
}
// Strip the null terminator, the rest is a prefix
n := len(val)
if n > 0 {
return val[:n-1], nil
}
return val, nil
}
type ServiceIntentionSourceIndex struct {
}
// Compile-time assert that these interfaces hold to ensure that the
// methods correctly exist across the oss/ent split.
var _ memdb.Indexer = (*ServiceIntentionSourceIndex)(nil)
var _ memdb.MultiIndexer = (*ServiceIntentionSourceIndex)(nil)
func (s *ServiceIntentionSourceIndex) FromObject(obj interface{}) (bool, [][]byte, error) {
entry, ok := obj.(structs.ConfigEntry)
if !ok {
return false, nil, fmt.Errorf("object is not a ConfigEntry")
}
if entry.GetKind() != structs.ServiceIntentions {
return false, nil, nil
}
ixnEntry, ok := entry.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return false, nil, nil
}
vals := make([][]byte, 0, len(ixnEntry.Sources))
for _, src := range ixnEntry.Sources {
sn := src.SourceServiceName()
vals = append(vals, []byte(sn.String()+"\x00"))
}
if len(vals) == 0 {
return false, nil, nil
}
return true, vals, nil
}
func (s *ServiceIntentionSourceIndex) FromArgs(args ...interface{}) ([]byte, error) {
if len(args) != 1 {
return nil, fmt.Errorf("must provide only a single argument")
}
arg, ok := args[0].(structs.ServiceName)
if !ok {
return nil, fmt.Errorf("argument must be a structs.ServiceID: %#v", args[0])
}
// Add the null character as a terminator
return []byte(arg.String() + "\x00"), nil
}
func (s *Store) configIntentionsListTxn(tx *txn, ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, bool, error) {
// unrolled part of configEntriesByKindTxn
idx := maxIndexTxn(tx, configTableName)
iter, err := getConfigEntryKindsWithTxn(tx, structs.ServiceIntentions, structs.WildcardEnterpriseMeta())
if err != nil {
return 0, nil, false, fmt.Errorf("failed config entry lookup: %s", err)
}
ws.Add(iter.WatchCh())
results := configIntentionsConvertToList(iter, entMeta)
// Sort by precedence just because that's nicer and probably what most clients
// want for presentation.
sort.Sort(structs.IntentionPrecedenceSorter(results))
return idx, results, true, nil
}
func (s *Store) configIntentionGetTxn(tx *txn, ws memdb.WatchSet, id string) (uint64, *structs.ServiceIntentionsConfigEntry, *structs.Intention, error) {
idx := maxIndexTxn(tx, configTableName)
if idx < 1 {
idx = 1
}
watchCh, existing, err := tx.FirstWatch(configTableName, "intention-legacy-id", id)
if err != nil {
return 0, nil, nil, fmt.Errorf("failed config entry lookup: %s", err)
}
ws.Add(watchCh)
if existing == nil {
return idx, nil, nil, nil
}
conf, ok := existing.(*structs.ServiceIntentionsConfigEntry)
if !ok {
return 0, nil, nil, fmt.Errorf("config entry is an invalid type: %T", conf)
}
for _, src := range conf.Sources {
if src.LegacyID == id {
return idx, conf, conf.ToIntention(src), nil
}
}
return idx, nil, nil, nil // Shouldn't happen.
}
func (s *Store) configIntentionGetExactTxn(tx *txn, ws memdb.WatchSet, args *structs.IntentionQueryExact) (uint64, *structs.ServiceIntentionsConfigEntry, *structs.Intention, error) {
if err := args.Validate(); err != nil {
return 0, nil, nil, err
}
idx, entry, err := getServiceIntentionsConfigEntryTxn(tx, ws, args.DestinationName, nil, args.DestinationEnterpriseMeta())
if err != nil {
return 0, nil, nil, err
} else if entry == nil {
return idx, nil, nil, nil
}
sn := structs.NewServiceName(args.SourceName, args.SourceEnterpriseMeta())
for _, src := range entry.Sources {
if sn == src.SourceServiceName() {
return idx, entry, entry.ToIntention(src), nil
}
}
return idx, nil, nil, nil
}
func (s *Store) configIntentionMatchTxn(tx *txn, ws memdb.WatchSet, args *structs.IntentionQueryMatch) (uint64, []structs.Intentions, error) {
maxIndex := uint64(1)
// Make all the calls and accumulate the results
results := make([]structs.Intentions, len(args.Entries))
for i, entry := range args.Entries {
// Note on performance: This is not the most optimal set of queries
// since we repeat some many times (such as */*). We can work on
// improving that in the future, the test cases shouldn't have to
// change for that.
index, ixns, err := s.configIntentionMatchOneTxn(tx, ws, entry, args.Type)
if err != nil {
return 0, nil, err
}
if index > maxIndex {
maxIndex = index
}
// Store the result
results[i] = ixns
}
return maxIndex, results, nil
}
func (s *Store) configIntentionMatchOneTxn(
tx *txn,
ws memdb.WatchSet,
matchEntry structs.IntentionMatchEntry,
matchType structs.IntentionMatchType,
) (uint64, structs.Intentions, error) {
switch matchType {
case structs.IntentionMatchSource:
return s.readSourceIntentionsFromConfigEntriesTxn(tx, ws, matchEntry.Name, matchEntry.GetEnterpriseMeta())
case structs.IntentionMatchDestination:
return s.readDestinationIntentionsFromConfigEntriesTxn(tx, ws, matchEntry.Name, matchEntry.GetEnterpriseMeta())
default:
return 0, nil, fmt.Errorf("invalid intention match type: %s", matchType)
}
}
func (s *Store) readSourceIntentionsFromConfigEntriesTxn(tx *txn, ws memdb.WatchSet, serviceName string, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, error) {
idx := maxIndexTxn(tx, configTableName)
var (
results structs.Intentions
err error
)
names := getIntentionPrecedenceMatchServiceNames(serviceName, entMeta)
for _, sn := range names {
results, err = s.readSourceIntentionsFromConfigEntriesForServiceTxn(
tx, ws, sn.Name, &sn.EnterpriseMeta, results,
)
if err != nil {
return 0, nil, err
}
}
// Sort the results by precedence
sort.Sort(structs.IntentionPrecedenceSorter(results))
return idx, results, nil
}
func (s *Store) readSourceIntentionsFromConfigEntriesForServiceTxn(tx *txn, ws memdb.WatchSet, serviceName string, entMeta *structs.EnterpriseMeta, results structs.Intentions) (structs.Intentions, error) {
sn := structs.NewServiceName(serviceName, entMeta)
iter, err := tx.Get(configTableName, "intention-source", sn)
if err != nil {
return nil, fmt.Errorf("failed config entry lookup: %s", err)
}
ws.Add(iter.WatchCh())
for v := iter.Next(); v != nil; v = iter.Next() {
entry := v.(*structs.ServiceIntentionsConfigEntry)
for _, src := range entry.Sources {
if src.SourceServiceName() == sn {
results = append(results, entry.ToIntention(src))
}
}
}
return results, nil
}
func jd(v interface{}) string {
d, _ := json.MarshalIndent(v, "", " ")
return string(d)
}
func (s *Store) readDestinationIntentionsFromConfigEntriesTxn(tx *txn, ws memdb.WatchSet, serviceName string, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, error) {
idx := maxIndexTxn(tx, configTableName)
var results structs.Intentions
names := getIntentionPrecedenceMatchServiceNames(serviceName, entMeta)
for _, sn := range names {
_, entry, err := getServiceIntentionsConfigEntryTxn(tx, ws, sn.Name, nil, &sn.EnterpriseMeta)
if err != nil {
return 0, nil, err
} else if entry != nil {
results = append(results, entry.ToIntentions()...)
}
}
// Sort the results by precedence
sort.Sort(structs.IntentionPrecedenceSorter(results))
return idx, results, nil
}

View File

@ -0,0 +1,20 @@
// +build !consulent
package state
import (
"github.com/hashicorp/consul/agent/structs"
)
func getIntentionPrecedenceMatchServiceNames(serviceName string, entMeta *structs.EnterpriseMeta) []structs.ServiceName {
if serviceName == structs.WildcardSpecifier {
return []structs.ServiceName{
structs.NewServiceName(structs.WildcardSpecifier, entMeta),
}
}
return []structs.ServiceName{
structs.NewServiceName(serviceName, entMeta),
structs.NewServiceName(structs.WildcardSpecifier, entMeta),
}
}

View File

@ -45,6 +45,18 @@ func configTableSchema() *memdb.TableSchema {
Unique: false,
Indexer: &ConfigEntryLinkIndex{},
},
"intention-legacy-id": {
Name: "intention-legacy-id",
AllowMissing: true,
Unique: true,
Indexer: &ServiceIntentionLegacyIDIndex{},
},
"intention-source": {
Name: "intention-source",
AllowMissing: true,
Unique: false,
Indexer: &ServiceIntentionSourceIndex{},
},
},
}
}
@ -73,3 +85,14 @@ func getAllConfigEntriesWithTxn(tx ReadTxn, _ *structs.EnterpriseMeta) (memdb.Re
func getConfigEntryKindsWithTxn(tx ReadTxn, kind string, _ *structs.EnterpriseMeta) (memdb.ResultIterator, error) {
return tx.Get(configTableName, "kind", kind)
}
func configIntentionsConvertToList(iter memdb.ResultIterator, _ *structs.EnterpriseMeta) structs.Intentions {
var results structs.Intentions
for v := iter.Next(); v != nil; v = iter.Next() {
entry := v.(*structs.ServiceIntentionsConfigEntry)
for _, src := range entry.Sources {
results = append(results, entry.ToIntention(src))
}
}
return results
}

View File

@ -5,13 +5,14 @@ import (
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil"
memdb "github.com/hashicorp/go-memdb"
"github.com/stretchr/testify/require"
)
func TestStore_ConfigEntry(t *testing.T) {
require := require.New(t)
s := testStateStore(t)
s := testConfigStateStore(t)
expected := &structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
@ -75,7 +76,7 @@ func TestStore_ConfigEntry(t *testing.T) {
func TestStore_ConfigEntryCAS(t *testing.T) {
require := require.New(t)
s := testStateStore(t)
s := testConfigStateStore(t)
expected := &structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
@ -123,9 +124,105 @@ func TestStore_ConfigEntryCAS(t *testing.T) {
require.Equal(updated, config)
}
func TestStore_ConfigEntry_UpdateOver(t *testing.T) {
// This test uses ServiceIntentions because they are the only
// kind that implements UpdateOver() at this time.
s := testConfigStateStore(t)
var (
idA = testUUID()
idB = testUUID()
loc = time.FixedZone("UTC-8", -8*60*60)
timeA = time.Date(1955, 11, 5, 6, 15, 0, 0, loc)
timeB = time.Date(1985, 10, 26, 1, 35, 0, 0, loc)
)
require.NotEqual(t, idA, idB)
initial := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "api",
Sources: []*structs.SourceIntention{
{
LegacyID: idA,
Name: "web",
Action: structs.IntentionActionAllow,
LegacyCreateTime: &timeA,
LegacyUpdateTime: &timeA,
},
},
}
// Create
nextIndex := uint64(1)
require.NoError(t, s.EnsureConfigEntry(nextIndex, initial.Clone(), nil))
idx, raw, err := s.ConfigEntry(nil, structs.ServiceIntentions, "api", nil)
require.NoError(t, err)
require.Equal(t, nextIndex, idx)
got, ok := raw.(*structs.ServiceIntentionsConfigEntry)
require.True(t, ok)
initial.RaftIndex = got.RaftIndex
require.Equal(t, initial, got)
t.Run("update and fail change legacyID", func(t *testing.T) {
// Update
updated := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "api",
Sources: []*structs.SourceIntention{
{
LegacyID: idB,
Name: "web",
Action: structs.IntentionActionDeny,
LegacyCreateTime: &timeB,
LegacyUpdateTime: &timeB,
},
},
}
nextIndex++
err := s.EnsureConfigEntry(nextIndex, updated.Clone(), nil)
testutil.RequireErrorContains(t, err, "cannot set this field to a different value")
})
t.Run("update and do not update create time", func(t *testing.T) {
// Update
updated := &structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "api",
Sources: []*structs.SourceIntention{
{
LegacyID: idA,
Name: "web",
Action: structs.IntentionActionDeny,
LegacyCreateTime: &timeB,
LegacyUpdateTime: &timeB,
},
},
}
nextIndex++
require.NoError(t, s.EnsureConfigEntry(nextIndex, updated.Clone(), nil))
// check
idx, raw, err = s.ConfigEntry(nil, structs.ServiceIntentions, "api", nil)
require.NoError(t, err)
require.Equal(t, nextIndex, idx)
got, ok = raw.(*structs.ServiceIntentionsConfigEntry)
require.True(t, ok)
updated.RaftIndex = got.RaftIndex
updated.Sources[0].LegacyCreateTime = &timeA // UpdateOver will not replace this
require.Equal(t, updated, got)
})
}
func TestStore_ConfigEntries(t *testing.T) {
require := require.New(t)
s := testStateStore(t)
s := testConfigStateStore(t)
// Create some config entries.
entry1 := &structs.ProxyConfigEntry{
@ -837,7 +934,7 @@ func TestStore_ConfigEntry_GraphValidation(t *testing.T) {
tc := tc
t.Run(name, func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
for _, entry := range tc.entries {
require.NoError(t, entry.Normalize())
require.NoError(t, s.EnsureConfigEntry(0, entry, nil))
@ -1146,7 +1243,7 @@ func TestStore_ReadDiscoveryChainConfigEntries_Overrides(t *testing.T) {
tc := tc
t.Run(tc.name, func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
for _, entry := range tc.entries {
require.NoError(t, s.EnsureConfigEntry(0, entry, nil))
}
@ -1212,7 +1309,7 @@ func entrySetToKindNames(entrySet *structs.DiscoveryChainConfigEntries) []struct
}
func TestStore_ReadDiscoveryChainConfigEntries_SubsetSplit(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
entries := []structs.ConfigEntry{
&structs.ServiceConfigEntry{
@ -1255,8 +1352,10 @@ func TestStore_ReadDiscoveryChainConfigEntries_SubsetSplit(t *testing.T) {
require.Len(t, entrySet.Services, 1)
}
// TODO(rb): add ServiceIntentions tests
func TestStore_ValidateGatewayNamesCannotBeShared(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
ingress := &structs.IngressGatewayConfigEntry{
Kind: structs.IngressGateway,
@ -1306,7 +1405,7 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
}
t.Run("http ingress fails with http upstream later changed to tcp", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
// First set the target service as http
expected := &structs.ServiceConfigEntry{
@ -1340,7 +1439,7 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("tcp ingress ok with tcp upstream (defaulted) later changed to http", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
// First configure tcp ingress to route to a defaulted tcp service
require.NoError(t, s.EnsureConfigEntry(0, newIngress("tcp", "web"), nil))
@ -1355,7 +1454,7 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("tcp ingress fails with tcp upstream (defaulted) later changed to http", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
// First configure tcp ingress to route to a defaulted tcp service
require.NoError(t, s.EnsureConfigEntry(0, newIngress("tcp", "web"), nil))
@ -1395,14 +1494,14 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("http ingress fails with tcp upstream (defaulted)", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
err := s.EnsureConfigEntry(0, newIngress("http", "web"), nil)
require.Error(t, err)
require.Contains(t, err.Error(), `has protocol "tcp"`)
})
t.Run("http ingress fails with http2 upstream (via proxy-defaults)", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
expected := &structs.ProxyConfigEntry{
Kind: structs.ProxyDefaults,
Name: "global",
@ -1418,7 +1517,7 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("http ingress fails with grpc upstream (via service-defaults)", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
expected := &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "web",
@ -1431,7 +1530,7 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("http ingress ok with http upstream (via service-defaults)", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
expected := &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "web",
@ -1442,12 +1541,12 @@ func TestStore_ValidateIngressGatewayErrorOnMismatchedProtocols(t *testing.T) {
})
t.Run("http ingress ignores wildcard specifier", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
require.NoError(t, s.EnsureConfigEntry(4, newIngress("http", "*"), nil))
})
t.Run("deleting ingress config entry ok", func(t *testing.T) {
s := testStateStore(t)
s := testConfigStateStore(t)
require.NoError(t, s.EnsureConfigEntry(1, newIngress("tcp", "web"), nil))
require.NoError(t, s.DeleteConfigEntry(5, structs.IngressGateway, "gateway", nil))
})

View File

@ -1,6 +1,7 @@
package state
import (
"errors"
"fmt"
"sort"
@ -97,8 +98,11 @@ func init() {
registerSchema(intentionsTableSchema)
}
// Intentions is used to pull all the intentions from the snapshot.
func (s *Snapshot) Intentions() (structs.Intentions, error) {
// LegacyIntentions is used to pull all the intentions from the snapshot.
//
// Deprecated: service-intentions config entries are handled as config entries
// in the snapshot.
func (s *Snapshot) LegacyIntentions() (structs.Intentions, error) {
ixns, err := s.tx.Get(intentionsTableName, "id")
if err != nil {
return nil, err
@ -112,8 +116,11 @@ func (s *Snapshot) Intentions() (structs.Intentions, error) {
return ret, nil
}
// Intention is used when restoring from a snapshot.
func (s *Restore) Intention(ixn *structs.Intention) error {
// LegacyIntention is used when restoring from a snapshot.
//
// Deprecated: service-intentions config entries are handled as config entries
// in the snapshot.
func (s *Restore) LegacyIntention(ixn *structs.Intention) error {
// Insert the intention
if err := s.tx.Insert(intentionsTableName, ixn); err != nil {
return fmt.Errorf("failed restoring intention: %s", err)
@ -125,11 +132,51 @@ func (s *Restore) Intention(ixn *structs.Intention) error {
return nil
}
// Intentions returns the list of all intentions.
func (s *Store) Intentions(ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, error) {
// AreIntentionsInConfigEntries determines which table is the canonical store
// for intentions data.
func (s *Store) AreIntentionsInConfigEntries() (bool, error) {
tx := s.db.Txn(false)
defer tx.Abort()
return areIntentionsInConfigEntries(tx)
}
func areIntentionsInConfigEntries(tx *txn) (bool, error) {
_, entry, err := systemMetadataGetTxn(tx, nil, structs.SystemMetadataIntentionFormatKey)
if err != nil {
return false, fmt.Errorf("failed system metadatalookup: %s", err)
}
if entry == nil {
return false, nil
}
return entry.Value == structs.SystemMetadataIntentionFormatConfigValue, nil
}
// LegacyIntentions is like Intentions() but only returns legacy intentions.
// This is exposed for migration purposes.
func (s *Store) LegacyIntentions(ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, error) {
tx := s.db.Txn(false)
defer tx.Abort()
idx, results, _, err := s.legacyIntentionsListTxn(tx, ws, entMeta)
return idx, results, err
}
// Intentions returns the list of all intentions. The boolean response value is true if it came from config entries.
func (s *Store) Intentions(ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, bool, error) {
tx := s.db.Txn(false)
defer tx.Abort()
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return 0, nil, false, err
}
if !usingConfigEntries {
return s.legacyIntentionsListTxn(tx, ws, entMeta)
}
return s.configIntentionsListTxn(tx, ws, entMeta)
}
func (s *Store) legacyIntentionsListTxn(tx *txn, ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (uint64, structs.Intentions, bool, error) {
// Get the index
idx := maxIndexTxn(tx, intentionsTableName)
if idx < 1 {
@ -138,7 +185,7 @@ func (s *Store) Intentions(ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (
iter, err := intentionListTxn(tx, entMeta)
if err != nil {
return 0, nil, fmt.Errorf("failed intention lookup: %s", err)
return 0, nil, false, fmt.Errorf("failed intention lookup: %s", err)
}
ws.Add(iter.WatchCh())
@ -152,30 +199,43 @@ func (s *Store) Intentions(ws memdb.WatchSet, entMeta *structs.EnterpriseMeta) (
// want for presentation.
sort.Sort(structs.IntentionPrecedenceSorter(results))
return idx, results, nil
return idx, results, false, nil
}
// IntentionSet creates or updates an intention.
func (s *Store) IntentionSet(idx uint64, ixn *structs.Intention) error {
var ErrLegacyIntentionsAreDisabled = errors.New("Legacy intention modifications are disabled after the config entry migration.")
// LegacyIntentionSet creates or updates an intention.
//
// Deprecated: Edit service-intentions config entries directly.
func (s *Store) LegacyIntentionSet(idx uint64, ixn *structs.Intention) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
if err := intentionSetTxn(tx, idx, ixn); err != nil {
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return err
}
if usingConfigEntries {
return ErrLegacyIntentionsAreDisabled
}
if err := legacyIntentionSetTxn(tx, idx, ixn); err != nil {
return err
}
return tx.Commit()
}
// intentionSetTxn is the inner method used to insert an intention with
// legacyIntentionSetTxn is the inner method used to insert an intention with
// the proper indexes into the state store.
func intentionSetTxn(tx *txn, idx uint64, ixn *structs.Intention) error {
func legacyIntentionSetTxn(tx *txn, idx uint64, ixn *structs.Intention) error {
// ID is required
if ixn.ID == "" {
return ErrMissingIntentionID
}
// Ensure Precedence is populated correctly on "write"
//nolint:staticcheck
ixn.UpdatePrecedence()
// Check for an existing intention
@ -224,10 +284,22 @@ func intentionSetTxn(tx *txn, idx uint64, ixn *structs.Intention) error {
}
// IntentionGet returns the given intention by ID.
func (s *Store) IntentionGet(ws memdb.WatchSet, id string) (uint64, *structs.Intention, error) {
func (s *Store) IntentionGet(ws memdb.WatchSet, id string) (uint64, *structs.ServiceIntentionsConfigEntry, *structs.Intention, error) {
tx := s.db.Txn(false)
defer tx.Abort()
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return 0, nil, nil, err
}
if !usingConfigEntries {
idx, ixn, err := s.legacyIntentionGetTxn(tx, ws, id)
return idx, nil, ixn, err
}
return s.configIntentionGetTxn(tx, ws, id)
}
func (s *Store) legacyIntentionGetTxn(tx *txn, ws memdb.WatchSet, id string) (uint64, *structs.Intention, error) {
// Get the table index.
idx := maxIndexTxn(tx, intentionsTableName)
if idx < 1 {
@ -251,10 +323,22 @@ func (s *Store) IntentionGet(ws memdb.WatchSet, id string) (uint64, *structs.Int
}
// IntentionGetExact returns the given intention by it's full unique name.
func (s *Store) IntentionGetExact(ws memdb.WatchSet, args *structs.IntentionQueryExact) (uint64, *structs.Intention, error) {
func (s *Store) IntentionGetExact(ws memdb.WatchSet, args *structs.IntentionQueryExact) (uint64, *structs.ServiceIntentionsConfigEntry, *structs.Intention, error) {
tx := s.db.Txn(false)
defer tx.Abort()
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return 0, nil, nil, err
}
if !usingConfigEntries {
idx, ixn, err := s.legacyIntentionGetExactTxn(tx, ws, args)
return idx, nil, ixn, err
}
return s.configIntentionGetExactTxn(tx, ws, args)
}
func (s *Store) legacyIntentionGetExactTxn(tx *txn, ws memdb.WatchSet, args *structs.IntentionQueryExact) (uint64, *structs.Intention, error) {
if err := args.Validate(); err != nil {
return 0, nil, err
}
@ -282,21 +366,31 @@ func (s *Store) IntentionGetExact(ws memdb.WatchSet, args *structs.IntentionQuer
return idx, result, nil
}
// IntentionDelete deletes the given intention by ID.
func (s *Store) IntentionDelete(idx uint64, id string) error {
// LegacyIntentionDelete deletes the given intention by ID.
//
// Deprecated: Edit service-intentions config entries directly.
func (s *Store) LegacyIntentionDelete(idx uint64, id string) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
if err := intentionDeleteTxn(tx, idx, id); err != nil {
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return err
}
if usingConfigEntries {
return ErrLegacyIntentionsAreDisabled
}
if err := legacyIntentionDeleteTxn(tx, idx, id); err != nil {
return fmt.Errorf("failed intention delete: %s", err)
}
return tx.Commit()
}
// intentionDeleteTxn is the inner method used to delete a intention
// legacyIntentionDeleteTxn is the inner method used to delete a legacy intention
// with the proper indexes into the state store.
func intentionDeleteTxn(tx *txn, idx uint64, queryID string) error {
func legacyIntentionDeleteTxn(tx *txn, idx uint64, queryID string) error {
// Pull the query.
wrapped, err := tx.First(intentionsTableName, "id", queryID)
if err != nil {
@ -317,6 +411,42 @@ func intentionDeleteTxn(tx *txn, idx uint64, queryID string) error {
return nil
}
// LegacyIntentionDeleteAll deletes all legacy intentions. This is part of the
// config entry migration code.
func (s *Store) LegacyIntentionDeleteAll(idx uint64) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
// Delete the table and update the index.
if _, err := tx.DeleteAll(intentionsTableName, "id"); err != nil {
return fmt.Errorf("failed intention delete-all: %s", err)
}
if err := tx.Insert("index", &IndexEntry{intentionsTableName, idx}); err != nil {
return fmt.Errorf("failed updating index: %s", err)
}
// Also bump the index for the config entry table so that
// secondaries can correctly know when they've replicated all of the service-intentions
// config entries that USED to exist in the old intentions table.
if err := tx.Insert("index", &IndexEntry{configTableName, idx}); err != nil {
return fmt.Errorf("failed updating index: %s", err)
}
// Also set a system metadata flag indicating the transition has occurred.
metadataEntry := &structs.SystemMetadataEntry{
Key: structs.SystemMetadataIntentionFormatKey,
Value: structs.SystemMetadataIntentionFormatConfigValue,
RaftIndex: structs.RaftIndex{
CreateIndex: idx,
ModifyIndex: idx,
},
}
if err := systemMetadataSetTxn(tx, idx, metadataEntry); err != nil {
return fmt.Errorf("failed updating system metadata key %q: %s", metadataEntry.Key, err)
}
return tx.Commit()
}
// IntentionMatch returns the list of intentions that match the namespace and
// name for either a source or destination. This applies the resolution rules
// so wildcards will match any value.
@ -329,6 +459,17 @@ func (s *Store) IntentionMatch(ws memdb.WatchSet, args *structs.IntentionQueryMa
tx := s.db.Txn(false)
defer tx.Abort()
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return 0, nil, err
}
if !usingConfigEntries {
return s.legacyIntentionMatchTxn(tx, ws, args)
}
return s.configIntentionMatchTxn(tx, ws, args)
}
func (s *Store) legacyIntentionMatchTxn(tx *txn, ws memdb.WatchSet, args *structs.IntentionQueryMatch) (uint64, []structs.Intentions, error) {
// Get the table index.
idx := maxIndexTxn(tx, intentionsTableName)
if idx < 1 {
@ -359,11 +500,30 @@ func (s *Store) IntentionMatch(ws memdb.WatchSet, args *structs.IntentionQueryMa
//
// The returned intentions are sorted based on the intention precedence rules.
// i.e. result[0] is the highest precedent rule to match
func (s *Store) IntentionMatchOne(ws memdb.WatchSet,
entry structs.IntentionMatchEntry, matchType structs.IntentionMatchType) (uint64, structs.Intentions, error) {
func (s *Store) IntentionMatchOne(
ws memdb.WatchSet,
entry structs.IntentionMatchEntry,
matchType structs.IntentionMatchType,
) (uint64, structs.Intentions, error) {
tx := s.db.Txn(false)
defer tx.Abort()
usingConfigEntries, err := areIntentionsInConfigEntries(tx)
if err != nil {
return 0, nil, err
}
if !usingConfigEntries {
return s.legacyIntentionMatchOneTxn(tx, ws, entry, matchType)
}
return s.configIntentionMatchOneTxn(tx, ws, entry, matchType)
}
func (s *Store) legacyIntentionMatchOneTxn(
tx *txn,
ws memdb.WatchSet,
entry structs.IntentionMatchEntry,
matchType structs.IntentionMatchType,
) (uint64, structs.Intentions, error) {
// Get the table index.
idx := maxIndexTxn(tx, intentionsTableName)
if idx < 1 {

File diff suppressed because it is too large Load Diff

View File

@ -126,13 +126,19 @@ func txnSession(tx *txn, idx uint64, op *structs.TxnSessionOp) error {
return nil
}
// txnIntention handles all Intention-related operations.
func txnIntention(tx *txn, idx uint64, op *structs.TxnIntentionOp) error {
// txnLegacyIntention handles all Intention-related operations.
//
// Deprecated: see TxnOp.Intention description
func txnLegacyIntention(tx *txn, idx uint64, op *structs.TxnIntentionOp) error {
switch op.Op {
case structs.IntentionOpCreate, structs.IntentionOpUpdate:
return intentionSetTxn(tx, idx, op.Intention)
return legacyIntentionSetTxn(tx, idx, op.Intention)
case structs.IntentionOpDelete:
return intentionDeleteTxn(tx, idx, op.Intention.ID)
return legacyIntentionDeleteTxn(tx, idx, op.Intention.ID)
case structs.IntentionOpDeleteAll:
fallthrough // deliberately not available via this api
case structs.IntentionOpUpsert:
fallthrough // deliberately not available via this api
default:
return fmt.Errorf("unknown Intention op %q", op.Op)
}
@ -343,8 +349,6 @@ func (s *Store) txnDispatch(tx *txn, idx uint64, ops structs.TxnOps) (structs.Tx
switch {
case op.KV != nil:
ret, err = s.txnKVS(tx, idx, op.KV)
case op.Intention != nil:
err = txnIntention(tx, idx, op.Intention)
case op.Node != nil:
ret, err = s.txnNode(tx, idx, op.Node)
case op.Service != nil:
@ -353,6 +357,10 @@ func (s *Store) txnDispatch(tx *txn, idx uint64, ops structs.TxnOps) (structs.Tx
ret, err = s.txnCheck(tx, idx, op.Check)
case op.Session != nil:
err = txnSession(tx, idx, op.Session)
case op.Intention != nil:
// NOTE: this branch is deprecated and exists for backwards
// compatibility with pre-1.9.0 raft logs and during upgrades.
err = txnLegacyIntention(tx, idx, op.Intention)
default:
err = fmt.Errorf("no operation specified")
}

View File

@ -11,10 +11,11 @@ import (
"github.com/stretchr/testify/require"
)
func TestStateStore_Txn_Intention(t *testing.T) {
//nolint:staticcheck
func TestStateStore_Txn_LegacyIntention(t *testing.T) {
s := testStateStore(t)
// Create some intentions.
// Create some legacy intentions.
ixn1 := &structs.Intention{
ID: testUUID(),
SourceNS: "default",
@ -43,8 +44,8 @@ func TestStateStore_Txn_Intention(t *testing.T) {
// Write the first two to the state store, leave the third
// to be created by the transaction operation.
require.NoError(t, s.IntentionSet(1, ixn1))
require.NoError(t, s.IntentionSet(2, ixn2))
require.NoError(t, s.LegacyIntentionSet(1, ixn1))
require.NoError(t, s.LegacyIntentionSet(2, ixn2))
// Set up a transaction that hits every operation.
ops := structs.TxnOps{
@ -77,9 +78,10 @@ func TestStateStore_Txn_Intention(t *testing.T) {
require.Equal(t, expected, results)
// Pull the resulting state store contents.
idx, actual, err := s.Intentions(nil, nil)
idx, actual, fromConfig, err := s.Intentions(nil, nil)
require.NoError(t, err)
require.Equal(t, uint64(3), idx, "wrong index")
require.False(t, fromConfig)
// Make sure it looks as expected.
intentions := structs.Intentions{

View File

@ -0,0 +1,51 @@
package consul
import (
"github.com/hashicorp/consul/agent/structs"
)
func (s *Server) getSystemMetadata(key string) (string, error) {
_, entry, err := s.fsm.State().SystemMetadataGet(nil, key)
if err != nil {
return "", err
}
if entry == nil {
return "", nil
}
return entry.Value, nil
}
func (s *Server) setSystemMetadataKey(key, val string) error {
args := &structs.SystemMetadataRequest{
Op: structs.SystemMetadataUpsert,
Entry: &structs.SystemMetadataEntry{Key: key, Value: val},
}
resp, err := s.raftApply(structs.SystemMetadataRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
return nil
}
func (s *Server) deleteSystemMetadataKey(key string) error {
args := &structs.SystemMetadataRequest{
Op: structs.SystemMetadataDelete,
Entry: &structs.SystemMetadataEntry{Key: key},
}
resp, err := s.raftApply(structs.SystemMetadataRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
return nil
}

View File

@ -14,7 +14,11 @@ func TestLeader_SystemMetadata_CRUD(t *testing.T) {
// doesn't have an exposed RPC. We're just testing the full round trip of
// raft+fsm For now,
dir1, srv := testServerWithConfig(t, nil)
dir1, srv := testServerWithConfig(t, func(c *Config) {
// We disable connect here so we skip inserting intention-migration
// related system metadata in the background.
c.ConnectEnabled = false
})
defer os.RemoveAll(dir1)
defer srv.Shutdown()
codec := rpcClient(t, srv)
@ -30,9 +34,9 @@ func TestLeader_SystemMetadata_CRUD(t *testing.T) {
require.Len(t, entries, 0)
// Create 3
require.NoError(t, setSystemMetadataKey(srv, "key1", "val1"))
require.NoError(t, setSystemMetadataKey(srv, "key2", "val2"))
require.NoError(t, setSystemMetadataKey(srv, "key3", ""))
require.NoError(t, srv.setSystemMetadataKey("key1", "val1"))
require.NoError(t, srv.setSystemMetadataKey("key2", "val2"))
require.NoError(t, srv.setSystemMetadataKey("key3", ""))
mapify := func(entries []*structs.SystemMetadataEntry) map[string]string {
m := make(map[string]string)
@ -53,8 +57,8 @@ func TestLeader_SystemMetadata_CRUD(t *testing.T) {
}, mapify(entries))
// Update one and delete one.
require.NoError(t, setSystemMetadataKey(srv, "key3", "val3"))
require.NoError(t, deleteSystemMetadataKey(srv, "key1"))
require.NoError(t, srv.setSystemMetadataKey("key3", "val3"))
require.NoError(t, srv.deleteSystemMetadataKey("key1"))
_, entries, err = state.SystemMetadataList(nil)
require.NoError(t, err)
@ -65,44 +69,3 @@ func TestLeader_SystemMetadata_CRUD(t *testing.T) {
"key3": "val3",
}, mapify(entries))
}
// Note when this behavior is actually used, consider promoting these 2
// functions out of test code.
func setSystemMetadataKey(s *Server, key, val string) error {
args := &structs.SystemMetadataRequest{
Op: structs.SystemMetadataUpsert,
Entry: &structs.SystemMetadataEntry{
Key: key, Value: val,
},
}
resp, err := s.raftApply(structs.SystemMetadataRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
return nil
}
func deleteSystemMetadataKey(s *Server, key string) error {
args := &structs.SystemMetadataRequest{
Op: structs.SystemMetadataDelete,
Entry: &structs.SystemMetadataEntry{
Key: key,
},
}
resp, err := s.raftApply(structs.SystemMetadataRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
return nil
}

View File

@ -609,6 +609,7 @@ func decodeBodyDeprecated(req *http.Request, out interface{}, cb func(interface{
decodeConf := &mapstructure.DecoderConfig{
DecodeHook: mapstructure.ComposeDecodeHookFunc(
mapstructure.StringToTimeDurationHookFunc(),
mapstructure.StringToTimeHookFunc(time.RFC3339),
stringToReadableDurationFunc(),
),
Result: &out,

View File

@ -76,7 +76,7 @@ func init() {
registerEndpoint("/v1/connect/intentions", []string{"GET", "POST"}, (*HTTPHandlers).IntentionEndpoint)
registerEndpoint("/v1/connect/intentions/match", []string{"GET"}, (*HTTPHandlers).IntentionMatch)
registerEndpoint("/v1/connect/intentions/check", []string{"GET"}, (*HTTPHandlers).IntentionCheck)
registerEndpoint("/v1/connect/intentions/exact", []string{"GET"}, (*HTTPHandlers).IntentionGetExact)
registerEndpoint("/v1/connect/intentions/exact", []string{"GET", "PUT", "DELETE"}, (*HTTPHandlers).IntentionExact)
registerEndpoint("/v1/connect/intentions/", []string{"GET", "PUT", "DELETE"}, (*HTTPHandlers).IntentionSpecific)
registerEndpoint("/v1/coordinate/datacenters", []string{"GET"}, (*HTTPHandlers).CoordinateDatacenters)
registerEndpoint("/v1/coordinate/nodes", []string{"GET"}, (*HTTPHandlers).CoordinateNodes)

View File

@ -27,7 +27,7 @@ func (s *HTTPHandlers) IntentionEndpoint(resp http.ResponseWriter, req *http.Req
func (s *HTTPHandlers) IntentionList(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
// Method is tested in IntentionEndpoint
var args structs.DCSpecificRequest
var args structs.IntentionListRequest
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
@ -124,12 +124,15 @@ func (s *HTTPHandlers) IntentionMatch(resp http.ResponseWriter, req *http.Reques
// order of the returned responses.
args.Match.Entries = make([]structs.IntentionMatchEntry, len(names))
for i, n := range names {
entry, err := parseIntentionMatchEntry(n, &entMeta)
ns, name, err := parseIntentionStringComponent(n, &entMeta)
if err != nil {
return nil, fmt.Errorf("name %q is invalid: %s", n, err)
}
args.Match.Entries[i] = entry
args.Match.Entries[i] = structs.IntentionMatchEntry{
Namespace: ns,
Name: name,
}
}
var reply structs.IndexedIntentionMatches
@ -185,21 +188,21 @@ func (s *HTTPHandlers) IntentionCheck(resp http.ResponseWriter, req *http.Reques
// We parse them the same way as matches to extract namespace/name
args.Check.SourceName = source[0]
if args.Check.SourceType == structs.IntentionSourceConsul {
entry, err := parseIntentionMatchEntry(source[0], &entMeta)
ns, name, err := parseIntentionStringComponent(source[0], &entMeta)
if err != nil {
return nil, fmt.Errorf("source %q is invalid: %s", source[0], err)
}
args.Check.SourceNS = entry.Namespace
args.Check.SourceName = entry.Name
args.Check.SourceNS = ns
args.Check.SourceName = name
}
// The destination is always in the Consul format
entry, err := parseIntentionMatchEntry(destination[0], &entMeta)
ns, name, err := parseIntentionStringComponent(destination[0], &entMeta)
if err != nil {
return nil, fmt.Errorf("destination %q is invalid: %s", destination[0], err)
}
args.Check.DestinationNS = entry.Namespace
args.Check.DestinationName = entry.Name
args.Check.DestinationNS = ns
args.Check.DestinationName = name
var reply structs.IntentionQueryCheckResponse
if err := s.agent.RPC("Intention.Check", args, &reply); err != nil {
@ -236,21 +239,21 @@ func (s *HTTPHandlers) IntentionGetExact(resp http.ResponseWriter, req *http.Req
}
{
entry, err := parseIntentionMatchEntry(source[0], &entMeta)
ns, name, err := parseIntentionStringComponent(source[0], &entMeta)
if err != nil {
return nil, fmt.Errorf("source %q is invalid: %s", source[0], err)
}
args.Exact.SourceNS = entry.Namespace
args.Exact.SourceName = entry.Name
args.Exact.SourceNS = ns
args.Exact.SourceName = name
}
{
entry, err := parseIntentionMatchEntry(destination[0], &entMeta)
ns, name, err := parseIntentionStringComponent(destination[0], &entMeta)
if err != nil {
return nil, fmt.Errorf("destination %q is invalid: %s", destination[0], err)
}
args.Exact.DestinationNS = entry.Namespace
args.Exact.DestinationName = entry.Name
args.Exact.DestinationNS = ns
args.Exact.DestinationName = name
}
var reply structs.IndexedIntentions
@ -283,6 +286,20 @@ func (s *HTTPHandlers) IntentionGetExact(resp http.ResponseWriter, req *http.Req
return reply.Intentions[0], nil
}
// IntentionExact handles the endpoint for /v1/connect/intentions/exact
func (s *HTTPHandlers) IntentionExact(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
switch req.Method {
case "GET":
return s.IntentionGetExact(resp, req)
case "PUT":
return s.IntentionPutExact(resp, req)
case "DELETE":
return s.IntentionDeleteExact(resp, req)
default:
return nil, MethodNotAllowedError{req.Method, []string{"GET", "PUT", "DELETE"}}
}
}
// IntentionSpecific handles the endpoint for /v1/connect/intentions/:id
func (s *HTTPHandlers) IntentionSpecific(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
id := strings.TrimPrefix(req.URL.Path, "/v1/connect/intentions/")
@ -376,6 +393,46 @@ func (s *HTTPHandlers) IntentionSpecificUpdate(id string, resp http.ResponseWrit
}
// PUT /v1/connect/intentions/exact
func (s *HTTPHandlers) IntentionPutExact(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
var entMeta structs.EnterpriseMeta
if err := s.parseEntMetaNoWildcard(req, &entMeta); err != nil {
return nil, err
}
exact, err := parseIntentionQueryExact(req, &entMeta)
if err != nil {
return nil, err
}
args := structs.IntentionRequest{
Op: structs.IntentionOpUpsert,
}
s.parseDC(req, &args.Datacenter)
s.parseToken(req, &args.Token)
if err := decodeBody(req.Body, &args.Intention); err != nil {
return nil, BadRequestError{Reason: fmt.Sprintf("Request decode failed: %v", err)}
}
// Explicitly CLEAR the old legacy ID field
args.Intention.ID = ""
// Use the intention identity from the URL.
args.Intention.SourceNS = exact.SourceNS
args.Intention.SourceName = exact.SourceName
args.Intention.DestinationNS = exact.DestinationNS
args.Intention.DestinationName = exact.DestinationName
args.Intention.FillNonDefaultNamespaces(&entMeta)
var ignored string
if err := s.agent.RPC("Intention.Apply", &args, &ignored); err != nil {
return nil, err
}
return true, nil
}
// DELETE /v1/connect/intentions/:id
func (s *HTTPHandlers) IntentionSpecificDelete(id string, resp http.ResponseWriter, req *http.Request) (interface{}, error) {
// Method is tested in IntentionEndpoint
@ -395,28 +452,91 @@ func (s *HTTPHandlers) IntentionSpecificDelete(id string, resp http.ResponseWrit
return true, nil
}
// DELETE /v1/connect/intentions/exact
func (s *HTTPHandlers) IntentionDeleteExact(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
var entMeta structs.EnterpriseMeta
if err := s.parseEntMetaNoWildcard(req, &entMeta); err != nil {
return nil, err
}
exact, err := parseIntentionQueryExact(req, &entMeta)
if err != nil {
return nil, err
}
args := structs.IntentionRequest{
Op: structs.IntentionOpDelete,
Intention: &structs.Intention{
// NOTE: ID is explicitly empty here
SourceNS: exact.SourceNS,
SourceName: exact.SourceName,
DestinationNS: exact.DestinationNS,
DestinationName: exact.DestinationName,
},
}
s.parseDC(req, &args.Datacenter)
s.parseToken(req, &args.Token)
var ignored string
if err := s.agent.RPC("Intention.Apply", &args, &ignored); err != nil {
return nil, err
}
return true, nil
}
// intentionCreateResponse is the response structure for creating an intention.
type intentionCreateResponse struct{ ID string }
// parseIntentionMatchEntry parses the query parameter for an intention
// match query entry.
func parseIntentionMatchEntry(input string, entMeta *structs.EnterpriseMeta) (structs.IntentionMatchEntry, error) {
var result structs.IntentionMatchEntry
result.Namespace = entMeta.NamespaceOrEmpty()
func parseIntentionQueryExact(req *http.Request, entMeta *structs.EnterpriseMeta) (*structs.IntentionQueryExact, error) {
q := req.URL.Query()
// Extract the source/destination
source, ok := q["source"]
if !ok || len(source) != 1 || source[0] == "" {
return nil, fmt.Errorf("required query parameter 'source' not set")
}
destination, ok := q["destination"]
if !ok || len(destination) != 1 || destination[0] == "" {
return nil, fmt.Errorf("required query parameter 'destination' not set")
}
var exact structs.IntentionQueryExact
{
ns, name, err := parseIntentionStringComponent(source[0], entMeta)
if err != nil {
return nil, fmt.Errorf("source %q is invalid: %s", source[0], err)
}
exact.SourceNS = ns
exact.SourceName = name
}
{
ns, name, err := parseIntentionStringComponent(destination[0], entMeta)
if err != nil {
return nil, fmt.Errorf("destination %q is invalid: %s", destination[0], err)
}
exact.DestinationNS = ns
exact.DestinationName = name
}
return &exact, nil
}
func parseIntentionStringComponent(input string, entMeta *structs.EnterpriseMeta) (string, string, error) {
// Get the index to the '/'. If it doesn't exist, we have just a name
// so just set that and return.
idx := strings.IndexByte(input, '/')
if idx == -1 {
result.Name = input
return result, nil
ns := entMeta.NamespaceOrEmpty()
return ns, input, nil
}
result.Namespace = input[:idx]
result.Name = input[idx+1:]
if strings.IndexByte(result.Name, '/') != -1 {
return result, fmt.Errorf("input can contain at most one '/'")
ns, name := input[:idx], input[idx+1:]
if strings.IndexByte(name, '/') != -1 {
return "", "", fmt.Errorf("input can contain at most one '/'")
}
return result, nil
return ns, name, nil
}

View File

@ -7,72 +7,89 @@ import (
"testing"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/testrpc"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestIntentionsList_empty(t *testing.T) {
func TestIntentionList(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Make sure an empty list is non-nil.
req, _ := http.NewRequest("GET", "/v1/connect/intentions", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionList(resp, req)
assert.Nil(err)
t.Run("empty", func(t *testing.T) {
// Make sure an empty list is non-nil.
req, _ := http.NewRequest("GET", "/v1/connect/intentions", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionList(resp, req)
require.NoError(t, err)
value := obj.(structs.Intentions)
assert.NotNil(value)
assert.Len(value, 0)
}
value := obj.(structs.Intentions)
require.NotNil(t, value)
require.Len(t, value, 0)
})
func TestIntentionsList_values(t *testing.T) {
t.Parallel()
t.Run("values", func(t *testing.T) {
// Create some intentions, note we create the lowest precedence first to test
// sorting.
//
// Also create one non-legacy one using a different destination.
var ids []string
for _, v := range []string{"*", "foo", "bar", "zim"} {
req := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpCreate,
Intention: structs.TestIntention(t),
}
req.Intention.SourceName = v
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
if v == "zim" {
req.Op = structs.IntentionOpUpsert // non-legacy
req.Intention.DestinationName = "gir"
}
// Create some intentions, note we create the lowest precedence first to test
// sorting.
for _, v := range []string{"*", "foo", "bar"} {
req := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpCreate,
Intention: structs.TestIntention(t),
var reply string
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
ids = append(ids, reply)
}
req.Intention.SourceName = v
var reply string
assert.Nil(a.RPC("Intention.Apply", &req, &reply))
}
// Request
req, err := http.NewRequest("GET", "/v1/connect/intentions", nil)
require.NoError(t, err)
// Request
req, _ := http.NewRequest("GET", "/v1/connect/intentions", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionList(resp, req)
assert.NoError(err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionList(resp, req)
require.NoError(t, err)
value := obj.(structs.Intentions)
assert.Len(value, 3)
value := obj.(structs.Intentions)
require.Len(t, value, 4)
expected := []string{"bar", "foo", "*"}
actual := []string{
value[0].SourceName,
value[1].SourceName,
value[2].SourceName,
}
assert.Equal(expected, actual)
require.Equal(t, []string{"bar->db", "foo->db", "zim->gir", "*->db"},
[]string{
value[0].SourceName + "->" + value[0].DestinationName,
value[1].SourceName + "->" + value[1].DestinationName,
value[2].SourceName + "->" + value[2].DestinationName,
value[3].SourceName + "->" + value[3].DestinationName,
})
require.Equal(t, []string{ids[2], ids[1], "", ids[0]},
[]string{
value[0].ID,
value[1].ID,
value[2].ID,
value[3].ID,
})
})
}
func TestIntentionsMatch_basic(t *testing.T) {
func TestIntentionMatch(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create some intentions
{
@ -93,95 +110,82 @@ func TestIntentionsMatch_basic(t *testing.T) {
ixn.Intention.DestinationNS = v[2]
ixn.Intention.DestinationName = v[3]
if ixn.Intention.DestinationName == "baz" {
// make the "baz" destination be non-legacy
ixn.Op = structs.IntentionOpUpsert
}
// Create
var reply string
require.Nil(t, a.RPC("Intention.Apply", &ixn, &reply))
require.NoError(t, a.RPC("Intention.Apply", &ixn, &reply))
}
}
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/match?by=destination&name=bar", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
require.Nil(t, err)
t.Run("no by", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/match?name=foo/bar", nil)
require.NoError(t, err)
value := obj.(map[string]structs.Intentions)
require.Len(t, value, 1)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
testutil.RequireErrorContains(t, err, "by")
require.Nil(t, obj)
})
var actual [][]string
expected := [][]string{
{"default", "*", "default", "bar"},
{"default", "*", "default", "*"},
}
for _, ixn := range value["bar"] {
actual = append(actual, []string{
ixn.SourceNS,
ixn.SourceName,
ixn.DestinationNS,
ixn.DestinationName,
})
}
t.Run("by invalid", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/match?by=datacenter", nil)
require.NoError(t, err)
require.Equal(t, expected, actual)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
testutil.RequireErrorContains(t, err, "'by' parameter")
require.Nil(t, obj)
})
t.Run("no name", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/match?by=source", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
testutil.RequireErrorContains(t, err, "'name' not set")
require.Nil(t, obj)
})
t.Run("success", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/match?by=destination&name=bar", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
require.NoError(t, err)
value := obj.(map[string]structs.Intentions)
require.Len(t, value, 1)
var actual [][]string
expected := [][]string{
{"default", "*", "default", "bar"},
{"default", "*", "default", "*"},
}
for _, ixn := range value["bar"] {
actual = append(actual, []string{
ixn.SourceNS,
ixn.SourceName,
ixn.DestinationNS,
ixn.DestinationName,
})
}
require.Equal(t, expected, actual)
})
}
func TestIntentionsMatch_noBy(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/match?name=foo/bar", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
assert.NotNil(err)
assert.Contains(err.Error(), "by")
assert.Nil(obj)
}
func TestIntentionsMatch_byInvalid(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/match?by=datacenter", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
assert.NotNil(err)
assert.Contains(err.Error(), "'by' parameter")
assert.Nil(obj)
}
func TestIntentionsMatch_noName(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/match?by=source", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionMatch(resp, req)
assert.NotNil(err)
assert.Contains(err.Error(), "'name' not set")
assert.Nil(obj)
}
func TestIntentionsCheck_basic(t *testing.T) {
func TestIntentionCheck(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create some intentions
{
@ -202,120 +206,225 @@ func TestIntentionsCheck_basic(t *testing.T) {
ixn.Intention.DestinationName = v[3]
ixn.Intention.Action = structs.IntentionActionDeny
if ixn.Intention.DestinationName == "baz" {
// make the "baz" destination be non-legacy
ixn.Op = structs.IntentionOpUpsert
}
// Create
var reply string
require.NoError(t, a.RPC("Intention.Apply", &ixn, &reply))
}
}
// Request matching intention
{
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/test?source=bar&destination=baz", nil)
t.Run("no source", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/test?destination=B", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
testutil.RequireErrorContains(t, err, "'source' not set")
require.Nil(t, obj)
})
t.Run("no destination", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/test?source=B", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
testutil.RequireErrorContains(t, err, "'destination' not set")
require.Nil(t, obj)
})
t.Run("success - matching intention", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/test?source=bar&destination=baz", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
require.NoError(t, err)
value := obj.(*structs.IntentionQueryCheckResponse)
require.False(t, value.Allowed)
}
})
t.Run("success - non-matching intention", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/connect/intentions/test?source=bar&destination=qux", nil)
require.NoError(t, err)
// Request non-matching intention
{
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/test?source=bar&destination=qux", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
require.NoError(t, err)
value := obj.(*structs.IntentionQueryCheckResponse)
require.True(t, value.Allowed)
}
})
}
func TestIntentionsCheck_noSource(t *testing.T) {
func TestIntentionPutExact(t *testing.T) {
t.Parallel()
require := require.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/test?destination=B", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
require.NotNil(err)
require.Contains(err.Error(), "'source' not set")
require.Nil(obj)
}
t.Run("no body", func(t *testing.T) {
// Create with no body
req, err := http.NewRequest("PUT", "/v1/connect/intentions", nil)
require.NoError(t, err)
func TestIntentionsCheck_noDestination(t *testing.T) {
t.Parallel()
resp := httptest.NewRecorder()
_, err = a.srv.IntentionExact(resp, req)
require.Error(t, err)
})
require := require.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
t.Run("source is required", func(t *testing.T) {
ixn := structs.TestIntention(t)
ixn.SourceName = "foo"
req, err := http.NewRequest("PUT", "/v1/connect/intentions?source=&destination=db", jsonReader(ixn))
require.NoError(t, err)
// Request
req, _ := http.NewRequest("GET",
"/v1/connect/intentions/test?source=B", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCheck(resp, req)
require.NotNil(err)
require.Contains(err.Error(), "'destination' not set")
require.Nil(obj)
}
resp := httptest.NewRecorder()
_, err = a.srv.IntentionExact(resp, req)
require.Error(t, err)
})
func TestIntentionsCreate_good(t *testing.T) {
t.Parallel()
t.Run("destination is required", func(t *testing.T) {
ixn := structs.TestIntention(t)
ixn.SourceName = "foo"
req, err := http.NewRequest("PUT", "/v1/connect/intentions?source=foo&destination=", jsonReader(ixn))
require.NoError(t, err)
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
resp := httptest.NewRecorder()
_, err = a.srv.IntentionExact(resp, req)
require.Error(t, err)
})
// Make sure an empty list is non-nil.
args := structs.TestIntention(t)
args.SourceName = "foo"
req, _ := http.NewRequest("POST", "/v1/connect/intentions", jsonReader(args))
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCreate(resp, req)
assert.Nil(err)
t.Run("success", func(t *testing.T) {
ixn := structs.TestIntention(t)
ixn.SourceName = "foo"
req, err := http.NewRequest("PUT", "/v1/connect/intentions?source=foo&destination=db", jsonReader(ixn))
require.NoError(t, err)
value := obj.(intentionCreateResponse)
assert.NotEqual("", value.ID)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionExact(resp, req)
require.NoError(t, err)
require.True(t, obj.(bool))
// Read the value
{
req := &structs.IntentionQueryRequest{
Datacenter: "dc1",
IntentionID: value.ID,
// Read the value
{
req := &structs.IntentionQueryRequest{
Datacenter: "dc1",
Exact: ixn.ToExact(),
}
var resp structs.IndexedIntentions
require.NoError(t, a.RPC("Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
require.Equal(t, "foo", actual.SourceName)
require.Empty(t, actual.ID) // new style
}
var resp structs.IndexedIntentions
assert.Nil(a.RPC("Intention.Get", req, &resp))
assert.Len(resp.Intentions, 1)
actual := resp.Intentions[0]
assert.Equal("foo", actual.SourceName)
})
}
func TestIntentionCreate(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
t.Run("no body", func(t *testing.T) {
// Create with no body
req, _ := http.NewRequest("POST", "/v1/connect/intentions", nil)
resp := httptest.NewRecorder()
_, err := a.srv.IntentionCreate(resp, req)
require.Error(t, err)
})
t.Run("success", func(t *testing.T) {
// Make sure an empty list is non-nil.
args := structs.TestIntention(t)
args.SourceName = "foo"
req, _ := http.NewRequest("POST", "/v1/connect/intentions", jsonReader(args))
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionCreate(resp, req)
require.NoError(t, err)
value := obj.(intentionCreateResponse)
require.NotEmpty(t, value.ID)
// Read the value
{
req := &structs.IntentionQueryRequest{
Datacenter: "dc1",
IntentionID: value.ID,
}
var resp structs.IndexedIntentions
require.NoError(t, a.RPC("Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
require.Equal(t, "foo", actual.SourceName)
}
})
}
func TestIntentionSpecificGet(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ixn := structs.TestIntention(t)
// Create an intention directly
var reply string
{
req := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpCreate,
Intention: ixn,
}
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
}
t.Run("invalid id", func(t *testing.T) {
// Read intention with bad ID
req, _ := http.NewRequest("GET", "/v1/connect/intentions/hello", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
require.Nil(t, obj)
require.Error(t, err)
require.IsType(t, BadRequestError{}, err)
require.Contains(t, err.Error(), "UUID")
})
t.Run("success", func(t *testing.T) {
// Get the value
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/connect/intentions/%s", reply), nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
require.NoError(t, err)
value := obj.(*structs.Intention)
require.Equal(t, reply, value.ID)
ixn.ID = value.ID
ixn.Precedence = value.Precedence
ixn.RaftIndex = value.RaftIndex
ixn.Hash = value.Hash
ixn.CreatedAt, ixn.UpdatedAt = value.CreatedAt, value.UpdatedAt
require.Equal(t, ixn, value)
})
}
func TestIntentionsCreate_noBody(t *testing.T) {
func TestIntentionSpecificUpdate(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
// Create with no body
req, _ := http.NewRequest("POST", "/v1/connect/intentions", nil)
resp := httptest.NewRecorder()
_, err := a.srv.IntentionCreate(resp, req)
require.Error(t, err)
}
func TestIntentionsSpecificGet_good(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// The intention
ixn := structs.TestIntention(t)
@ -328,60 +437,7 @@ func TestIntentionsSpecificGet_good(t *testing.T) {
Op: structs.IntentionOpCreate,
Intention: ixn,
}
assert.Nil(a.RPC("Intention.Apply", &req, &reply))
}
// Get the value
req, _ := http.NewRequest("GET", fmt.Sprintf("/v1/connect/intentions/%s", reply), nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
assert.Nil(err)
value := obj.(*structs.Intention)
assert.Equal(reply, value.ID)
ixn.ID = value.ID
ixn.RaftIndex = value.RaftIndex
ixn.CreatedAt, ixn.UpdatedAt = value.CreatedAt, value.UpdatedAt
assert.Equal(ixn, value)
}
func TestIntentionsSpecificGet_invalidId(t *testing.T) {
t.Parallel()
require := require.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
// Read intention with bad ID
req, _ := http.NewRequest("GET", "/v1/connect/intentions/hello", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
require.Nil(obj)
require.Error(err)
require.IsType(BadRequestError{}, err)
require.Contains(err.Error(), "UUID")
}
func TestIntentionsSpecificUpdate_good(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
// The intention
ixn := structs.TestIntention(t)
// Create an intention directly
var reply string
{
req := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpCreate,
Intention: ixn,
}
assert.Nil(a.RPC("Intention.Apply", &req, &reply))
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
}
// Update the intention
@ -390,10 +446,10 @@ func TestIntentionsSpecificUpdate_good(t *testing.T) {
req, _ := http.NewRequest("PUT", fmt.Sprintf("/v1/connect/intentions/%s", reply), jsonReader(ixn))
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
assert.Nil(err)
require.NoError(t, err)
value := obj.(intentionCreateResponse)
assert.Equal(reply, value.ID)
require.Equal(t, reply, value.ID)
// Read the value
{
@ -402,19 +458,99 @@ func TestIntentionsSpecificUpdate_good(t *testing.T) {
IntentionID: reply,
}
var resp structs.IndexedIntentions
assert.Nil(a.RPC("Intention.Get", req, &resp))
assert.Len(resp.Intentions, 1)
require.NoError(t, a.RPC("Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
assert.Equal("bar", actual.SourceName)
require.Equal(t, "bar", actual.SourceName)
}
}
func TestIntentionsSpecificDelete_good(t *testing.T) {
func TestIntentionDeleteExact(t *testing.T) {
t.Parallel()
assert := assert.New(t)
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ixn := structs.TestIntention(t)
ixn.SourceName = "foo"
exact := ixn.ToExact()
// Create an intention directly
{
req := structs.IntentionRequest{
Datacenter: "dc1",
Op: structs.IntentionOpUpsert,
Intention: ixn,
}
var ignored string
require.NoError(t, a.RPC("Intention.Apply", &req, &ignored))
}
// Sanity check that the intention exists
{
req := &structs.IntentionQueryRequest{
Datacenter: "dc1",
Exact: exact,
}
var resp structs.IndexedIntentions
require.NoError(t, a.RPC("Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
require.Equal(t, "foo", actual.SourceName)
require.Empty(t, actual.ID) // new style
}
t.Run("source is required", func(t *testing.T) {
// Delete the intention
req, err := http.NewRequest("DELETE", "/v1/connect/intentions/exact?source=&destination=db", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
_, err = a.srv.IntentionExact(resp, req)
require.Error(t, err)
})
t.Run("destination is required", func(t *testing.T) {
// Delete the intention
req, err := http.NewRequest("DELETE", "/v1/connect/intentions/exact?source=foo&destination=", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
_, err = a.srv.IntentionExact(resp, req)
require.Error(t, err)
})
t.Run("success", func(t *testing.T) {
// Delete the intention
req, err := http.NewRequest("DELETE", "/v1/connect/intentions/exact?source=foo&destination=db", nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionExact(resp, req)
require.NoError(t, err)
require.Equal(t, true, obj)
// Verify the intention is gone
{
req := &structs.IntentionQueryRequest{
Datacenter: "dc1",
Exact: exact,
}
var resp structs.IndexedIntentions
err := a.RPC("Intention.Get", req, &resp)
testutil.RequireErrorContains(t, err, "not found")
}
})
}
func TestIntentionSpecificDelete(t *testing.T) {
t.Parallel()
a := NewTestAgent(t, "")
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// The intention
ixn := structs.TestIntention(t)
@ -428,7 +564,7 @@ func TestIntentionsSpecificDelete_good(t *testing.T) {
Op: structs.IntentionOpCreate,
Intention: ixn,
}
assert.Nil(a.RPC("Intention.Apply", &req, &reply))
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
}
// Sanity check that the intention exists
@ -438,18 +574,18 @@ func TestIntentionsSpecificDelete_good(t *testing.T) {
IntentionID: reply,
}
var resp structs.IndexedIntentions
assert.Nil(a.RPC("Intention.Get", req, &resp))
assert.Len(resp.Intentions, 1)
require.NoError(t, a.RPC("Intention.Get", req, &resp))
require.Len(t, resp.Intentions, 1)
actual := resp.Intentions[0]
assert.Equal("foo", actual.SourceName)
require.Equal(t, "foo", actual.SourceName)
}
// Delete the intention
req, _ := http.NewRequest("DELETE", fmt.Sprintf("/v1/connect/intentions/%s", reply), nil)
resp := httptest.NewRecorder()
obj, err := a.srv.IntentionSpecific(resp, req)
assert.Nil(err)
assert.Equal(true, obj)
require.NoError(t, err)
require.Equal(t, true, obj)
// Verify the intention is gone
{
@ -459,50 +595,50 @@ func TestIntentionsSpecificDelete_good(t *testing.T) {
}
var resp structs.IndexedIntentions
err := a.RPC("Intention.Get", req, &resp)
assert.NotNil(err)
assert.Contains(err.Error(), "not found")
testutil.RequireErrorContains(t, err, "not found")
}
}
func TestParseIntentionMatchEntry(t *testing.T) {
func TestParseIntentionStringComponent(t *testing.T) {
cases := []struct {
Input string
Expected structs.IntentionMatchEntry
Err bool
Input string
ExpectedNS, ExpectedName string
Err bool
}{
{
"foo",
structs.IntentionMatchEntry{
Name: "foo",
},
"", "foo",
false,
},
{
"foo/bar",
structs.IntentionMatchEntry{
Namespace: "foo",
Name: "bar",
},
"foo", "bar",
false,
},
{
"/bar",
"", "bar",
false,
},
{
"foo/bar/baz",
structs.IntentionMatchEntry{},
"", "",
true,
},
}
for _, tc := range cases {
t.Run(tc.Input, func(t *testing.T) {
assert := assert.New(t)
var entMeta structs.EnterpriseMeta
actual, err := parseIntentionMatchEntry(tc.Input, &entMeta)
assert.Equal(err != nil, tc.Err, err)
if err != nil {
return
}
ns, name, err := parseIntentionStringComponent(tc.Input, &entMeta)
if tc.Err {
require.Error(t, err)
} else {
require.NoError(t, err)
assert.Equal(tc.Expected, actual)
assert.Equal(t, tc.ExpectedNS, ns)
assert.Equal(t, tc.ExpectedName, name)
}
})
}
}

View File

@ -4,6 +4,7 @@ import (
"fmt"
"strconv"
"strings"
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/cache"
@ -23,12 +24,24 @@ const (
ServiceResolver string = "service-resolver"
IngressGateway string = "ingress-gateway"
TerminatingGateway string = "terminating-gateway"
ServiceIntentions string = "service-intentions"
ProxyConfigGlobal string = "global"
DefaultServiceProtocol = "tcp"
)
var AllConfigEntryKinds = []string{
ServiceDefaults,
ProxyDefaults,
ServiceRouter,
ServiceSplitter,
ServiceResolver,
IngressGateway,
TerminatingGateway,
ServiceIntentions,
}
// ConfigEntry is the interface for centralized configuration stored in Raft.
// Currently only service-defaults and proxy-defaults are supported.
type ConfigEntry interface {
@ -49,6 +62,19 @@ type ConfigEntry interface {
GetRaftIndex() *RaftIndex
}
// UpdatableConfigEntry is the optional interface implemented by a ConfigEntry
// if it wants more control over how the update part of upsert works
// differently than a straight create. By default without this implementation
// all upsert operations are replacements.
type UpdatableConfigEntry interface {
// UpdateOver is called from the state machine when an identically named
// config entry already exists. This lets the config entry optionally
// choose to use existing information from a config entry (such as
// CreateTime) to slightly adjust how the update actually happens.
UpdateOver(prev ConfigEntry) error
ConfigEntry
}
// ServiceConfiguration is the top-level struct for the configuration of a service
// across the entire cluster.
type ServiceConfigEntry struct {
@ -311,6 +337,7 @@ func DecodeConfigEntry(raw map[string]interface{}) (ConfigEntry, error) {
decode.HookWeakDecodeFromSlice,
decode.HookTranslateKeys,
mapstructure.StringToTimeDurationHookFunc(),
mapstructure.StringToTimeHookFunc(time.RFC3339),
),
Metadata: &md,
Result: &entry,
@ -421,6 +448,8 @@ func MakeConfigEntry(kind, name string) (ConfigEntry, error) {
return &IngressGatewayConfigEntry{Name: name}, nil
case TerminatingGateway:
return &TerminatingGatewayConfigEntry{Name: name}, nil
case ServiceIntentions:
return &ServiceIntentionsConfigEntry{Name: name}, nil
default:
return nil, fmt.Errorf("invalid config entry kind: %s", kind)
}
@ -434,6 +463,8 @@ func ValidateConfigEntryKind(kind string) bool {
return true
case IngressGateway, TerminatingGateway:
return true
case ServiceIntentions:
return true
default:
return false
}
@ -479,6 +510,26 @@ func (r *ConfigEntryQuery) CacheInfo() cache.RequestInfo {
return info
}
// ConfigEntryListAllRequest is used when requesting to list all config entries
// of a set of kinds.
type ConfigEntryListAllRequest struct {
// Kinds should always be set. For backwards compatibility with versions
// prior to 1.9.0, if this is omitted or left empty it is assumed to mean
// the subset of config entry kinds that were present in 1.8.0:
//
// proxy-defaults, service-defaults, service-resolver, service-splitter,
// service-router, terminating-gateway, and ingress-gateway.
Kinds []string
Datacenter string
EnterpriseMeta `hcl:",squash" mapstructure:",squash"`
QueryOptions
}
func (r *ConfigEntryListAllRequest) RequestDatacenter() string {
return r.Datacenter
}
// ServiceConfigRequest is used when requesting the resolved configuration
// for a service.
type ServiceConfigRequest struct {

View File

@ -0,0 +1,539 @@
package structs
import (
"fmt"
"sort"
"strings"
"time"
"github.com/hashicorp/consul/acl"
)
type ServiceIntentionsConfigEntry struct {
Kind string
Name string // formerly DestinationName
Sources []*SourceIntention
Meta map[string]string `json:",omitempty"` // formerly Intention.Meta
EnterpriseMeta `hcl:",squash" mapstructure:",squash"` // formerly DestinationNS
RaftIndex
}
var _ UpdatableConfigEntry = (*ServiceIntentionsConfigEntry)(nil)
func (e *ServiceIntentionsConfigEntry) GetKind() string {
return ServiceIntentions
}
func (e *ServiceIntentionsConfigEntry) GetName() string {
if e == nil {
return ""
}
return e.Name
}
func (e *ServiceIntentionsConfigEntry) GetMeta() map[string]string {
if e == nil {
return nil
}
return e.Meta
}
func (e *ServiceIntentionsConfigEntry) Clone() *ServiceIntentionsConfigEntry {
e2 := *e
e2.Meta = cloneStringStringMap(e.Meta)
e2.Sources = make([]*SourceIntention, len(e.Sources))
for i, src := range e.Sources {
e2.Sources[i] = src.Clone()
}
return &e2
}
func (e *ServiceIntentionsConfigEntry) DestinationServiceName() ServiceName {
return NewServiceName(e.Name, &e.EnterpriseMeta)
}
func (e *ServiceIntentionsConfigEntry) ToIntention(src *SourceIntention) *Intention {
meta := e.Meta
if src.LegacyID != "" {
meta = src.LegacyMeta
}
ixn := &Intention{
ID: src.LegacyID,
Description: src.Description,
SourceNS: src.NamespaceOrDefault(),
SourceName: src.Name,
SourceType: src.Type,
Action: src.Action,
Meta: meta,
Precedence: src.Precedence,
DestinationNS: e.NamespaceOrDefault(),
DestinationName: e.Name,
RaftIndex: e.RaftIndex,
}
if src.LegacyCreateTime != nil {
ixn.CreatedAt = *src.LegacyCreateTime
}
if src.LegacyUpdateTime != nil {
ixn.UpdatedAt = *src.LegacyUpdateTime
}
if src.LegacyID != "" {
// Ensure that pre-1.9.0 secondaries can still replicate legacy
// intentions via the APIs. These require the Hash field to be
// populated.
//
//nolint:staticcheck
ixn.SetHash()
}
return ixn
}
func (e *ServiceIntentionsConfigEntry) LegacyIDFieldsAreAllEmpty() bool {
for _, src := range e.Sources {
if src.LegacyID != "" {
return false
}
}
return true
}
func (e *ServiceIntentionsConfigEntry) LegacyIDFieldsAreAllSet() bool {
for _, src := range e.Sources {
if src.LegacyID == "" {
return false
}
}
return true
}
func (e *ServiceIntentionsConfigEntry) ToIntentions() Intentions {
out := make(Intentions, 0, len(e.Sources))
for _, src := range e.Sources {
out = append(out, e.ToIntention(src))
}
return out
}
type SourceIntention struct {
// Name is the name of the source service. This can be a wildcard "*", but
// only the full value can be a wildcard. Partial wildcards are not
// allowed.
//
// The source may also be a non-Consul service, as specified by SourceType.
//
// formerly Intention.SourceName
Name string
// Action is whether this is an allowlist or denylist intention.
//
// formerly Intention.Action
Action IntentionAction
// Precedence is the order that the intention will be applied, with
// larger numbers being applied first. This is a read-only field, on
// any intention update it is updated.
//
// Note we will technically decode this over the wire during a write, but
// we always recompute it on save.
//
// formerly Intention.Precedence
Precedence int
// LegacyID is manipulated just by the bridging code
// used as part of backwards compatibility.
//
// formerly Intention.ID
LegacyID string `json:",omitempty" alias:"legacy_id"`
// Type is the type of the value for the source.
//
// formerly Intention.SourceType
Type IntentionSourceType
// Description is a human-friendly description of this intention.
// It is opaque to Consul and is only stored and transferred in API
// requests.
//
// formerly Intention.Description
Description string `json:",omitempty"`
// LegacyMeta is arbitrary metadata associated with the intention. This is
// opaque to Consul but is served in API responses.
//
// formerly Intention.Meta
LegacyMeta map[string]string `json:",omitempty" alias:"legacy_meta"`
// LegacyCreateTime is formerly Intention.CreatedAt
LegacyCreateTime *time.Time `json:",omitempty" alias:"legacy_create_time"`
// LegacyUpdateTime is formerly Intention.UpdatedAt
LegacyUpdateTime *time.Time `json:",omitempty" alias:"legacy_update_time"`
// Things like L7 rules or Sentinel rules could go here later.
// formerly Intention.SourceNS
EnterpriseMeta `hcl:",squash" mapstructure:",squash"`
}
func cloneStringStringMap(m map[string]string) map[string]string {
if m == nil {
return nil
}
m2 := make(map[string]string)
for k, v := range m {
m2[k] = v
}
return m2
}
func (x *SourceIntention) SourceServiceName() ServiceName {
return NewServiceName(x.Name, &x.EnterpriseMeta)
}
func (x *SourceIntention) Clone() *SourceIntention {
x2 := *x
x2.LegacyMeta = cloneStringStringMap(x.LegacyMeta)
return &x2
}
func (e *ServiceIntentionsConfigEntry) UpdateOver(rawPrev ConfigEntry) error {
if rawPrev == nil {
return nil
}
prev, ok := rawPrev.(*ServiceIntentionsConfigEntry)
if !ok {
return fmt.Errorf("previous config entry is not of type %T: %T", e, rawPrev)
}
var (
prevSourceByName = make(map[ServiceName]*SourceIntention)
prevSourceByLegacyID = make(map[string]*SourceIntention)
)
for _, src := range prev.Sources {
prevSourceByName[src.SourceServiceName()] = src
if src.LegacyID != "" {
prevSourceByLegacyID[src.LegacyID] = src
}
}
for i, src := range e.Sources {
if src.LegacyID == "" {
continue
}
// Check that the LegacyID fields are handled correctly during updates.
if prevSrc, ok := prevSourceByName[src.SourceServiceName()]; ok {
if prevSrc.LegacyID == "" {
return fmt.Errorf("Sources[%d].LegacyID: cannot set this field", i)
} else if src.LegacyID != prevSrc.LegacyID {
return fmt.Errorf("Sources[%d].LegacyID: cannot set this field to a different value", i)
}
}
// Now ensure legacy timestamps carry over properly. We always retain the LegacyCreateTime.
if prevSrc, ok := prevSourceByLegacyID[src.LegacyID]; ok {
if prevSrc.LegacyCreateTime != nil {
// NOTE: we don't want to share the memory here
src.LegacyCreateTime = timePointer(*prevSrc.LegacyCreateTime)
}
}
}
return nil
}
func (e *ServiceIntentionsConfigEntry) Normalize() error {
return e.normalize(false)
}
func (e *ServiceIntentionsConfigEntry) LegacyNormalize() error {
return e.normalize(true)
}
func (e *ServiceIntentionsConfigEntry) normalize(legacyWrite bool) error {
if e == nil {
return fmt.Errorf("config entry is nil")
}
e.Kind = ServiceIntentions
e.EnterpriseMeta.Normalize()
for _, src := range e.Sources {
// Default source type
if src.Type == "" {
src.Type = IntentionSourceConsul
}
// If the source namespace is omitted it inherits that of the
// destination.
src.EnterpriseMeta.MergeNoWildcard(&e.EnterpriseMeta)
src.EnterpriseMeta.Normalize()
// Compute the precedence only AFTER normalizing namespaces since the
// namespaces are factored into the calculation.
src.Precedence = computeIntentionPrecedence(e, src)
if legacyWrite {
// We always force meta to be non-nil so that it's an empty map. This
// makes it easy for API responses to not nil-check this everywhere.
if src.LegacyMeta == nil {
src.LegacyMeta = make(map[string]string)
}
// Set the created/updated times. If this is an update instead of an insert
// the UpdateOver() will fix it up appropriately.
now := time.Now().UTC()
src.LegacyCreateTime = timePointer(now)
src.LegacyUpdateTime = timePointer(now)
} else {
// Legacy fields are cleared, except LegacyMeta which we leave
// populated so that we can later fail the write in Validate() and
// give the user a warning about possible data loss.
src.LegacyID = ""
src.LegacyCreateTime = nil
src.LegacyUpdateTime = nil
}
}
// The source intentions closer to the head of the list have higher
// precedence. i.e. index 0 has the highest precedence.
sort.SliceStable(e.Sources, func(i, j int) bool {
return e.Sources[i].Precedence > e.Sources[j].Precedence
})
return nil
}
func timePointer(t time.Time) *time.Time {
if t.IsZero() {
return nil
}
return &t
}
// NOTE: this assumes that the namespaces have been fully normalized.
func computeIntentionPrecedence(entry *ServiceIntentionsConfigEntry, src *SourceIntention) int {
// Max maintains the maximum value that the precedence can be depending
// on the number of exact values in the destination.
var max int
switch intentionCountExact(entry.Name, &entry.EnterpriseMeta) {
case 2:
max = 9
case 1:
max = 6
case 0:
max = 3
default:
// This shouldn't be possible, just set it to zero
return 0
}
// Given the maximum, the exact value is determined based on the
// number of source exact values.
countSrc := intentionCountExact(src.Name, &src.EnterpriseMeta)
return max - (2 - countSrc)
}
// intentionCountExact counts the number of exact values (not wildcards) in
// the given namespace and name.
func intentionCountExact(name string, entMeta *EnterpriseMeta) int {
ns := entMeta.NamespaceOrDefault()
// If NS is wildcard, pair must be */* since an exact service cannot follow a wildcard NS
// */* is allowed, but */foo is not
if ns == WildcardSpecifier {
return 0
}
// only the namespace must be exact, since the */* case already returned.
if name == WildcardSpecifier {
return 1
}
return 2
}
func (e *ServiceIntentionsConfigEntry) Validate() error {
return e.validate(false)
}
func (e *ServiceIntentionsConfigEntry) LegacyValidate() error {
return e.validate(true)
}
func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
if e.Name == "" {
return fmt.Errorf("Name is required")
}
if err := validateIntentionWildcards(e.Name, &e.EnterpriseMeta); err != nil {
return err
}
if legacyWrite {
if len(e.Meta) > 0 {
return fmt.Errorf("Meta must be omitted for legacy intention writes")
}
} else {
if err := validateConfigEntryMeta(e.Meta); err != nil {
return err
}
}
if len(e.Sources) == 0 {
return fmt.Errorf("At least one source is required")
}
seenSources := make(map[ServiceName]struct{})
for i, src := range e.Sources {
if src.Name == "" {
return fmt.Errorf("Sources[%d].Name is required", i)
}
if err := validateIntentionWildcards(src.Name, &src.EnterpriseMeta); err != nil {
return fmt.Errorf("Sources[%d].%v", i, err)
}
// Length of opaque values
if len(src.Description) > metaValueMaxLength {
return fmt.Errorf(
"Sources[%d].Description exceeds maximum length %d", i, metaValueMaxLength)
}
if legacyWrite {
if len(src.LegacyMeta) > metaMaxKeyPairs {
return fmt.Errorf(
"Sources[%d].Meta exceeds maximum element count %d", i, metaMaxKeyPairs)
}
for k, v := range src.LegacyMeta {
if len(k) > metaKeyMaxLength {
return fmt.Errorf(
"Sources[%d].Meta key %q exceeds maximum length %d",
i, k, metaKeyMaxLength,
)
}
if len(v) > metaValueMaxLength {
return fmt.Errorf(
"Sources[%d].Meta value for key %q exceeds maximum length %d",
i, k, metaValueMaxLength,
)
}
}
} else {
if len(src.LegacyMeta) > 0 {
return fmt.Errorf("Sources[%d].LegacyMeta must be omitted", i)
}
src.LegacyMeta = nil // ensure it's completely unset
}
if legacyWrite {
if src.LegacyID == "" {
return fmt.Errorf("Sources[%d].LegacyID must be set", i)
}
} else {
if src.LegacyID != "" {
return fmt.Errorf("Sources[%d].LegacyID must be omitted", i)
}
}
switch src.Action {
case IntentionActionAllow, IntentionActionDeny:
default:
return fmt.Errorf("Sources[%d].Action must be set to 'allow' or 'deny'", i)
}
switch src.Type {
case IntentionSourceConsul:
default:
return fmt.Errorf("Sources[%d].Type must be set to 'consul'", i)
}
serviceName := src.SourceServiceName()
if _, exists := seenSources[serviceName]; exists {
return fmt.Errorf("Sources[%d] defines %q more than once", i, serviceName.String())
}
seenSources[serviceName] = struct{}{}
}
return nil
}
// Wildcard usage verification
func validateIntentionWildcards(name string, entMeta *EnterpriseMeta) error {
ns := entMeta.NamespaceOrDefault()
if ns != WildcardSpecifier {
if strings.Contains(ns, WildcardSpecifier) {
return fmt.Errorf("Namespace: wildcard character '*' cannot be used with partial values")
}
}
if name != WildcardSpecifier {
if strings.Contains(name, WildcardSpecifier) {
return fmt.Errorf("Name: wildcard character '*' cannot be used with partial values")
}
if ns == WildcardSpecifier {
return fmt.Errorf("Name: exact value cannot follow wildcard namespace")
}
}
return nil
}
func (e *ServiceIntentionsConfigEntry) GetRaftIndex() *RaftIndex {
if e == nil {
return &RaftIndex{}
}
return &e.RaftIndex
}
func (e *ServiceIntentionsConfigEntry) GetEnterpriseMeta() *EnterpriseMeta {
if e == nil {
return nil
}
return &e.EnterpriseMeta
}
func (e *ServiceIntentionsConfigEntry) CanRead(authz acl.Authorizer) bool {
var authzContext acl.AuthorizerContext
e.FillAuthzContext(&authzContext)
return authz.IntentionRead(e.GetName(), &authzContext) == acl.Allow
}
func (e *ServiceIntentionsConfigEntry) CanWrite(authz acl.Authorizer) bool {
var authzContext acl.AuthorizerContext
e.FillAuthzContext(&authzContext)
return authz.IntentionWrite(e.GetName(), &authzContext) == acl.Allow
}
func MigrateIntentions(ixns Intentions) []*ServiceIntentionsConfigEntry {
if len(ixns) == 0 {
return nil
}
collated := make(map[ServiceName]*ServiceIntentionsConfigEntry)
for _, ixn := range ixns {
thisEntry := ixn.ToConfigEntry()
sn := thisEntry.DestinationServiceName()
if entry, ok := collated[sn]; ok {
entry.Sources = append(entry.Sources, thisEntry.Sources...)
} else {
collated[sn] = thisEntry
}
}
out := make([]*ServiceIntentionsConfigEntry, 0, len(collated))
for _, entry := range collated {
out = append(out, entry)
}
return out
}

View File

@ -0,0 +1,719 @@
package structs
import (
"fmt"
"strings"
"testing"
"time"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/go-uuid"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func generateUUID() (ret string) {
var err error
if ret, err = uuid.GenerateUUID(); err != nil {
panic(fmt.Sprintf("Unable to generate a UUID, %v", err))
}
return ret
}
func TestServiceIntentionsConfigEntry(t *testing.T) {
type testcase struct {
entry *ServiceIntentionsConfigEntry
legacy bool
normalizeErr string
validateErr string
// check is called between normalize and validate
check func(t *testing.T, entry *ServiceIntentionsConfigEntry)
}
legacyIDs := []string{
generateUUID(),
generateUUID(),
generateUUID(),
}
defaultMeta := DefaultEnterpriseMeta()
fooName := NewServiceName("foo", defaultMeta)
cases := map[string]testcase{
"nil": {
entry: nil,
normalizeErr: "config entry is nil",
},
"no name": {
entry: &ServiceIntentionsConfigEntry{},
validateErr: "Name is required",
},
"dest name has partial wildcard": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test*",
},
validateErr: "Name: wildcard character '*' cannot be used with partial values",
},
"empty": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
},
validateErr: "At least one source is required",
},
"source specified twice": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
},
{
Name: "foo",
Action: IntentionActionDeny,
},
},
},
validateErr: `Sources[1] defines "` + fooName.String() + `" more than once`,
},
"no source name": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Action: IntentionActionAllow,
},
},
},
validateErr: `Sources[0].Name is required`,
},
"source name has partial wildcard": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo*",
Action: IntentionActionAllow,
},
},
},
validateErr: `Sources[0].Name: wildcard character '*' cannot be used with partial values`,
},
"description too long": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
Description: strings.Repeat("x", 513),
},
},
},
validateErr: `Sources[0].Description exceeds maximum length 512`,
},
"config entry meta not allowed on legacy writes": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
},
},
Meta: map[string]string{
"key1": "val1",
},
},
validateErr: `Meta must be omitted for legacy intention writes`,
},
"config entry meta too many keys": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
},
},
Meta: makeStringMap(65, 5, 5),
},
validateErr: `Meta exceeds maximum element count 64`,
},
"config entry meta key too large": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
},
},
Meta: makeStringMap(64, 129, 5),
},
validateErr: `exceeds maximum length 128`,
},
"config entry meta value too large": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
},
},
Meta: makeStringMap(64, 128, 513),
},
validateErr: `exceeds maximum length 512`,
},
"config entry meta value just big enough": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
},
},
Meta: makeStringMap(64, 128, 512),
},
},
"legacy meta not allowed": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
Description: strings.Repeat("x", 512),
LegacyMeta: map[string]string{ // stray Meta will be dropped
"old": "data",
},
},
},
},
validateErr: "Sources[0].LegacyMeta must be omitted",
},
"legacy meta too many keys": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
LegacyMeta: makeStringMap(65, 5, 5),
},
},
},
validateErr: `Sources[0].Meta exceeds maximum element count 64`,
},
"legacy meta key too large": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
LegacyMeta: makeStringMap(64, 129, 5),
},
},
},
validateErr: `exceeds maximum length 128`,
},
"legacy meta value too large": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
LegacyMeta: makeStringMap(64, 128, 513),
},
},
},
validateErr: `exceeds maximum length 512`,
},
"legacy meta value just big enough": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Name: "foo",
Action: IntentionActionAllow,
LegacyMeta: makeStringMap(64, 128, 512),
},
},
},
},
"legacy ID is required in legacy mode": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: IntentionActionAllow,
Description: strings.Repeat("x", 512),
},
},
},
validateErr: "Sources[0].LegacyID must be set",
},
"action required for L4": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Description: strings.Repeat("x", 512),
},
},
},
validateErr: `Sources[0].Action must be set to 'allow' or 'deny'`,
},
"action must be allow or deny for L4": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
Action: "blah",
Description: strings.Repeat("x", 512),
},
},
},
validateErr: `Sources[0].Action must be set to 'allow' or 'deny'`,
},
"L4 normalize": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0], // stray ID will be dropped
Name: WildcardSpecifier,
Action: IntentionActionDeny,
},
{
Name: "foo",
Action: IntentionActionAllow,
},
{
Name: "bar",
Action: IntentionActionDeny,
},
},
Meta: map[string]string{
"key1": "val1",
"key2": "val2",
},
},
check: func(t *testing.T, entry *ServiceIntentionsConfigEntry) {
// Note the stable precedence sort has been applied here.
assert.Equal(t, []*SourceIntention{
{
Name: "foo",
EnterpriseMeta: *defaultMeta,
Action: IntentionActionAllow,
Precedence: 9,
Type: IntentionSourceConsul,
},
{
Name: "bar",
EnterpriseMeta: *defaultMeta,
Action: IntentionActionDeny,
Precedence: 9,
Type: IntentionSourceConsul,
},
{
Name: WildcardSpecifier,
EnterpriseMeta: *defaultMeta,
Action: IntentionActionDeny,
Precedence: 8,
Type: IntentionSourceConsul,
},
}, entry.Sources)
assert.Equal(t, map[string]string{
"key1": "val1",
"key2": "val2",
}, entry.Meta)
},
},
"L4 legacy normalize": {
legacy: true,
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: WildcardSpecifier,
Action: IntentionActionDeny,
LegacyID: legacyIDs[0],
},
{
Name: "foo",
Action: IntentionActionAllow,
LegacyID: legacyIDs[1],
LegacyMeta: map[string]string{
"key1": "val1",
"key2": "val2",
},
},
{
Name: "bar",
Action: IntentionActionDeny,
LegacyID: legacyIDs[2],
},
},
},
check: func(t *testing.T, entry *ServiceIntentionsConfigEntry) {
require.Len(t, entry.Sources, 3)
assert.False(t, entry.Sources[0].LegacyCreateTime.IsZero())
assert.False(t, entry.Sources[0].LegacyUpdateTime.IsZero())
assert.False(t, entry.Sources[1].LegacyCreateTime.IsZero())
assert.False(t, entry.Sources[1].LegacyUpdateTime.IsZero())
assert.False(t, entry.Sources[2].LegacyCreateTime.IsZero())
assert.False(t, entry.Sources[2].LegacyUpdateTime.IsZero())
assert.Equal(t, []*SourceIntention{
{
LegacyID: legacyIDs[1],
Name: "foo",
EnterpriseMeta: *defaultMeta,
Action: IntentionActionAllow,
Precedence: 9,
Type: IntentionSourceConsul,
LegacyMeta: map[string]string{
"key1": "val1",
"key2": "val2",
},
LegacyCreateTime: entry.Sources[0].LegacyCreateTime,
LegacyUpdateTime: entry.Sources[0].LegacyUpdateTime,
},
{
LegacyID: legacyIDs[2],
Name: "bar",
EnterpriseMeta: *defaultMeta,
Action: IntentionActionDeny,
Precedence: 9,
Type: IntentionSourceConsul,
LegacyMeta: map[string]string{},
LegacyCreateTime: entry.Sources[1].LegacyCreateTime,
LegacyUpdateTime: entry.Sources[1].LegacyUpdateTime,
},
{
LegacyID: legacyIDs[0],
Name: WildcardSpecifier,
EnterpriseMeta: *defaultMeta,
Action: IntentionActionDeny,
Precedence: 8,
Type: IntentionSourceConsul,
LegacyMeta: map[string]string{},
LegacyCreateTime: entry.Sources[2].LegacyCreateTime,
LegacyUpdateTime: entry.Sources[2].LegacyUpdateTime,
},
}, entry.Sources)
},
},
"L4 validate": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0], // stray ID will be dropped
Name: WildcardSpecifier,
Action: IntentionActionDeny,
},
{
Name: "foo",
Action: IntentionActionAllow,
},
{
Name: "bar",
Action: IntentionActionDeny,
},
},
Meta: map[string]string{
"key1": "val1",
"key2": "val2",
},
},
},
}
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
var err error
if tc.legacy {
err = tc.entry.LegacyNormalize()
} else {
err = tc.entry.Normalize()
}
if tc.normalizeErr != "" {
// require.Error(t, err)
// require.Contains(t, err.Error(), tc.normalizeErr)
testutil.RequireErrorContains(t, err, tc.normalizeErr)
return
}
require.NoError(t, err)
if tc.check != nil {
tc.check(t, tc.entry)
}
if tc.legacy {
err = tc.entry.LegacyValidate()
} else {
err = tc.entry.Validate()
}
if tc.validateErr != "" {
// require.Error(t, err)
// require.Contains(t, err.Error(), tc.validateErr)
testutil.RequireErrorContains(t, err, tc.validateErr)
return
}
require.NoError(t, err)
})
}
}
func makeStringMap(keys, keySize, valSize int) map[string]string {
m := make(map[string]string)
for i := 0; i < keys; i++ {
base := fmt.Sprintf("%d:", i)
if len(base) > keySize || len(base) > valSize {
panic("makeStringMap called with incompatible inputs")
}
// this is not performant
if keySize > valSize {
base = strings.Repeat(base, keySize)
} else {
base = strings.Repeat(base, valSize)
}
m[base[0:keySize]] = base[0:valSize]
}
return m
}
func TestMigrateIntentions(t *testing.T) {
type testcase struct {
in Intentions
expect []*ServiceIntentionsConfigEntry
}
legacyIDs := []string{
generateUUID(),
generateUUID(),
generateUUID(),
}
anyTime := time.Now().UTC()
cases := map[string]testcase{
"nil": {},
"one": {
in: Intentions{
{
ID: legacyIDs[0],
Description: "desc",
SourceName: "foo",
DestinationName: "bar",
SourceType: IntentionSourceConsul,
Action: IntentionActionAllow,
Meta: map[string]string{
"key1": "val1",
},
Precedence: 9,
CreatedAt: anyTime,
UpdatedAt: anyTime,
},
},
expect: []*ServiceIntentionsConfigEntry{
{
Kind: ServiceIntentions,
Name: "bar",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Description: "desc",
Name: "foo",
Type: IntentionSourceConsul,
Action: IntentionActionAllow,
LegacyMeta: map[string]string{
"key1": "val1",
},
},
},
},
},
},
"two in same": {
in: Intentions{
{
ID: legacyIDs[0],
Description: "desc",
SourceName: "foo",
DestinationName: "bar",
SourceType: IntentionSourceConsul,
Action: IntentionActionAllow,
Meta: map[string]string{
"key1": "val1",
},
Precedence: 9,
CreatedAt: anyTime,
UpdatedAt: anyTime,
},
{
ID: legacyIDs[1],
Description: "desc2",
SourceName: "*",
DestinationName: "bar",
SourceType: IntentionSourceConsul,
Action: IntentionActionDeny,
Meta: map[string]string{
"key2": "val2",
},
Precedence: 9,
CreatedAt: anyTime,
UpdatedAt: anyTime,
},
},
expect: []*ServiceIntentionsConfigEntry{
{
Kind: ServiceIntentions,
Name: "bar",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Description: "desc",
Name: "foo",
Type: IntentionSourceConsul,
Action: IntentionActionAllow,
LegacyMeta: map[string]string{
"key1": "val1",
},
},
{
LegacyID: legacyIDs[1],
Description: "desc2",
Name: "*",
Type: IntentionSourceConsul,
Action: IntentionActionDeny,
LegacyMeta: map[string]string{
"key2": "val2",
},
},
},
},
},
},
"two in different": {
in: Intentions{
{
ID: legacyIDs[0],
Description: "desc",
SourceName: "foo",
DestinationName: "bar",
SourceType: IntentionSourceConsul,
Action: IntentionActionAllow,
Meta: map[string]string{
"key1": "val1",
},
Precedence: 9,
CreatedAt: anyTime,
UpdatedAt: anyTime,
},
{
ID: legacyIDs[1],
Description: "desc2",
SourceName: "*",
DestinationName: "bar2",
SourceType: IntentionSourceConsul,
Action: IntentionActionDeny,
Meta: map[string]string{
"key2": "val2",
},
Precedence: 9,
CreatedAt: anyTime,
UpdatedAt: anyTime,
},
},
expect: []*ServiceIntentionsConfigEntry{
{
Kind: ServiceIntentions,
Name: "bar",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[0],
Description: "desc",
Name: "foo",
Type: IntentionSourceConsul,
Action: IntentionActionAllow,
LegacyMeta: map[string]string{
"key1": "val1",
},
},
},
},
{
Kind: ServiceIntentions,
Name: "bar2",
Sources: []*SourceIntention{
{
LegacyID: legacyIDs[1],
Description: "desc2",
Name: "*",
Type: IntentionSourceConsul,
Action: IntentionActionDeny,
LegacyMeta: map[string]string{
"key2": "val2",
},
},
},
},
},
},
}
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
got := MigrateIntentions(tc.in)
require.ElementsMatch(t, tc.expect, got)
})
}
}

View File

@ -953,6 +953,121 @@ func TestDecodeConfigEntry(t *testing.T) {
},
},
},
{
name: "service-intentions: kitchen sink",
snake: `
kind = "service-intentions"
name = "web"
meta {
"foo" = "bar"
"gir" = "zim"
}
sources = [
{
name = "foo"
action = "deny"
type = "consul"
description = "foo desc"
},
{
name = "bar"
action = "allow"
description = "bar desc"
}
]
sources {
name = "*"
action = "deny"
description = "wild desc"
}
`,
camel: `
Kind = "service-intentions"
Name = "web"
Meta {
"foo" = "bar"
"gir" = "zim"
}
Sources = [
{
Name = "foo"
Action = "deny"
Type = "consul"
Description = "foo desc"
},
{
Name = "bar"
Action = "allow"
Description = "bar desc"
}
]
Sources {
Name = "*"
Action = "deny"
Description = "wild desc"
}
`,
expect: &ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "web",
Meta: map[string]string{
"foo": "bar",
"gir": "zim",
},
Sources: []*SourceIntention{
{
Name: "foo",
Action: "deny",
Type: "consul",
Description: "foo desc",
},
{
Name: "bar",
Action: "allow",
Description: "bar desc",
},
{
Name: "*",
Action: "deny",
Description: "wild desc",
},
},
},
},
{
name: "service-intentions: wildcard destination",
snake: `
kind = "service-intentions"
name = "*"
sources {
name = "foo"
action = "deny"
# should be parsed, but we'll ignore it later
precedence = 6
}
`,
camel: `
Kind = "service-intentions"
Name = "*"
Sources {
Name = "foo"
Action = "deny"
# should be parsed, but we'll ignore it later
Precedence = 6
}
`,
expect: &ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "*",
Sources: []*SourceIntention{
{
Name: "foo",
Action: "deny",
Precedence: 6,
},
},
},
},
} {
tc := tc

View File

@ -2,6 +2,7 @@ package structs
import (
"encoding/binary"
"encoding/json"
"errors"
"fmt"
"sort"
@ -32,12 +33,12 @@ const (
// Connect.
type Intention struct {
// ID is the UUID-based ID for the intention, always generated by Consul.
ID string
ID string `json:",omitempty"`
// Description is a human-friendly description of this intention.
// It is opaque to Consul and is only stored and transferred in API
// requests.
Description string
Description string `json:",omitempty"`
// SourceNS, SourceName are the namespace and name, respectively, of
// the source service. Either of these may be the wildcard "*", but only
@ -58,14 +59,14 @@ type Intention struct {
// DefaultAddr is not used.
// Deprecated: DefaultAddr is not used and may be removed in a future version.
DefaultAddr string `bexpr:"-" codec:",omitempty"`
DefaultAddr string `bexpr:"-" codec:",omitempty" json:",omitempty"`
// DefaultPort is not used.
// Deprecated: DefaultPort is not used and may be removed in a future version.
DefaultPort int `bexpr:"-" codec:",omitempty"`
DefaultPort int `bexpr:"-" codec:",omitempty" json:",omitempty"`
// Meta is arbitrary metadata associated with the intention. This is
// opaque to Consul but is served in API responses.
Meta map[string]string
Meta map[string]string `json:",omitempty"`
// Precedence is the order that the intention will be applied, with
// larger numbers being applied first. This is a read-only field, on
@ -81,28 +82,49 @@ type Intention struct {
// This is needed mainly for replication purposes. When replicating from
// one DC to another keeping the content Hash will allow us to detect
// content changes more efficiently than checking every single field
Hash []byte `bexpr:"-"`
Hash []byte `bexpr:"-" json:",omitempty"`
RaftIndex `bexpr:"-"`
}
func (t *Intention) Clone() *Intention {
t2 := *t
if t.Meta != nil {
t2.Meta = make(map[string]string)
for k, v := range t.Meta {
t2.Meta[k] = v
}
}
t2.Meta = cloneStringStringMap(t.Meta)
t2.Hash = nil
return &t2
}
func (t *Intention) ToExact() *IntentionQueryExact {
return &IntentionQueryExact{
SourceNS: t.SourceNS,
SourceName: t.SourceName,
DestinationNS: t.DestinationNS,
DestinationName: t.DestinationName,
}
}
func (t *Intention) MarshalJSON() ([]byte, error) {
type Alias Intention
exported := &struct {
CreatedAt, UpdatedAt *time.Time `json:",omitempty"`
*Alias
}{
Alias: (*Alias)(t),
}
if !t.CreatedAt.IsZero() {
exported.CreatedAt = &t.CreatedAt
}
if !t.UpdatedAt.IsZero() {
exported.UpdatedAt = &t.UpdatedAt
}
return json.Marshal(exported)
}
func (t *Intention) UnmarshalJSON(data []byte) (err error) {
type Alias Intention
aux := &struct {
Hash string
CreatedAt, UpdatedAt string // effectively `json:"-"` on Intention type
CreatedAt, UpdatedAt string // effectively `json:"-"` on CreatedAt and UpdatedAt
*Alias
}{
@ -120,10 +142,10 @@ func (t *Intention) UnmarshalJSON(data []byte) (err error) {
// SetHash calculates Intention.Hash from any mutable "content" fields.
//
// The Hash is primarily used for replication to determine if a token
// has changed and should be updated locally.
// The Hash is primarily used for legacy intention replication to determine if
// an intention has changed and should be updated locally.
//
// TODO: move to agent/consul where it is called
// Deprecated: this is only used for legacy intention CRUD and replication
func (x *Intention) SetHash() {
hash, err := blake2b.New256(nil)
if err != nil {
@ -162,7 +184,9 @@ func (x *Intention) SetHash() {
}
// Validate returns an error if the intention is invalid for inserting
// or updating.
// or updating via the legacy APIs.
//
// Deprecated: this is only used for legacy intention CRUD
func (x *Intention) Validate() error {
var result error
@ -296,6 +320,8 @@ func (ixn *Intention) CanWrite(authz acl.Authorizer) bool {
// UpdatePrecedence sets the Precedence value based on the fields of this
// structure.
//
// Deprecated: this is only used for legacy intention CRUD.
func (x *Intention) UpdatePrecedence() {
// Max maintains the maximum value that the precedence can be depending
// on the number of exact values in the destination.
@ -346,8 +372,10 @@ func (x *Intention) String() string {
x.ID, x.Precedence)
}
// EstimateSize returns an estimate (in bytes) of the size of this structure when encoded.
func (x *Intention) EstimateSize() int {
// LegacyEstimateSize returns an estimate (in bytes) of the size of this structure when encoded.
//
// Deprecated: only exists for legacy intention replication during migration to 1.9.0+ cluster.
func (x *Intention) LegacyEstimateSize() int {
// 56 = 36 (uuid) + 16 (RaftIndex) + 4 (Precedence)
size := 56 + len(x.Description) + len(x.SourceNS) + len(x.SourceName) + len(x.DestinationNS) +
len(x.DestinationName) + len(x.SourceType) + len(x.Action)
@ -367,6 +395,32 @@ func (x *Intention) DestinationServiceName() ServiceName {
return NewServiceName(x.DestinationName, x.DestinationEnterpriseMeta())
}
// NOTE this is just used to manipulate user-provided data before an insert
// The RPC execution will do Normalize + Validate for us.
func (x *Intention) ToConfigEntry() *ServiceIntentionsConfigEntry {
return &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: x.DestinationName,
EnterpriseMeta: *x.DestinationEnterpriseMeta(),
Sources: []*SourceIntention{x.ToSourceIntention()},
}
}
func (x *Intention) ToSourceIntention() *SourceIntention {
return &SourceIntention{
Name: x.SourceName,
EnterpriseMeta: *x.SourceEnterpriseMeta(),
Action: x.Action,
Precedence: 0, // Ignore, let it be computed.
LegacyID: x.ID,
Type: x.SourceType,
Description: x.Description,
LegacyMeta: x.Meta,
LegacyCreateTime: nil, // Ignore
LegacyUpdateTime: nil, // Ignore
}
}
// IntentionAction is the action that the intention represents. This
// can be "allow" or "deny".
type IntentionAction string
@ -390,9 +444,20 @@ type Intentions []*Intention
// IndexedIntentions represents a list of intentions for RPC responses.
type IndexedIntentions struct {
Intentions Intentions
// DataOrigin is used to indicate if this query was satisfied against the
// old legacy intentions ("legacy") memdb table or via config entries
// ("config"). This is really only of value for the legacy intention
// replication routine to correctly detect that it should exit.
DataOrigin string `json:"-"`
QueryMeta
}
const (
IntentionDataOriginLegacy = "legacy"
IntentionDataOriginConfigEntries = "config"
)
// IndexedIntentionMatches represents the list of matches for a match query.
type IndexedIntentionMatches struct {
Matches []Intentions
@ -403,9 +468,11 @@ type IndexedIntentionMatches struct {
type IntentionOp string
const (
IntentionOpCreate IntentionOp = "create"
IntentionOpUpdate IntentionOp = "update"
IntentionOpDelete IntentionOp = "delete"
IntentionOpCreate IntentionOp = "create"
IntentionOpUpdate IntentionOp = "update"
IntentionOpDelete IntentionOp = "delete"
IntentionOpDeleteAll IntentionOp = "delete-all" // NOTE: this is only accepted when it comes from the leader, RPCs will reject this
IntentionOpUpsert IntentionOp = "upsert" // config-entry only
)
// IntentionRequest is used to create, update, and delete intentions.
@ -561,6 +628,17 @@ func (q *IntentionQueryExact) Validate() error {
return err
}
type IntentionListRequest struct {
Datacenter string
Legacy bool `json:"-"`
EnterpriseMeta `hcl:",squash" mapstructure:",squash"`
QueryOptions
}
func (r *IntentionListRequest) RequestDatacenter() string {
return r.Datacenter
}
// IntentionPrecedenceSorter takes a list of intentions and sorts them
// based on the match precedence rules for intentions. The intentions
// closer to the head of the list have higher precedence. i.e. index 0 has

View File

@ -14,6 +14,18 @@ func (ixn *Intention) DestinationEnterpriseMeta() *EnterpriseMeta {
return DefaultEnterpriseMeta()
}
func (e *IntentionMatchEntry) GetEnterpriseMeta() *EnterpriseMeta {
return DefaultEnterpriseMeta()
}
func (e *IntentionQueryExact) SourceEnterpriseMeta() *EnterpriseMeta {
return DefaultEnterpriseMeta()
}
func (e *IntentionQueryExact) DestinationEnterpriseMeta() *EnterpriseMeta {
return DefaultEnterpriseMeta()
}
// FillAuthzContext can fill in an acl.AuthorizerContext object to setup
// extra parameters for ACL enforcement. In OSS there is currently nothing
// extra to be done.

View File

@ -24,6 +24,12 @@ type SystemMetadataRequest struct {
WriteRequest
}
const (
SystemMetadataIntentionFormatKey = "intention-format"
SystemMetadataIntentionFormatConfigValue = "config-entry"
SystemMetadataIntentionFormatLegacyValue = "legacy"
)
type SystemMetadataEntry struct {
Key string
Value string `json:",omitempty"`

View File

@ -60,19 +60,34 @@ type TxnSessionOp struct {
Session Session
}
// TxnKVOp is used to define a single operation on an Intention inside a
// TxnIntentionOp is used to define a single operation on an Intention inside a
// transaction.
//
// Deprecated: see TxnOp.Intention description
type TxnIntentionOp IntentionRequest
// TxnOp is used to define a single operation inside a transaction. Only one
// of the types should be filled out per entry.
type TxnOp struct {
KV *TxnKVOp
KV *TxnKVOp
Node *TxnNodeOp
Service *TxnServiceOp
Check *TxnCheckOp
Session *TxnSessionOp
// Intention was an internal-only (not exposed in API or RPC)
// implementation detail of legacy intention replication. This is
// deprecated but retained for backwards compatibility with versions
// of consul pre-dating 1.9.0. We need it for two reasons:
//
// 1. If a secondary DC is upgraded first, we need to continue to
// replicate legacy intentions UNTIL the primary DC is upgraded.
// Legacy intention replication exclusively writes using a TxnOp.
// 2. If we attempt to reprocess raft-log contents pre-dating 1.9.0
// (such as when updating a secondary DC) we need to be able to
// recreate the state machine from the snapshot and whatever raft logs are
// present.
Intention *TxnIntentionOp
Node *TxnNodeOp
Service *TxnServiceOp
Check *TxnCheckOp
Session *TxnSessionOp
}
// TxnOps is a list of operations within a transaction.

View File

@ -620,7 +620,7 @@ func TestUIGatewayServiceNodes_Terminating(t *testing.T) {
req, _ := http.NewRequest("GET", "/v1/internal/ui/gateway-services-nodes/terminating-gateway", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.UIGatewayServicesNodes(resp, req)
assert.Nil(t, err)
require.Nil(t, err)
assertIndex(t, resp)
summary := obj.([]*ServiceSummary)
@ -648,7 +648,7 @@ func TestUIGatewayServiceNodes_Terminating(t *testing.T) {
EnterpriseMeta: *structs.DefaultEnterpriseMeta(),
},
}
assert.ElementsMatch(t, expect, summary)
require.ElementsMatch(t, expect, summary)
}
func TestUIGatewayServiceNodes_Ingress(t *testing.T) {
@ -773,7 +773,7 @@ func TestUIGatewayServiceNodes_Ingress(t *testing.T) {
req, _ := http.NewRequest("GET", "/v1/internal/ui/gateway-services-nodes/ingress-gateway", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.UIGatewayServicesNodes(resp, req)
assert.Nil(t, err)
require.Nil(t, err)
assertIndex(t, resp)
// Construct expected addresses so that differences between OSS/Ent are handled by code
@ -819,7 +819,7 @@ func TestUIGatewayServiceNodes_Ingress(t *testing.T) {
sum.GatewayConfig.addressesSet = nil
sum.checks = nil
}
assert.ElementsMatch(t, expect, dump)
require.ElementsMatch(t, expect, dump)
}
func TestUIGatewayIntentions(t *testing.T) {
@ -887,7 +887,7 @@ func TestUIGatewayIntentions(t *testing.T) {
req.Intention.DestinationName = v
var reply string
assert.NoError(t, a.RPC("Intention.Apply", &req, &reply))
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
req = structs.IntentionRequest{
Datacenter: "dc1",
@ -896,7 +896,7 @@ func TestUIGatewayIntentions(t *testing.T) {
}
req.Intention.SourceName = v
req.Intention.DestinationName = "api"
assert.NoError(t, a.RPC("Intention.Apply", &req, &reply))
require.NoError(t, a.RPC("Intention.Apply", &req, &reply))
}
}
@ -904,11 +904,11 @@ func TestUIGatewayIntentions(t *testing.T) {
req, _ := http.NewRequest("GET", "/v1/internal/ui/gateway-intentions/terminating-gateway", nil)
resp := httptest.NewRecorder()
obj, err := a.srv.UIGatewayIntentions(resp, req)
assert.Nil(t, err)
require.Nil(t, err)
assertIndex(t, resp)
intentions := obj.(structs.Intentions)
assert.Len(t, intentions, 3)
require.Len(t, intentions, 3)
// Only intentions with linked services as a destination should be returned, and wildcard matches should be deduped
expected := []string{"postgres", "*", "redis"}
@ -917,7 +917,7 @@ func TestUIGatewayIntentions(t *testing.T) {
intentions[1].DestinationName,
intentions[2].DestinationName,
}
assert.ElementsMatch(t, expected, actual)
require.ElementsMatch(t, expected, actual)
}
func TestUIEndpoint_modifySummaryForGatewayService_UseRequestedDCInsteadOfConfigured(t *testing.T) {

View File

@ -18,6 +18,7 @@ func TestMakeRBACNetworkFilter(t *testing.T) {
ixn.SourceName = src
ixn.DestinationName = dst
ixn.Action = action
//nolint:staticcheck
ixn.UpdatePrecedence()
return ixn
}
@ -117,6 +118,7 @@ func TestRemoveSameSourceIntentions(t *testing.T) {
ixn := structs.TestIntention(t)
ixn.SourceName = src
ixn.DestinationName = dst
//nolint:staticcheck
ixn.UpdatePrecedence()
return ixn
}

View File

@ -7,6 +7,7 @@ import (
"io"
"strconv"
"strings"
"time"
"github.com/mitchellh/mapstructure"
)
@ -19,6 +20,7 @@ const (
ServiceResolver string = "service-resolver"
IngressGateway string = "ingress-gateway"
TerminatingGateway string = "terminating-gateway"
ServiceIntentions string = "service-intentions"
ProxyConfigGlobal string = "global"
)
@ -178,6 +180,8 @@ func makeConfigEntry(kind, name string) (ConfigEntry, error) {
return &IngressGatewayConfigEntry{Kind: kind, Name: name}, nil
case TerminatingGateway:
return &TerminatingGatewayConfigEntry{Kind: kind, Name: name}, nil
case ServiceIntentions:
return &ServiceIntentionsConfigEntry{Kind: kind, Name: name}, nil
default:
return nil, fmt.Errorf("invalid config entry kind: %s", kind)
}
@ -220,7 +224,10 @@ func DecodeConfigEntry(raw map[string]interface{}) (ConfigEntry, error) {
}
decodeConf := &mapstructure.DecoderConfig{
DecodeHook: mapstructure.StringToTimeDurationHookFunc(),
DecodeHook: mapstructure.ComposeDecodeHookFunc(
mapstructure.StringToTimeDurationHookFunc(),
mapstructure.StringToTimeHookFunc(time.RFC3339),
),
Result: &entry,
WeaklyTypedInput: true,
}

View File

@ -0,0 +1,54 @@
package api
import "time"
type ServiceIntentionsConfigEntry struct {
Kind string
Name string
Namespace string `json:",omitempty"`
Sources []*SourceIntention
Meta map[string]string `json:",omitempty"`
CreateIndex uint64
ModifyIndex uint64
}
type SourceIntention struct {
Name string
Namespace string `json:",omitempty"`
Action IntentionAction
Precedence int
Type IntentionSourceType
Description string `json:",omitempty"`
LegacyID string `json:",omitempty" alias:"legacy_id"`
LegacyMeta map[string]string `json:",omitempty" alias:"legacy_meta"`
LegacyCreateTime *time.Time `json:",omitempty" alias:"legacy_create_time"`
LegacyUpdateTime *time.Time `json:",omitempty" alias:"legacy_update_time"`
}
func (e *ServiceIntentionsConfigEntry) GetKind() string {
return e.Kind
}
func (e *ServiceIntentionsConfigEntry) GetName() string {
return e.Name
}
func (e *ServiceIntentionsConfigEntry) GetNamespace() string {
return e.Namespace
}
func (e *ServiceIntentionsConfigEntry) GetMeta() map[string]string {
return e.Meta
}
func (e *ServiceIntentionsConfigEntry) GetCreateIndex() uint64 {
return e.CreateIndex
}
func (e *ServiceIntentionsConfigEntry) GetModifyIndex() uint64 {
return e.ModifyIndex
}

View File

@ -0,0 +1,136 @@
package api
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestAPI_ConfigEntries_ServiceIntentions(t *testing.T) {
t.Parallel()
c, s := makeClient(t)
defer s.Stop()
s.WaitForServiceIntentions(t)
config_entries := c.ConfigEntries()
// Allow L7 for all services.
_, _, err := config_entries.Set(&ProxyConfigEntry{
Kind: ProxyDefaults,
Name: ProxyConfigGlobal,
Config: map[string]interface{}{
"protocol": "http",
},
}, nil)
require.NoError(t, err)
entries := []*ServiceIntentionsConfigEntry{
{
Kind: ServiceIntentions,
Name: "foo",
Sources: []*SourceIntention{
{
Name: "one",
Action: IntentionActionAllow,
},
{
Name: "two",
Action: IntentionActionDeny,
},
},
},
{
Kind: ServiceIntentions,
Name: "bar",
Sources: []*SourceIntention{
{
Name: "three",
Action: IntentionActionAllow,
},
},
},
}
// set them
for _, entry := range entries {
_, wm, err := config_entries.Set(entry, nil)
require.NoError(t, err)
require.NotNil(t, wm)
require.NotEqual(t, 0, wm.RequestTime)
}
// get one
entry, qm, err := config_entries.Get(ServiceIntentions, "foo", nil)
require.NoError(t, err)
require.NotNil(t, qm)
require.NotEqual(t, 0, qm.RequestTime)
// verify it
readIxn, ok := entry.(*ServiceIntentionsConfigEntry)
require.True(t, ok)
require.Equal(t, "service-intentions", readIxn.Kind)
require.Equal(t, "foo", readIxn.Name)
require.Len(t, readIxn.Sources, 2)
// update it
entries[0].Meta = map[string]string{"a": "b"}
// CAS fail
written, _, err := config_entries.CAS(entries[0], 0, nil)
require.NoError(t, err)
require.False(t, written)
// CAS success
written, wm, err := config_entries.CAS(entries[0], readIxn.ModifyIndex, nil)
require.NoError(t, err)
require.NotNil(t, wm)
require.NotEqual(t, 0, wm.RequestTime)
require.True(t, written)
// update no cas
entries[0].Meta = map[string]string{"x": "y"}
_, wm, err = config_entries.Set(entries[0], nil)
require.NoError(t, err)
require.NotNil(t, wm)
require.NotEqual(t, 0, wm.RequestTime)
// list them
gotEntries, qm, err := config_entries.List(ServiceIntentions, nil)
require.NoError(t, err)
require.NotNil(t, qm)
require.NotEqual(t, 0, qm.RequestTime)
require.Len(t, gotEntries, 2)
for _, entry = range gotEntries {
switch entry.GetName() {
case "foo":
// this also verifies that the update value was persisted and
// the updated values are seen
readIxn, ok = entry.(*ServiceIntentionsConfigEntry)
require.True(t, ok)
require.Equal(t, "service-intentions", readIxn.Kind)
require.Equal(t, "foo", readIxn.Name)
require.Len(t, readIxn.Sources, 2)
require.Equal(t, map[string]string{"x": "y"}, readIxn.Meta)
case "bar":
readIxn, ok = entry.(*ServiceIntentionsConfigEntry)
require.True(t, ok)
require.Equal(t, "service-intentions", readIxn.Kind)
require.Equal(t, "bar", readIxn.Name)
require.Len(t, readIxn.Sources, 1)
require.Empty(t, readIxn.Meta)
}
}
// delete one
wm, err = config_entries.Delete(ServiceIntentions, "foo", nil)
require.NoError(t, err)
require.NotNil(t, wm)
require.NotEqual(t, 0, wm.RequestTime)
// verify deletion
_, _, err = config_entries.Get(ServiceIntentions, "foo", nil)
require.Error(t, err)
}

View File

@ -12,12 +12,12 @@ import (
// Connect.
type Intention struct {
// ID is the UUID-based ID for the intention, always generated by Consul.
ID string
ID string `json:",omitempty"`
// Description is a human-friendly description of this intention.
// It is opaque to Consul and is only stored and transferred in API
// requests.
Description string
Description string `json:",omitempty"`
// SourceNS, SourceName are the namespace and name, respectively, of
// the source service. Either of these may be the wildcard "*", but only
@ -45,7 +45,7 @@ type Intention struct {
// Meta is arbitrary metadata associated with the intention. This is
// opaque to Consul but is served in API responses.
Meta map[string]string
Meta map[string]string `json:",omitempty"`
// Precedence is the order that the intention will be applied, with
// larger numbers being applied first. This is a read-only field, on
@ -61,7 +61,7 @@ type Intention struct {
// This is needed mainly for replication purposes. When replicating from
// one DC to another keeping the content Hash will allow us to detect
// content changes more efficiently than checking every single field
Hash []byte
Hash []byte `json:",omitempty"`
CreateIndex uint64
ModifyIndex uint64
@ -166,7 +166,42 @@ func (h *Connect) Intentions(q *QueryOptions) ([]*Intention, *QueryMeta, error)
return out, qm, nil
}
// IntentionGetExact retrieves a single intention by its unique name instead of
// its ID.
func (h *Connect) IntentionGetExact(source, destination string, q *QueryOptions) (*Intention, *QueryMeta, error) {
r := h.c.newRequest("GET", "/v1/connect/intentions/exact")
r.setQueryOptions(q)
r.params.Set("source", source)
r.params.Set("destination", destination)
rtt, resp, err := h.c.doRequest(r)
if err != nil {
return nil, nil, err
}
defer resp.Body.Close()
qm := &QueryMeta{}
parseQueryMeta(resp, qm)
qm.RequestTime = rtt
if resp.StatusCode == 404 {
return nil, qm, nil
} else if resp.StatusCode != 200 {
var buf bytes.Buffer
io.Copy(&buf, resp.Body)
return nil, nil, fmt.Errorf(
"Unexpected response %d: %s", resp.StatusCode, buf.String())
}
var out Intention
if err := decodeBody(resp, &out); err != nil {
return nil, nil, err
}
return &out, qm, nil
}
// IntentionGet retrieves a single intention.
//
// Deprecated: use IntentionGetExact instead
func (h *Connect) IntentionGet(id string, q *QueryOptions) (*Intention, *QueryMeta, error) {
r := h.c.newRequest("GET", "/v1/connect/intentions/"+id)
r.setQueryOptions(q)
@ -196,7 +231,28 @@ func (h *Connect) IntentionGet(id string, q *QueryOptions) (*Intention, *QueryMe
return &out, qm, nil
}
// IntentionDeleteExact deletes a single intention by its unique name instead of its ID.
func (h *Connect) IntentionDeleteExact(source, destination string, q *WriteOptions) (*WriteMeta, error) {
r := h.c.newRequest("DELETE", "/v1/connect/intentions/exact")
r.setWriteOptions(q)
r.params.Set("source", source)
r.params.Set("destination", destination)
rtt, resp, err := requireOK(h.c.doRequest(r))
if err != nil {
return nil, err
}
defer resp.Body.Close()
qm := &WriteMeta{}
qm.RequestTime = rtt
return qm, nil
}
// IntentionDelete deletes a single intention.
//
// Deprecated: use IntentionDeleteExact instead
func (h *Connect) IntentionDelete(id string, q *WriteOptions) (*WriteMeta, error) {
r := h.c.newRequest("DELETE", "/v1/connect/intentions/"+id)
r.setWriteOptions(q)
@ -270,42 +326,37 @@ func (h *Connect) IntentionCheck(args *IntentionCheck, q *QueryOptions) (bool, *
return out.Allowed, qm, nil
}
// IntentionGetExact retrieves a single intention by its unique name instead of
// its ID.
func (h *Connect) IntentionGetExact(source, destination string, q *QueryOptions) (*Intention, *QueryMeta, error) {
r := h.c.newRequest("GET", "/v1/connect/intentions/exact")
r.setQueryOptions(q)
r.params.Set("source", source)
r.params.Set("destination", destination)
rtt, resp, err := h.c.doRequest(r)
// IntentionUpsert will update an existing intention. The Source & Destination parameters
// in the structure must be non-empty. The ID must be empty.
func (c *Connect) IntentionUpsert(ixn *Intention, q *WriteOptions) (*WriteMeta, error) {
r := c.c.newRequest("PUT", "/v1/connect/intentions/exact")
r.setWriteOptions(q)
r.params.Set("source", maybePrefixNamespace(ixn.SourceNS, ixn.SourceName))
r.params.Set("destination", maybePrefixNamespace(ixn.DestinationNS, ixn.DestinationName))
r.obj = ixn
rtt, resp, err := requireOK(c.c.doRequest(r))
if err != nil {
return nil, nil, err
return nil, err
}
defer resp.Body.Close()
qm := &QueryMeta{}
parseQueryMeta(resp, qm)
qm.RequestTime = rtt
wm := &WriteMeta{}
wm.RequestTime = rtt
return wm, nil
}
if resp.StatusCode == 404 {
return nil, qm, nil
} else if resp.StatusCode != 200 {
var buf bytes.Buffer
io.Copy(&buf, resp.Body)
return nil, nil, fmt.Errorf(
"Unexpected response %d: %s", resp.StatusCode, buf.String())
func maybePrefixNamespace(ns, name string) string {
if ns == "" {
return name
}
var out Intention
if err := decodeBody(resp, &out); err != nil {
return nil, nil, err
}
return &out, qm, nil
return ns + "/" + name
}
// IntentionCreate will create a new intention. The ID in the given
// structure must be empty and a generate ID will be returned on
// success.
//
// Deprecated: use IntentionUpsert instead
func (c *Connect) IntentionCreate(ixn *Intention, q *WriteOptions) (string, *WriteMeta, error) {
r := c.c.newRequest("POST", "/v1/connect/intentions")
r.setWriteOptions(q)
@ -328,6 +379,8 @@ func (c *Connect) IntentionCreate(ixn *Intention, q *WriteOptions) (string, *Wri
// IntentionUpdate will update an existing intention. The ID in the given
// structure must be non-empty.
//
// Deprecated: use IntentionUpsert instead
func (c *Connect) IntentionUpdate(ixn *Intention, q *WriteOptions) (*WriteMeta, error) {
r := c.c.newRequest("PUT", "/v1/connect/intentions/"+ixn.ID)
r.setWriteOptions(q)

View File

@ -9,7 +9,6 @@ import (
func TestAPI_ConnectIntentionCreateListGetUpdateDelete(t *testing.T) {
t.Parallel()
require := require.New(t)
c, s := makeClient(t)
defer s.Stop()
@ -18,13 +17,13 @@ func TestAPI_ConnectIntentionCreateListGetUpdateDelete(t *testing.T) {
// Create
ixn := testIntention()
id, _, err := connect.IntentionCreate(ixn, nil)
require.Nil(err)
require.NotEmpty(id)
require.Nil(t, err)
require.NotEmpty(t, id)
// List it
list, _, err := connect.Intentions(nil)
require.Nil(err)
require.Len(list, 1)
require.Nil(t, err)
require.Len(t, list, 1)
actual := list[0]
ixn.ID = id
@ -33,40 +32,39 @@ func TestAPI_ConnectIntentionCreateListGetUpdateDelete(t *testing.T) {
ixn.CreateIndex = actual.CreateIndex
ixn.ModifyIndex = actual.ModifyIndex
ixn.Hash = actual.Hash
require.Equal(ixn, actual)
require.Equal(t, ixn, actual)
// Get it
actual, _, err = connect.IntentionGet(id, nil)
require.Nil(err)
require.Equal(ixn, actual)
require.Nil(t, err)
require.Equal(t, ixn, actual)
// Update it
ixn.SourceName = ixn.SourceName + "-different"
_, err = connect.IntentionUpdate(ixn, nil)
require.NoError(err)
require.NoError(t, err)
// Get it
actual, _, err = connect.IntentionGet(id, nil)
require.NoError(err)
require.NoError(t, err)
ixn.UpdatedAt = actual.UpdatedAt
ixn.ModifyIndex = actual.ModifyIndex
ixn.Hash = actual.Hash
require.Equal(ixn, actual)
require.Equal(t, ixn, actual)
// Delete it
_, err = connect.IntentionDelete(id, nil)
require.Nil(err)
require.Nil(t, err)
// Get it (should be gone)
actual, _, err = connect.IntentionGet(id, nil)
require.Nil(err)
require.Nil(actual)
require.Nil(t, err)
require.Nil(t, actual)
}
func TestAPI_ConnectIntentionGet_invalidId(t *testing.T) {
t.Parallel()
require := require.New(t)
c, s := makeClient(t)
defer s.Stop()
@ -74,15 +72,14 @@ func TestAPI_ConnectIntentionGet_invalidId(t *testing.T) {
// Get it
actual, _, err := connect.IntentionGet("hello", nil)
require.Nil(actual)
require.Error(err)
require.Contains(err.Error(), "UUID") // verify it contains the message
require.Nil(t, actual)
require.Error(t, err)
require.Contains(t, err.Error(), "UUID") // verify it contains the message
}
func TestAPI_ConnectIntentionMatch(t *testing.T) {
t.Parallel()
require := require.New(t)
c, s := makeClient(t)
defer s.Stop()
@ -101,8 +98,8 @@ func TestAPI_ConnectIntentionMatch(t *testing.T) {
ixn.DestinationNS = v[0]
ixn.DestinationName = v[1]
id, _, err := connect.IntentionCreate(ixn, nil)
require.Nil(err)
require.NotEmpty(id)
require.Nil(t, err)
require.NotEmpty(t, id)
}
}
@ -111,8 +108,8 @@ func TestAPI_ConnectIntentionMatch(t *testing.T) {
By: IntentionMatchDestination,
Names: []string{"bar"},
}, nil)
require.Nil(err)
require.Len(result, 1)
require.Nil(t, err)
require.Len(t, result, 1)
var actual [][]string
expected := [][]string{
@ -123,13 +120,12 @@ func TestAPI_ConnectIntentionMatch(t *testing.T) {
actual = append(actual, []string{ixn.DestinationNS, ixn.DestinationName})
}
require.Equal(expected, actual)
require.Equal(t, expected, actual)
}
func TestAPI_ConnectIntentionCheck(t *testing.T) {
t.Parallel()
require := require.New(t)
c, s := makeClient(t)
defer s.Stop()
@ -150,8 +146,8 @@ func TestAPI_ConnectIntentionCheck(t *testing.T) {
ixn.DestinationName = v[3]
ixn.Action = IntentionAction(v[4])
id, _, err := connect.IntentionCreate(ixn, nil)
require.Nil(err)
require.NotEmpty(id)
require.Nil(t, err)
require.NotEmpty(t, id)
}
}
@ -161,8 +157,8 @@ func TestAPI_ConnectIntentionCheck(t *testing.T) {
Source: "default/qux",
Destination: "default/bar",
}, nil)
require.NoError(err)
require.False(result)
require.NoError(t, err)
require.False(t, result)
}
// Match the allow rule
@ -171,8 +167,8 @@ func TestAPI_ConnectIntentionCheck(t *testing.T) {
Source: "default/foo",
Destination: "default/bar",
}, nil)
require.NoError(err)
require.True(result)
require.NoError(t, err)
require.True(t, result)
}
}
@ -185,6 +181,5 @@ func testIntention() *Intention {
Precedence: 9,
Action: IntentionActionAllow,
SourceType: IntentionSourceConsul,
Meta: map[string]string{},
}
}

View File

@ -4,6 +4,7 @@ import (
"flag"
"fmt"
"io"
"time"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/command/flags"
@ -137,6 +138,7 @@ func newDecodeConfigEntry(raw map[string]interface{}) (api.ConfigEntry, error) {
decode.HookWeakDecodeFromSlice,
decode.HookTranslateKeys,
mapstructure.StringToTimeDurationHookFunc(),
mapstructure.StringToTimeHookFunc(time.RFC3339),
),
Metadata: &md,
Result: &entry,

View File

@ -1,12 +1,13 @@
package write
import (
"github.com/hashicorp/consul/agent/structs"
"io"
"strings"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil"
@ -1910,6 +1911,203 @@ func TestParseConfigEntry(t *testing.T) {
},
},
},
{
name: "service-intentions: kitchen sink",
snake: `
kind = "service-intentions"
name = "web"
meta {
"foo" = "bar"
"gir" = "zim"
}
sources = [
{
name = "foo"
action = "deny"
type = "consul"
description = "foo desc"
},
{
name = "bar"
action = "allow"
description = "bar desc"
}
]
sources {
name = "*"
action = "deny"
description = "wild desc"
}
`,
camel: `
Kind = "service-intentions"
Name = "web"
Meta {
"foo" = "bar"
"gir" = "zim"
}
Sources = [
{
Name = "foo"
Action = "deny"
Type = "consul"
Description = "foo desc"
},
{
Name = "bar"
Action = "allow"
Description = "bar desc"
}
]
Sources {
Name = "*"
Action = "deny"
Description = "wild desc"
}
`,
snakeJSON: `
{
"kind": "service-intentions",
"name": "web",
"meta" : {
"foo": "bar",
"gir": "zim"
},
"sources": [
{
"name": "foo",
"action": "deny",
"type": "consul",
"description": "foo desc"
},
{
"name": "bar",
"action": "allow",
"description": "bar desc"
},
{
"name": "*",
"action": "deny",
"description": "wild desc"
}
]
}
`,
camelJSON: `
{
"Kind": "service-intentions",
"Name": "web",
"Meta" : {
"foo": "bar",
"gir": "zim"
},
"Sources": [
{
"Name": "foo",
"Action": "deny",
"Type": "consul",
"Description": "foo desc"
},
{
"Name": "bar",
"Action": "allow",
"Description": "bar desc"
},
{
"Name": "*",
"Action": "deny",
"Description": "wild desc"
}
]
}
`,
expect: &api.ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "web",
Meta: map[string]string{
"foo": "bar",
"gir": "zim",
},
Sources: []*api.SourceIntention{
{
Name: "foo",
Action: "deny",
Type: "consul",
Description: "foo desc",
},
{
Name: "bar",
Action: "allow",
Description: "bar desc",
},
{
Name: "*",
Action: "deny",
Description: "wild desc",
},
},
},
},
{
name: "service-intentions: wildcard destination",
snake: `
kind = "service-intentions"
name = "*"
sources {
name = "foo"
action = "deny"
# should be parsed, but we'll ignore it later
precedence = 6
}
`,
camel: `
Kind = "service-intentions"
Name = "*"
Sources {
Name = "foo"
Action = "deny"
# should be parsed, but we'll ignore it later
Precedence = 6
}
`,
snakeJSON: `
{
"kind": "service-intentions",
"name": "*",
"sources": [
{
"name": "foo",
"action": "deny",
"precedence": 6
}
]
}
`,
camelJSON: `
{
"Kind": "service-intentions",
"Name": "*",
"Sources": [
{
"Name": "foo",
"Action": "deny",
"Precedence": 6
}
]
}
`,
expect: &api.ServiceIntentionsConfigEntry{
Kind: "service-intentions",
Name: "*",
Sources: []*api.SourceIntention{
{
Name: "foo",
Action: "deny",
Precedence: 6,
},
},
},
},
} {
tc := tc

View File

@ -9,7 +9,7 @@ import (
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/command/flags"
"github.com/hashicorp/consul/command/intention/create"
"github.com/hashicorp/consul/command/intention"
"github.com/mitchellh/cli"
)
@ -79,7 +79,7 @@ func (c *cmd) Run(args []string) int {
c.UI.Error("A service name must be given via the -service flag.")
return 1
}
svc, svcNamespace, err := create.ParseIntentionTarget(c.service)
svc, svcNamespace, err := intention.ParseIntentionTarget(c.service)
if err != nil {
c.UI.Error(fmt.Sprintf("Invalid service name: %s", err))
return 1
@ -89,7 +89,7 @@ func (c *cmd) Run(args []string) int {
c.UI.Error("An ingress gateway service must be given via the -ingress-gateway flag.")
return 1
}
gateway, gatewayNamespace, err := create.ParseIntentionTarget(c.ingressGateway)
gateway, gatewayNamespace, err := intention.ParseIntentionTarget(c.ingressGateway)
if err != nil {
c.UI.Error(fmt.Sprintf("Invalid ingress gateway name: %s", err))
return 1
@ -201,18 +201,9 @@ func (c *cmd) Run(args []string) int {
SourceType: api.IntentionSourceConsul,
Action: api.IntentionActionAllow,
}
if existing == nil {
_, _, err = client.Connect().IntentionCreate(ixn, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error creating intention: %s", err))
return 1
}
} else {
_, err = client.Connect().IntentionUpdate(ixn, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error updating intention: %s", err))
return 1
}
if _, err = client.Connect().IntentionUpsert(ixn, nil); err != nil {
c.UI.Error(fmt.Sprintf("Error upserting intention: %s", err))
return 1
}
c.UI.Output(fmt.Sprintf("Successfully set up intention for %q -> %q", c.ingressGateway, c.service))

View File

@ -6,18 +6,19 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testrpc"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestCommand_noTabs(t *testing.T) {
func TestIntentionCheck_noTabs(t *testing.T) {
t.Parallel()
if strings.ContainsRune(New(nil).Help(), '\t') {
t.Fatal("help has tabs")
}
}
func TestCommand_Validation(t *testing.T) {
func TestIntentionCheck_Validation(t *testing.T) {
t.Parallel()
ui := cli.NewMockUi()
@ -64,7 +65,7 @@ func TestCommand_Validation(t *testing.T) {
}
}
func TestCommand(t *testing.T) {
func TestIntentionCheck(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -72,8 +73,11 @@ func TestCommand(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create the intention
{
//nolint:staticcheck
_, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",

View File

@ -6,7 +6,6 @@ import (
"fmt"
"io"
"os"
"strings"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/command/flags"
@ -111,6 +110,7 @@ func (c *cmd) Run(args []string) int {
// We set the ID of our intention so we overwrite it
ixn.ID = oldIxn.ID
//nolint:staticcheck
if _, err := client.Connect().IntentionUpdate(ixn, nil); err != nil {
c.UI.Error(fmt.Sprintf(
"Error replacing intention with source %q "+
@ -126,6 +126,7 @@ func (c *cmd) Run(args []string) int {
}
}
//nolint:staticcheck
_, _, err := client.Connect().IntentionCreate(ixn, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error creating intention %q: %s", ixn, err))
@ -138,27 +139,6 @@ func (c *cmd) Run(args []string) int {
return 0
}
// ParseIntentionTarget parses a target of the form <namespace>/<name> and returns
// the two distinct parts. In some cases the namespace may be elided and this function
// will return the empty string for the namespace then.
func ParseIntentionTarget(input string) (name string, namespace string, err error) {
// Get the index to the '/'. If it doesn't exist, we have just a name
// so just set that and return.
idx := strings.IndexByte(input, '/')
if idx == -1 {
// let the agent do token based defaulting of the namespace
return input, "", nil
}
namespace = input[:idx]
name = input[idx+1:]
if strings.IndexByte(name, '/') != -1 {
return "", "", fmt.Errorf("target can contain at most one '/'")
}
return name, namespace, nil
}
// ixnsFromArgs returns the set of intentions to create based on the arguments
// given and the flags set. This will call ixnsFromFiles if the -file flag
// was set.
@ -173,12 +153,12 @@ func (c *cmd) ixnsFromArgs(args []string) ([]*api.Intention, error) {
return nil, fmt.Errorf("Must specify two arguments: source and destination")
}
srcName, srcNamespace, err := ParseIntentionTarget(args[0])
srcName, srcNamespace, err := intention.ParseIntentionTarget(args[0])
if err != nil {
return nil, fmt.Errorf("Invalid intention source: %v", err)
}
dstName, dstNamespace, err := ParseIntentionTarget(args[1])
dstName, dstNamespace, err := intention.ParseIntentionTarget(args[1])
if err != nil {
return nil, fmt.Errorf("Invalid intention destination: %v", err)
}

View File

@ -7,18 +7,19 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/testrpc"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestCommand_noTabs(t *testing.T) {
func TestIntentionCreate_noTabs(t *testing.T) {
t.Parallel()
if strings.ContainsRune(New(nil).Help(), '\t') {
t.Fatal("help has tabs")
}
}
func TestCommand_Validation(t *testing.T) {
func TestIntentionCreate_Validation(t *testing.T) {
t.Parallel()
ui := cli.NewMockUi()
@ -55,7 +56,7 @@ func TestCommand_Validation(t *testing.T) {
}
}
func TestCommand(t *testing.T) {
func TestIntentionCreate(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -63,6 +64,8 @@ func TestCommand(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ui := cli.NewMockUi()
c := New(ui)
@ -80,7 +83,7 @@ func TestCommand(t *testing.T) {
require.Equal(api.IntentionActionAllow, ixns[0].Action)
}
func TestCommand_deny(t *testing.T) {
func TestIntentionCreate_deny(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -88,6 +91,8 @@ func TestCommand_deny(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ui := cli.NewMockUi()
c := New(ui)
@ -106,7 +111,7 @@ func TestCommand_deny(t *testing.T) {
require.Equal(api.IntentionActionDeny, ixns[0].Action)
}
func TestCommand_meta(t *testing.T) {
func TestIntentionCreate_meta(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -114,6 +119,8 @@ func TestCommand_meta(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ui := cli.NewMockUi()
c := New(ui)
@ -132,7 +139,7 @@ func TestCommand_meta(t *testing.T) {
require.Equal(map[string]string{"hello": "world"}, ixns[0].Meta)
}
func TestCommand_File(t *testing.T) {
func TestIntentionCreate_File(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -140,6 +147,8 @@ func TestCommand_File(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ui := cli.NewMockUi()
c := New(ui)
@ -165,13 +174,15 @@ func TestCommand_File(t *testing.T) {
require.Equal(api.IntentionActionAllow, ixns[0].Action)
}
func TestCommand_FileNoExist(t *testing.T) {
func TestIntentionCreate_FileNoExist(t *testing.T) {
t.Parallel()
require := require.New(t)
a := agent.NewTestAgent(t, ``)
defer a.Shutdown()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ui := cli.NewMockUi()
c := New(ui)
@ -185,7 +196,7 @@ func TestCommand_FileNoExist(t *testing.T) {
require.Contains(ui.ErrorWriter.String(), "no such file")
}
func TestCommand_replace(t *testing.T) {
func TestIntentionCreate_replace(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -193,6 +204,8 @@ func TestCommand_replace(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create the first
{
ui := cli.NewMockUi()
@ -223,7 +236,7 @@ func TestCommand_replace(t *testing.T) {
"foo", "bar",
}
require.Equal(1, c.Run(args), ui.ErrorWriter.String())
require.Contains(ui.ErrorWriter.String(), "duplicate")
require.Contains(ui.ErrorWriter.String(), "more than once")
}
// Replace it

View File

@ -6,7 +6,6 @@ import (
"io"
"github.com/hashicorp/consul/command/flags"
"github.com/hashicorp/consul/command/intention/finder"
"github.com/mitchellh/cli"
)
@ -47,17 +46,25 @@ func (c *cmd) Run(args []string) int {
return 1
}
// Get the intention ID to load
id, err := finder.IDFromArgs(client, c.flags.Args())
if err != nil {
c.UI.Error(fmt.Sprintf("Error: %s", err))
switch args := c.flags.Args(); len(args) {
case 1:
// old-style
id := args[0]
//nolint:staticcheck
_, err = client.Connect().IntentionDelete(id, nil)
case 2:
// new-style
source, destination := args[0], args[1]
_, err = client.Connect().IntentionDeleteExact(source, destination, nil)
default:
c.UI.Error("command requires exactly 1 or 2 arguments")
return 1
}
// Read the intention
_, err = client.Connect().IntentionDelete(id, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error reading the intention: %s", err))
c.UI.Error(fmt.Sprintf("Error deleting the intention: %s", err))
return 1
}

View File

@ -6,18 +6,19 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testrpc"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestCommand_noTabs(t *testing.T) {
func TestIntentionDelete_noTabs(t *testing.T) {
t.Parallel()
if strings.ContainsRune(New(nil).Help(), '\t') {
t.Fatal("help has tabs")
}
}
func TestCommand_Validation(t *testing.T) {
func TestIntentionDelete_Validation(t *testing.T) {
t.Parallel()
ui := cli.NewMockUi()
@ -40,8 +41,6 @@ func TestCommand_Validation(t *testing.T) {
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
require := require.New(t)
c.init()
// Ensure our buffer is always clear
@ -52,48 +51,69 @@ func TestCommand_Validation(t *testing.T) {
ui.OutputWriter.Reset()
}
require.Equal(1, c.Run(tc.args))
require.Equal(t, 1, c.Run(tc.args))
output := ui.ErrorWriter.String()
require.Contains(output, tc.output)
require.Contains(t, output, tc.output)
})
}
}
func TestCommand(t *testing.T) {
func TestIntentionDelete(t *testing.T) {
t.Parallel()
require := require.New(t)
a := agent.NewTestAgent(t, ``)
defer a.Shutdown()
client := a.Client()
// Create the intention
{
_, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
Action: api.IntentionActionDeny,
}, nil)
require.NoError(err)
}
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Delete it
{
ui := cli.NewMockUi()
c := New(ui)
// Create some intentions.
args := []string{
"-http-addr=" + a.HTTPAddr(),
"web", "db",
}
require.Equal(0, c.Run(args), ui.ErrorWriter.String())
require.Contains(ui.OutputWriter.String(), "deleted")
}
//nolint:staticcheck
id0, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
Action: api.IntentionActionDeny,
}, nil)
require.NoError(t, err)
// Find it (should be gone)
//nolint:staticcheck
_, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "queue",
Action: api.IntentionActionDeny,
}, nil)
require.NoError(t, err)
t.Run("l4 intention", func(t *testing.T) {
t.Run("one arg", func(t *testing.T) {
ui := cli.NewMockUi()
c := New(ui)
args := []string{
"-http-addr=" + a.HTTPAddr(),
id0,
}
require.Equal(t, 0, c.Run(args), ui.ErrorWriter.String())
require.Contains(t, ui.OutputWriter.String(), "deleted")
})
t.Run("two args", func(t *testing.T) {
ui := cli.NewMockUi()
c := New(ui)
args := []string{
"-http-addr=" + a.HTTPAddr(),
"web", "queue",
}
require.Equal(t, 0, c.Run(args), ui.ErrorWriter.String())
require.Contains(t, ui.OutputWriter.String(), "deleted")
})
})
// They should all be gone.
{
ixns, _, err := client.Connect().Intentions(nil)
require.NoError(err)
require.Len(ixns, 0)
require.NoError(t, err)
require.Len(t, ixns, 0)
}
}

View File

@ -1,34 +0,0 @@
package finder
import (
"fmt"
"github.com/hashicorp/consul/api"
)
// IDFromArgs returns the intention ID for the given CLI args. An error is returned
// if args is not 1 or 2 elements.
func IDFromArgs(client *api.Client, args []string) (string, error) {
switch len(args) {
case 1:
return args[0], nil
case 2:
ixn, _, err := client.Connect().IntentionGetExact(
args[0], args[1], nil,
)
if err != nil {
return "", err
}
if ixn == nil {
return "", fmt.Errorf(
"Intention with source %q and destination %q not found.",
args[0], args[1])
}
return ixn.ID, nil
default:
return "", fmt.Errorf("command requires exactly 1 or 2 arguments")
}
}

View File

@ -1,44 +0,0 @@
package finder
import (
"testing"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/stretchr/testify/require"
)
func TestIDFromArgs(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t, ``)
defer a.Shutdown()
client := a.Client()
// Create a set of intentions
var ids []string
{
insert := [][]string{
{"a", "b"},
}
for _, v := range insert {
ixn := &api.Intention{
SourceName: v[0],
DestinationName: v[1],
Action: api.IntentionActionAllow,
}
id, _, err := client.Connect().IntentionCreate(ixn, nil)
require.NoError(t, err)
ids = append(ids, id)
}
}
id, err := IDFromArgs(client, []string{"a", "b"})
require.NoError(t, err)
require.Equal(t, ids[0], id)
_, err = IDFromArgs(client, []string{"c", "d"})
require.Error(t, err)
}

View File

@ -8,7 +8,7 @@ import (
"time"
"github.com/hashicorp/consul/command/flags"
"github.com/hashicorp/consul/command/intention/finder"
"github.com/hashicorp/consul/command/intention"
"github.com/mitchellh/cli"
"github.com/ryanuber/columnize"
)
@ -50,17 +50,9 @@ func (c *cmd) Run(args []string) int {
return 1
}
// Get the intention ID to load
id, err := finder.IDFromArgs(client, c.flags.Args())
ixn, err := intention.GetFromArgs(client, c.flags.Args())
if err != nil {
c.UI.Error(fmt.Sprintf("Error: %s", err))
return 1
}
// Read the intention
ixn, _, err := client.Connect().IntentionGet(id, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error reading the intention: %s", err))
c.UI.Error(err.Error())
return 1
}
@ -69,7 +61,9 @@ func (c *cmd) Run(args []string) int {
fmt.Sprintf("Source:\x1f%s", ixn.SourceString()),
fmt.Sprintf("Destination:\x1f%s", ixn.DestinationString()),
fmt.Sprintf("Action:\x1f%s", ixn.Action),
fmt.Sprintf("ID:\x1f%s", ixn.ID),
}
if ixn.ID != "" {
data = append(data, fmt.Sprintf("ID:\x1f%s", ixn.ID))
}
if v := ixn.Description; v != "" {
data = append(data, fmt.Sprintf("Description:\x1f%s", v))
@ -89,6 +83,7 @@ func (c *cmd) Run(args []string) int {
)
c.UI.Output(columnize.Format(data, &columnize.Config{Delim: string([]byte{0x1f})}))
return 0
}

View File

@ -6,18 +6,21 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testrpc"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestCommand_noTabs(t *testing.T) {
// TODO(intentions): add test for viewing permissions and ID-less
func TestIntentionGet_noTabs(t *testing.T) {
t.Parallel()
if strings.ContainsRune(New(nil).Help(), '\t') {
t.Fatal("help has tabs")
}
}
func TestCommand_Validation(t *testing.T) {
func TestIntentionGet_Validation(t *testing.T) {
t.Parallel()
ui := cli.NewMockUi()
@ -59,7 +62,7 @@ func TestCommand_Validation(t *testing.T) {
}
}
func TestCommand_id(t *testing.T) {
func TestIntentionGet_id(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -67,10 +70,13 @@ func TestCommand_id(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create the intention
var id string
{
var err error
//nolint:staticcheck
id, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
@ -91,7 +97,7 @@ func TestCommand_id(t *testing.T) {
require.Contains(ui.OutputWriter.String(), id)
}
func TestCommand_srcDst(t *testing.T) {
func TestIntentionGet_srcDst(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -99,10 +105,13 @@ func TestCommand_srcDst(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create the intention
var id string
{
var err error
//nolint:staticcheck
id, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: "web",
DestinationName: "db",
@ -123,7 +132,7 @@ func TestCommand_srcDst(t *testing.T) {
require.Contains(ui.OutputWriter.String(), id)
}
func TestCommand_verticalBar(t *testing.T) {
func TestIntentionGet_verticalBar(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -131,12 +140,15 @@ func TestCommand_verticalBar(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
sourceName := "source|name|with|bars"
// Create the intention
var id string
{
var err error
//nolint:staticcheck
id, _, err = client.Connect().IntentionCreate(&api.Intention{
SourceName: sourceName,
DestinationName: "db",

View File

@ -0,0 +1,61 @@
package intention
import (
"fmt"
"strings"
"github.com/hashicorp/consul/api"
)
// ParseIntentionTarget parses a target of the form <namespace>/<name> and returns
// the two distinct parts. In some cases the namespace may be elided and this function
// will return the empty string for the namespace then.
func ParseIntentionTarget(input string) (name string, namespace string, err error) {
// Get the index to the '/'. If it doesn't exist, we have just a name
// so just set that and return.
idx := strings.IndexByte(input, '/')
if idx == -1 {
// let the agent do token based defaulting of the namespace
return input, "", nil
}
namespace = input[:idx]
name = input[idx+1:]
if strings.IndexByte(name, '/') != -1 {
return "", "", fmt.Errorf("target can contain at most one '/'")
}
return name, namespace, nil
}
func GetFromArgs(client *api.Client, args []string) (*api.Intention, error) {
switch len(args) {
case 1:
id := args[0]
//nolint:staticcheck
ixn, _, err := client.Connect().IntentionGet(id, nil)
if err != nil {
return nil, fmt.Errorf("Error reading the intention: %s", err)
} else if ixn == nil {
return nil, fmt.Errorf("Intention not found with ID %q", id)
}
return ixn, nil
case 2:
source, destination := args[0], args[1]
ixn, _, err := client.Connect().IntentionGetExact(source, destination, nil)
if err != nil {
return nil, fmt.Errorf("Error reading the intention: %s", err)
} else if ixn == nil {
return nil, fmt.Errorf("Intention not found with source %q and destination %q", source, destination)
}
return ixn, nil
default:
return nil, fmt.Errorf("command requires exactly 1 or 2 arguments")
}
}

View File

@ -0,0 +1,61 @@
package intention
import (
"testing"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testrpc"
"github.com/stretchr/testify/require"
)
func TestGetFromArgs(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t, ``)
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create some intentions.
//nolint:staticcheck
id0, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: "a",
DestinationName: "b",
Action: api.IntentionActionAllow,
}, nil)
require.NoError(t, err)
t.Run("l4 intention", func(t *testing.T) {
t.Run("one arg", func(t *testing.T) {
ixn, err := GetFromArgs(client, []string{id0})
require.NoError(t, err)
require.Equal(t, id0, ixn.ID)
require.Equal(t, "a", ixn.SourceName)
require.Equal(t, "b", ixn.DestinationName)
require.Equal(t, api.IntentionActionAllow, ixn.Action)
})
t.Run("two args", func(t *testing.T) {
ixn, err := GetFromArgs(client, []string{"a", "b"})
require.NoError(t, err)
require.Equal(t, id0, ixn.ID)
require.Equal(t, "a", ixn.SourceName)
require.Equal(t, "b", ixn.DestinationName)
require.Equal(t, api.IntentionActionAllow, ixn.Action)
})
})
t.Run("missing intention", func(t *testing.T) {
t.Run("one arg", func(t *testing.T) {
fakeID := "59208cab-b431-422e-87dc-290b18513082"
_, err := GetFromArgs(client, []string{fakeID})
require.Error(t, err)
})
t.Run("two args", func(t *testing.T) {
_, err := GetFromArgs(client, []string{"c", "d"})
require.Error(t, err)
})
})
}

View File

@ -6,18 +6,19 @@ import (
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/testrpc"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestCommand_noTabs(t *testing.T) {
func TestIntentionMatch_noTabs(t *testing.T) {
t.Parallel()
if strings.ContainsRune(New(nil).Help(), '\t') {
t.Fatal("help has tabs")
}
}
func TestCommand_Validation(t *testing.T) {
func TestIntentionMatch_Validation(t *testing.T) {
t.Parallel()
ui := cli.NewMockUi()
@ -64,7 +65,7 @@ func TestCommand_Validation(t *testing.T) {
}
}
func TestCommand_matchDst(t *testing.T) {
func TestIntentionMatch_matchDst(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -72,6 +73,8 @@ func TestCommand_matchDst(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create some intentions
{
insert := [][]string{
@ -81,6 +84,7 @@ func TestCommand_matchDst(t *testing.T) {
}
for _, v := range insert {
//nolint:staticcheck
id, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: v[0],
DestinationName: v[1],
@ -107,7 +111,7 @@ func TestCommand_matchDst(t *testing.T) {
}
}
func TestCommand_matchSource(t *testing.T) {
func TestIntentionMatch_matchSource(t *testing.T) {
t.Parallel()
require := require.New(t)
@ -115,6 +119,8 @@ func TestCommand_matchSource(t *testing.T) {
defer a.Shutdown()
client := a.Client()
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
// Create some intentions
{
insert := [][]string{
@ -124,6 +130,7 @@ func TestCommand_matchSource(t *testing.T) {
}
for _, v := range insert {
//nolint:staticcheck
id, _, err := client.Connect().IntentionCreate(&api.Intention{
SourceName: v[0],
DestinationName: v[1],

View File

@ -173,6 +173,7 @@ func TestServerSideVerifier(t *testing.T) {
SourceType: api.IntentionSourceConsul,
Meta: map[string]string{},
}
//nolint:staticcheck
id, _, err := connect.IntentionCreate(ixn, nil)
require.NoError(t, err)
require.NotEmpty(t, id)
@ -186,6 +187,7 @@ func TestServerSideVerifier(t *testing.T) {
SourceType: api.IntentionSourceConsul,
Meta: map[string]string{},
}
//nolint:staticcheck
id, _, err = connect.IntentionCreate(ixn, nil)
require.NoError(t, err)
require.NotEmpty(t, id)

View File

@ -393,7 +393,7 @@ func (s *TestServer) WaitForLeader(t *testing.T) {
}
defer resp.Body.Close()
if err := s.requireOK(resp); err != nil {
r.Fatal("failed OK response", err)
r.Fatalf("failed OK response: %v", err)
}
// Ensure we have a leader and a node registration.
@ -402,7 +402,7 @@ func (s *TestServer) WaitForLeader(t *testing.T) {
}
index, err := strconv.ParseInt(resp.Header.Get("X-Consul-Index"), 10, 64)
if err != nil {
r.Fatal("bad consul index", err)
r.Fatalf("bad consul index: %v", err)
}
if index < 2 {
r.Fatal("consul index should be at least 2")
@ -433,7 +433,7 @@ func (s *TestServer) WaitForActiveCARoot(t *testing.T) {
// since this is used in both `api` and consul test or duplication. The 200
// is all we really need to wait for.
if err := s.requireOK(resp); err != nil {
r.Fatal("failed OK response", err)
r.Fatalf("failed OK response: %v", err)
}
var roots rootsResponse
@ -449,6 +449,27 @@ func (s *TestServer) WaitForActiveCARoot(t *testing.T) {
})
}
// WaitForServiceIntentions waits until the server can accept config entry
// kinds of service-intentions meaning any migration bootstrapping from pre-1.9
// intentions has completed.
func (s *TestServer) WaitForServiceIntentions(t *testing.T) {
const fakeConfigName = "Sa4ohw5raith4si0Ohwuqu3lowiethoh"
retry.Run(t, func(r *retry.R) {
// Try to delete a non-existent service-intentions config entry. The
// preflightCheck call in agent/consul/config_endpoint.go will fail if
// we aren't ready yet, vs just doing no work instead.
url := s.url("/v1/config/service-intentions/" + fakeConfigName)
resp, err := s.masterDelete(url)
if err != nil {
r.Fatalf("failed http get '%s': %v", url, err)
}
defer resp.Body.Close()
if err := s.requireOK(resp); err != nil {
r.Fatalf("failed OK response: %v", err)
}
})
}
// WaitForSerfCheck ensures we have a node with serfHealth check registered
// Behavior mirrors testrpc.WaitForTestAgent but avoids the dependency cycle in api pkg
func (s *TestServer) WaitForSerfCheck(t *testing.T) {
@ -457,11 +478,11 @@ func (s *TestServer) WaitForSerfCheck(t *testing.T) {
url := s.url("/v1/catalog/nodes?index=0")
resp, err := s.masterGet(url)
if err != nil {
r.Fatal("failed http get", err)
r.Fatalf("failed http get: %v", err)
}
defer resp.Body.Close()
if err := s.requireOK(resp); err != nil {
r.Fatal("failed OK response", err)
r.Fatalf("failed OK response: %v", err)
}
// Watch for the anti-entropy sync to finish.
@ -478,11 +499,11 @@ func (s *TestServer) WaitForSerfCheck(t *testing.T) {
url = s.url(fmt.Sprintf("/v1/health/node/%s", payload[0]["Node"]))
resp, err = s.masterGet(url)
if err != nil {
r.Fatal("failed http get", err)
r.Fatalf("failed http get: %v", err)
}
defer resp.Body.Close()
if err := s.requireOK(resp); err != nil {
r.Fatal("failed OK response", err)
r.Fatalf("failed OK response: %v", err)
}
dec = json.NewDecoder(resp.Body)
if err = dec.Decode(&payload); err != nil {
@ -512,3 +533,14 @@ func (s *TestServer) masterGet(url string) (*http.Response, error) {
}
return s.HTTPClient.Do(req)
}
func (s *TestServer) masterDelete(url string) (*http.Response, error) {
req, err := http.NewRequest("DELETE", url, nil)
if err != nil {
return nil, err
}
if s.Config.ACL.Tokens.Master != "" {
req.Header.Set("x-consul-token", s.Config.ACL.Tokens.Master)
}
return s.HTTPClient.Do(req)
}