Add integration test for central config; fix central config WIP (#5752)

* Add integration test for central config; fix central config WIP

* Add integration test for central config; fix central config WIP

* Set proxy protocol correctly and begin adding upstream support

* Add upstreams to service config cache key and start new notify watcher if they change.

This doesn't update the tests to pass though.

* Fix some merging logic get things working manually with a hack (TODO fix properly)

* Simplification to not allow enabling sidecars centrally - it makes no sense without upstreams anyway

* Test compile again and obvious ones pass. Lots of failures locally not debugged yet but may be flakes. Pushing up to see what CI does

* Fix up service manageer and API test failures

* Remove the enable command since it no longer makes much sense without being able to turn on sidecar proxies centrally

* Remove version.go hack - will make integration test fail until release

* Remove unused code from commands and upstream merge

* Re-bump version to 1.5.0
This commit is contained in:
Paul Banks 2019-05-02 00:39:31 +01:00 committed by Jack Pearkes
parent bda1e80841
commit 078f4cf5bb
36 changed files with 630 additions and 527 deletions

View File

@ -28,9 +28,13 @@ func TestResolvedServiceConfig(t *testing.T) {
require.True(req.AllowStale)
reply := args.Get(2).(*structs.ServiceConfigResponse)
reply.Definition = structs.ServiceDefinition{
ID: "1234",
Name: "foo",
reply.ProxyConfig = map[string]interface{}{
"protocol": "http",
}
reply.UpstreamConfigs = map[string]map[string]interface{}{
"s2": map[string]interface{}{
"protocol": "http",
},
}
reply.QueryMeta.Index = 48

View File

@ -258,22 +258,45 @@ func (c *ConfigEntry) ResolveServiceConfig(args *structs.ServiceConfigRequest, r
}
}
// Resolve the service definition by overlaying the service config onto the global
// proxy config.
definition := structs.ServiceDefinition{
Name: args.Name,
}
if proxyConf != nil {
definition.Proxy = &structs.ConnectProxyConfig{
Config: proxyConf.Config,
reply.Index = index
// Apply the proxy defaults to the sidecar's proxy config
reply.ProxyConfig = proxyConf.Config
if serviceConf != nil && serviceConf.Protocol != "" {
if reply.ProxyConfig == nil {
reply.ProxyConfig = make(map[string]interface{})
}
reply.ProxyConfig["protocol"] = serviceConf.Protocol
}
// Apply the upstream protocols to the upstream configs
for _, upstream := range args.Upstreams {
_, upstreamEntry, err := state.ConfigEntry(ws, structs.ServiceDefaults, upstream)
if err != nil {
return err
}
var upstreamConf *structs.ServiceConfigEntry
var ok bool
if upstreamEntry != nil {
upstreamConf, ok = upstreamEntry.(*structs.ServiceConfigEntry)
if !ok {
return fmt.Errorf("invalid service config type %T", upstreamEntry)
}
}
// Nothing to configure if a protocol hasn't been set.
if upstreamConf == nil || upstreamConf.Protocol == "" {
continue
}
if reply.UpstreamConfigs == nil {
reply.UpstreamConfigs = make(map[string]map[string]interface{})
}
reply.UpstreamConfigs[upstream] = map[string]interface{}{
"protocol": upstreamConf.Protocol,
}
}
if serviceConf != nil {
definition.Name = serviceConf.Name
}
reply.Index = index
reply.Definition = definition
return nil
})
}

View File

@ -6,6 +6,7 @@ import (
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/testrpc"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/require"
@ -659,31 +660,51 @@ func TestConfigEntry_ResolveServiceConfig(t *testing.T) {
Kind: structs.ProxyDefaults,
Name: structs.ProxyConfigGlobal,
Config: map[string]interface{}{
"foo": "bar",
"foo": 1,
},
}))
require.NoError(state.EnsureConfigEntry(2, &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "foo",
Kind: structs.ServiceDefaults,
Name: "foo",
Protocol: "http",
}))
require.NoError(state.EnsureConfigEntry(2, &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "bar",
Protocol: "grpc",
}))
args := structs.ServiceConfigRequest{
Name: "foo",
Datacenter: s1.config.Datacenter,
Upstreams: []string{"bar", "baz"},
}
var out structs.ServiceConfigResponse
require.NoError(msgpackrpc.CallWithCodec(codec, "ConfigEntry.ResolveServiceConfig", &args, &out))
// Hack to fix up the string encoding in the map[string]interface{}.
// msgpackRPC's codec doesn't use RawToString.
var err error
out.ProxyConfig, err = lib.MapWalk(out.ProxyConfig)
require.NoError(err)
for k := range out.UpstreamConfigs {
out.UpstreamConfigs[k], err = lib.MapWalk(out.UpstreamConfigs[k])
require.NoError(err)
}
expected := structs.ServiceDefinition{
Name: "foo",
Proxy: &structs.ConnectProxyConfig{
Config: map[string]interface{}{
"foo": "bar",
expected := structs.ServiceConfigResponse{
ProxyConfig: map[string]interface{}{
"foo": int64(1),
"protocol": "http",
},
UpstreamConfigs: map[string]map[string]interface{}{
"bar": map[string]interface{}{
"protocol": "grpc",
},
},
// Don't know what this is deterministically
QueryMeta: out.QueryMeta,
}
out.Definition.Proxy.Config["foo"] = structs.Uint8ToString(out.Definition.Proxy.Config["foo"].([]uint8))
require.Equal(expected, out.Definition)
require.Equal(expected, out)
}
func TestConfigEntry_ResolveServiceConfig_ACLDeny(t *testing.T) {

View File

@ -93,7 +93,6 @@ func TestReplication_ConfigEntries(t *testing.T) {
require.True(t, ok)
require.Equal(t, remoteSvc.Protocol, localSvc.Protocol)
require.Equal(t, remoteSvc.Connect, localSvc.Connect)
case structs.ProxyDefaults:
localProxy, ok := local[i].(*structs.ProxyConfigEntry)
require.True(t, ok)

View File

@ -7,6 +7,8 @@ import (
"github.com/hashicorp/consul/agent/cache"
cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/hashicorp/consul/agent/structs"
"github.com/imdario/mergo"
"github.com/mitchellh/copystructure"
"golang.org/x/net/context"
)
@ -34,6 +36,12 @@ func NewServiceManager(agent *Agent) *ServiceManager {
// to fetch the merged global defaults that apply to the service in order to compose the
// initial registration.
func (s *ServiceManager) AddService(service *structs.NodeService, chkTypes []*structs.CheckType, persist bool, token string, source configSource) error {
// For now only sidecar proxies have anything that can be configured
// centrally. So bypass the whole manager for regular services.
if !service.IsSidecarProxy() {
return s.agent.addServiceInternal(service, chkTypes, persist, token, source)
}
s.lock.Lock()
defer s.lock.Unlock()
@ -111,7 +119,7 @@ type serviceRegistration struct {
// service/proxy defaults.
type serviceConfigWatch struct {
registration *serviceRegistration
config *structs.ServiceDefinition
defaults *structs.ServiceConfigResponse
agent *Agent
@ -119,10 +127,24 @@ type serviceConfigWatch struct {
// for the resolved service config is received from the cache.
readyCh chan error
updateCh chan cache.UpdateEvent
// ctx and cancelFunc store the overall context that lives as long as the
// Watch instance is needed, possibly spanning multiple cache.Notify
// lifetimes.
ctx context.Context
cancelFunc func()
// cacheKey stores the key of the current request, when registration changes
// we check to see if a new cache watch is needed.
cacheKey string
// updateCh receives changes from cache watchers or registration changes.
updateCh chan cache.UpdateEvent
// notifyCancel, if non-nil it the cancel func that will stop the currently
// active Notify loop. It does not cancel ctx and is used when we need to
// switch to a new Notify call because cache key changed.
notifyCancel func()
lock sync.Mutex
}
@ -130,7 +152,7 @@ type serviceConfigWatch struct {
// the updateCh. This is not safe to call more than once.
func (s *serviceConfigWatch) Start() error {
s.ctx, s.cancelFunc = context.WithCancel(context.Background())
if err := s.startConfigWatch(); err != nil {
if err := s.ensureConfigWatch(); err != nil {
return err
}
go s.runWatch()
@ -194,20 +216,34 @@ func (s *serviceConfigWatch) handleUpdate(event cache.UpdateEvent, locked, first
return fmt.Errorf("error watching service config: %v", event.Err)
}
} else {
switch event.Result.(type) {
switch res := event.Result.(type) {
case *serviceRegistration:
s.registration = event.Result.(*serviceRegistration)
s.registration = res
// We may need to restart watch if upstreams changed
if err := s.ensureConfigWatch(); err != nil {
return err
}
case *structs.ServiceConfigResponse:
resp := event.Result.(*structs.ServiceConfigResponse)
s.config = &resp.Definition
// Sanity check this even came from the currently active watch to ignore
// rare races when switching cache keys
if event.CorrelationID != s.cacheKey {
// It's a no-op. The new watcher will deliver (or may have already
// delivered) the correct config so just ignore this old message.
return nil
}
s.defaults = res
default:
return fmt.Errorf("unknown update event type: %T", event)
}
}
service := s.mergeServiceConfig()
err := s.agent.addServiceInternal(service, s.registration.chkTypes, s.registration.persist, s.registration.token, s.registration.source)
// Merge the local registration with the central defaults and update this service
// in the local state.
service, err := s.mergeServiceConfig()
if err != nil {
return err
}
if err := s.updateAgentRegistration(service); err != nil {
// If this is the initial registration, return the error through the readyCh
// so it can be passed back to the original caller.
if firstRun {
@ -224,20 +260,75 @@ func (s *serviceConfigWatch) handleUpdate(event cache.UpdateEvent, locked, first
return nil
}
// startConfigWatch starts a cache.Notify goroutine to run a continuous blocking query
// on the resolved service config for this service.
func (s *serviceConfigWatch) startConfigWatch() error {
name := s.registration.service.Service
// updateAgentRegistration updates the service (and its sidecar, if applicable) in the
// local state.
func (s *serviceConfigWatch) updateAgentRegistration(ns *structs.NodeService) error {
return s.agent.addServiceInternal(ns, s.registration.chkTypes, s.registration.persist, s.registration.token, s.registration.source)
}
// ensureConfigWatch starts a cache.Notify goroutine to run a continuous
// blocking query on the resolved service config for this service. If the
// registration has changed in a way that requires a new blocking query, it will
// cancel any current watch and start a new one. It is a no-op if there is an
// existing watch that is sufficient for the current registration. It is not
// thread-safe and must only be called from the Start method (which is only safe
// to call once as documented) or from inside the run loop.
func (s *serviceConfigWatch) ensureConfigWatch() error {
ns := s.registration.service
name := ns.Service
var upstreams []string
// Note that only sidecar proxies should even make it here for now although
// later that will change to add the condition.
if ns.IsSidecarProxy() {
// This is a sidecar proxy, ignore the proxy service's config since we are
// managed by the target service config.
name = ns.Proxy.DestinationServiceName
// Also if we have any upstreams defined, add them to the request so we can
// learn about their configs.
for _, us := range ns.Proxy.Upstreams {
if us.DestinationType == "" || us.DestinationType == structs.UpstreamDestTypeService {
upstreams = append(upstreams, us.DestinationName)
}
}
}
req := &structs.ServiceConfigRequest{
Name: name,
Datacenter: s.agent.config.Datacenter,
QueryOptions: structs.QueryOptions{Token: s.agent.config.ACLAgentToken},
Upstreams: upstreams,
}
if s.registration.token != "" {
req.QueryOptions.Token = s.registration.token
}
err := s.agent.cache.Notify(s.ctx, cachetype.ResolvedServiceConfigName, req, fmt.Sprintf("service-config:%s", name), s.updateCh)
// See if this request is different from the current one
cacheKey := req.CacheInfo().Key
if cacheKey == s.cacheKey {
return nil
}
// If there is an existing notify running, stop it first. This may leave a
// blocking query running in the background but the Notify loop will swallow
// the response and exit when it next unblocks so we can consider it stopped.
if s.notifyCancel != nil {
s.notifyCancel()
}
// Make a new context just for this Notify call
ctx, cancel := context.WithCancel(s.ctx)
s.notifyCancel = cancel
s.cacheKey = cacheKey
// We use the cache key as the correlationID here. Notify in general will not
// respond on the updateCh after the context is cancelled however there could
// possible be a race where it has only just got an update and checked the
// context before we cancel and so might still deliver the old event. Using
// the cacheKey allows us to ignore updates from the old cache watch and makes
// even this rare edge case safe.
err := s.agent.cache.Notify(ctx, cachetype.ResolvedServiceConfigName, req,
s.cacheKey, s.updateCh)
return err
}
@ -252,13 +343,40 @@ func (s *serviceConfigWatch) updateRegistration(registration *serviceRegistratio
// mergeServiceConfig returns the final effective config for the watched service,
// including the latest known global defaults from the servers.
func (s *serviceConfigWatch) mergeServiceConfig() *structs.NodeService {
if s.config == nil {
return s.registration.service
func (s *serviceConfigWatch) mergeServiceConfig() (*structs.NodeService, error) {
if s.defaults == nil || !s.registration.service.IsSidecarProxy() {
return s.registration.service, nil
}
svc := s.config.NodeService()
svc.Merge(s.registration.service)
// We don't want to change s.registration in place since it is our source of
// truth about what was actually registered before defaults applied. So copy
// it first.
nsRaw, err := copystructure.Copy(s.registration.service)
if err != nil {
return nil, err
}
return svc
// Merge proxy defaults
ns := nsRaw.(*structs.NodeService)
if err := mergo.Merge(&ns.Proxy.Config, s.defaults.ProxyConfig); err != nil {
return nil, err
}
// Merge upstream defaults if there were any returned
for i := range ns.Proxy.Upstreams {
// Get a pointer not a value copy of the upstream struct
us := &ns.Proxy.Upstreams[i]
if us.DestinationType != "" && us.DestinationType != structs.UpstreamDestTypeService {
continue
}
usCfg, ok := s.defaults.UpstreamConfigs[us.DestinationName]
if !ok {
// No config defaults to merge
continue
}
if err := mergo.Merge(&us.Config, usCfg); err != nil {
return nil, err
}
}
return ns, err
}

View File

@ -16,42 +16,155 @@ func TestServiceManager_RegisterService(t *testing.T) {
testrpc.WaitForLeader(t, a.RPC, "dc1")
// Register some global proxy config
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ProxyConfigEntry{
Config: map[string]interface{}{
"foo": 1,
// Register a global proxy and service config
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ProxyConfigEntry{
Config: map[string]interface{}{
"foo": 1,
},
},
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "redis",
Protocol: "tcp",
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
out := false
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
// Now register a service locally and make sure the resulting State entry
// has the global config in it.
// Now register a service locally with no sidecar, it should be a no-op.
svc := &structs.NodeService{
ID: "redis",
Service: "redis",
Port: 8000,
}
require.NoError(a.AddService(svc, nil, false, "", ConfigSourceLocal))
mergedService := a.State.Service("redis")
require.NotNil(mergedService)
// Verify both the service and sidecar.
redisService := a.State.Service("redis")
require.NotNil(redisService)
require.Equal(&structs.NodeService{
ID: "redis",
Service: "redis",
Port: 8000,
Weights: &structs.Weights{
Passing: 1,
Warning: 1,
},
}, redisService)
}
func TestServiceManager_RegisterSidecar(t *testing.T) {
require := require.New(t)
a := NewTestAgent(t, t.Name(), "enable_central_service_config = true")
defer a.Shutdown()
testrpc.WaitForLeader(t, a.RPC, "dc1")
// Register a global proxy and service config
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ProxyConfigEntry{
Config: map[string]interface{}{
"foo": 1,
},
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "web",
Protocol: "http",
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "redis",
Protocol: "tcp",
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
// Now register a sidecar proxy. Note we don't use SidecarService here because
// that gets resolved earlier in config handling than the AddService call
// here.
svc := &structs.NodeService{
Kind: structs.ServiceKindConnectProxy,
ID: "web-sidecar-proxy",
Service: "web-sidecar-proxy",
Port: 21000,
Proxy: structs.ConnectProxyConfig{
DestinationServiceName: "web",
DestinationServiceID: "web",
LocalServiceAddress: "127.0.0.1",
LocalServicePort: 8000,
Upstreams: structs.Upstreams{
{
DestinationName: "redis",
LocalBindPort: 5000,
},
},
},
}
require.NoError(a.AddService(svc, nil, false, "", ConfigSourceLocal))
// Verify sidecar got global config loaded
sidecarService := a.State.Service("web-sidecar-proxy")
require.NotNil(sidecarService)
require.Equal(&structs.NodeService{
Kind: structs.ServiceKindConnectProxy,
ID: "web-sidecar-proxy",
Service: "web-sidecar-proxy",
Port: 21000,
Proxy: structs.ConnectProxyConfig{
DestinationServiceName: "web",
DestinationServiceID: "web",
LocalServiceAddress: "127.0.0.1",
LocalServicePort: 8000,
Config: map[string]interface{}{
"foo": int64(1),
"foo": int64(1),
"protocol": "http",
},
Upstreams: structs.Upstreams{
{
DestinationName: "redis",
LocalBindPort: 5000,
Config: map[string]interface{}{
"protocol": "tcp",
},
},
},
},
Weights: &structs.Weights{
Passing: 1,
Warning: 1,
},
}, mergedService)
}, sidecarService)
}
func TestServiceManager_Disabled(t *testing.T) {
@ -62,37 +175,92 @@ func TestServiceManager_Disabled(t *testing.T) {
testrpc.WaitForLeader(t, a.RPC, "dc1")
// Register some global proxy config
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ProxyConfigEntry{
Config: map[string]interface{}{
"foo": 1,
// Register a global proxy and service config
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ProxyConfigEntry{
Config: map[string]interface{}{
"foo": 1,
},
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "web",
Protocol: "http",
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
{
args := &structs.ConfigEntryRequest{
Datacenter: "dc1",
Entry: &structs.ServiceConfigEntry{
Kind: structs.ServiceDefaults,
Name: "redis",
Protocol: "tcp",
},
}
var out bool
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
}
// Now register a sidecar proxy. Note we don't use SidecarService here because
// that gets resolved earlier in config handling than the AddService call
// here.
svc := &structs.NodeService{
Kind: structs.ServiceKindConnectProxy,
ID: "web-sidecar-proxy",
Service: "web-sidecar-proxy",
Port: 21000,
Proxy: structs.ConnectProxyConfig{
DestinationServiceName: "web",
DestinationServiceID: "web",
LocalServiceAddress: "127.0.0.1",
LocalServicePort: 8000,
Upstreams: structs.Upstreams{
{
DestinationName: "redis",
LocalBindPort: 5000,
},
},
},
}
out := false
require.NoError(a.RPC("ConfigEntry.Apply", args, &out))
// Now register a service locally and make sure the resulting State entry
// has the global config in it.
svc := &structs.NodeService{
ID: "redis",
Service: "redis",
Port: 8000,
}
require.NoError(a.AddService(svc, nil, false, "", ConfigSourceLocal))
mergedService := a.State.Service("redis")
require.NotNil(mergedService)
// The proxy config map shouldn't be present; the agent should ignore global
// defaults here.
// Verify sidecar got global config loaded
sidecarService := a.State.Service("web-sidecar-proxy")
require.NotNil(sidecarService)
require.Equal(&structs.NodeService{
ID: "redis",
Service: "redis",
Port: 8000,
Kind: structs.ServiceKindConnectProxy,
ID: "web-sidecar-proxy",
Service: "web-sidecar-proxy",
Port: 21000,
Proxy: structs.ConnectProxyConfig{
DestinationServiceName: "web",
DestinationServiceID: "web",
LocalServiceAddress: "127.0.0.1",
LocalServicePort: 8000,
// No config added
Upstreams: structs.Upstreams{
{
DestinationName: "redis",
LocalBindPort: 5000,
// No config added
},
},
},
Weights: &structs.Weights{
Passing: 1,
Warning: 1,
},
}, mergedService)
}, sidecarService)
}

View File

@ -46,7 +46,11 @@ type ServiceConfigEntry struct {
Kind string
Name string
Protocol string
Connect ConnectConfiguration
// TODO(banks): enable this once we have upstreams supported too. Enabling
// sidecars actually makes no sense and adds complications when you don't
// allow upstreams to be specified centrally too.
//
// Connect ConnectConfiguration
RaftIndex
}
@ -368,6 +372,7 @@ func (c *ConfigEntryQuery) RequestDatacenter() string {
type ServiceConfigRequest struct {
Name string
Datacenter string
Upstreams []string
QueryOptions
}
@ -386,10 +391,18 @@ func (r *ServiceConfigRequest) CacheInfo() cache.RequestInfo {
MustRevalidate: r.MustRevalidate,
}
// To calculate the cache key we only hash the service name. The
// datacenter is handled by the cache framework. The other fields are
// not, but should not be used in any cache types.
v, err := hashstructure.Hash(r.Name, nil)
// To calculate the cache key we only hash the service name and upstream set.
// We don't want ordering of the upstreams to affect the outcome so use an
// anonymous struct field with hash:set behavior. Note the order of fields in
// the slice would affect cache keys if we ever persist between agent restarts
// and change it.
v, err := hashstructure.Hash(struct {
Name string
Upstreams []string `hash:"set"`
}{
Name: r.Name,
Upstreams: r.Upstreams,
}, nil)
if err == nil {
// If there is an error, we don't set the key. A blank key forces
// no cache for this request so the request is forwarded directly
@ -401,8 +414,8 @@ func (r *ServiceConfigRequest) CacheInfo() cache.RequestInfo {
}
type ServiceConfigResponse struct {
Definition ServiceDefinition
ProxyConfig map[string]interface{}
UpstreamConfigs map[string]map[string]interface{}
QueryMeta
}

View File

@ -62,7 +62,7 @@ func TestDecodeConfigEntry(t *testing.T) {
Kind: ServiceDefaults,
Name: "foo",
Protocol: "tcp",
Connect: ConnectConfiguration{SidecarProxy: true},
//Connect: ConnectConfiguration{SidecarProxy: true},
},
},
"service-defaults translations": tcase{
@ -78,7 +78,7 @@ func TestDecodeConfigEntry(t *testing.T) {
Kind: ServiceDefaults,
Name: "foo",
Protocol: "tcp",
Connect: ConnectConfiguration{SidecarProxy: true},
//Connect: ConnectConfiguration{SidecarProxy: true},
},
},
}

View File

@ -12,13 +12,14 @@ import (
"strings"
"time"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-msgpack/codec"
multierror "github.com/hashicorp/go-multierror"
"github.com/hashicorp/serf/coordinate"
"github.com/mitchellh/hashstructure"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/types"
)
type MessageType uint8
@ -771,76 +772,9 @@ type ServiceConnect struct {
SidecarService *ServiceDefinition `json:",omitempty" bexpr:"-"`
}
// Merge overlays any non-empty fields of other onto s. Tags, metadata and proxy
// config are unioned together instead of overwritten. The Connect field and the
// non-config proxy fields are taken from other.
func (s *NodeService) Merge(other *NodeService) {
if other.Kind != "" {
s.Kind = other.Kind
}
if other.ID != "" {
s.ID = other.ID
}
if other.Service != "" {
s.Service = other.Service
}
if s.Tags == nil {
s.Tags = other.Tags
} else if other.Tags != nil {
// Both nodes have tags, so deduplicate and merge them.
tagSet := make(map[string]struct{})
for _, tag := range s.Tags {
tagSet[tag] = struct{}{}
}
for _, tag := range other.Tags {
tagSet[tag] = struct{}{}
}
tags := make([]string, 0, len(tagSet))
for tag, _ := range tagSet {
tags = append(tags, tag)
}
sort.Strings(tags)
s.Tags = tags
}
if other.Address != "" {
s.Address = other.Address
}
if s.Meta == nil {
s.Meta = other.Meta
} else {
for k, v := range other.Meta {
s.Meta[k] = v
}
}
if other.Port != 0 {
s.Port = other.Port
}
if other.Weights != nil {
s.Weights = other.Weights
}
s.EnableTagOverride = other.EnableTagOverride
if other.ProxyDestination != "" {
s.ProxyDestination = other.ProxyDestination
}
// Take the incoming service's proxy fields and merge the config map.
proxyConf := s.Proxy.Config
s.Proxy = other.Proxy
if proxyConf == nil {
proxyConf = other.Proxy.Config
} else {
for k, v := range other.Proxy.Config {
proxyConf[k] = v
}
}
s.Proxy.Config = proxyConf
// Just take the entire Connect block from the other node.
// We can revisit this when adding more fields to centralized config.
s.Connect = other.Connect
s.LocallyRegisteredAsSidecar = other.LocallyRegisteredAsSidecar
// IsSidecarProxy returns true if the NodeService is a sidecar proxy.
func (s *NodeService) IsSidecarProxy() bool {
return s.Kind == ServiceKindConnectProxy && s.Proxy.DestinationServiceID != ""
}
// Validate validates the node service configuration.

View File

@ -561,103 +561,6 @@ func TestStructs_NodeService_IsSame(t *testing.T) {
}
}
func TestStructs_NodeService_Merge(t *testing.T) {
a := &NodeService{
Kind: "service",
ID: "foo:1",
Service: "foo",
Tags: []string{"a", "b"},
Address: "127.0.0.1",
Meta: map[string]string{"a": "b"},
Port: 1234,
Weights: &Weights{
Passing: 1,
Warning: 1,
},
EnableTagOverride: false,
ProxyDestination: "asdf",
Proxy: ConnectProxyConfig{
DestinationServiceName: "baz",
DestinationServiceID: "baz:1",
LocalServiceAddress: "127.0.0.1",
LocalServicePort: 2345,
Config: map[string]interface{}{
"foo": 1,
},
},
Connect: ServiceConnect{
Native: false,
},
LocallyRegisteredAsSidecar: false,
}
b := &NodeService{
Kind: "other",
ID: "bar:1",
Service: "bar",
Tags: []string{"c", "d"},
Address: "127.0.0.2",
Meta: map[string]string{"c": "d"},
Port: 4567,
Weights: &Weights{
Passing: 2,
Warning: 2,
},
EnableTagOverride: true,
ProxyDestination: "qwer",
Proxy: ConnectProxyConfig{
DestinationServiceName: "zoo",
DestinationServiceID: "zoo:1",
LocalServiceAddress: "127.0.0.2",
LocalServicePort: 6789,
Config: map[string]interface{}{
"bar": 2,
},
},
Connect: ServiceConnect{
Native: true,
},
LocallyRegisteredAsSidecar: true,
}
expected := &NodeService{
Kind: "other",
ID: "bar:1",
Service: "bar",
Tags: []string{"a", "b", "c", "d"},
Address: "127.0.0.2",
Meta: map[string]string{
"a": "b",
"c": "d",
},
Port: 4567,
Weights: &Weights{
Passing: 2,
Warning: 2,
},
EnableTagOverride: true,
ProxyDestination: "qwer",
Proxy: ConnectProxyConfig{
DestinationServiceName: "zoo",
DestinationServiceID: "zoo:1",
LocalServiceAddress: "127.0.0.2",
LocalServicePort: 6789,
Config: map[string]interface{}{
"foo": 1,
"bar": 2,
},
},
Connect: ServiceConnect{
Native: true,
},
LocallyRegisteredAsSidecar: true,
}
a.Merge(b)
require.Equal(t, expected, a)
}
func TestStructs_HealthCheck_IsSame(t *testing.T) {
hc := &HealthCheck{
Node: "node1",

View File

@ -24,15 +24,10 @@ type ConfigEntry interface {
GetModifyIndex() uint64
}
type ConnectConfiguration struct {
SidecarProxy bool
}
type ServiceConfigEntry struct {
Kind string
Name string
Protocol string
Connect ConnectConfiguration
CreateIndex uint64
ModifyIndex uint64
}

View File

@ -139,7 +139,7 @@ func TestAPI_ConfigEntries(t *testing.T) {
require.True(t, written)
// update no cas
service.Connect.SidecarProxy = true
service.Protocol = "http"
_, wm, err = config_entries.Set(service, nil)
require.NoError(t, err)
@ -156,14 +156,13 @@ func TestAPI_ConfigEntries(t *testing.T) {
for _, entry = range entries {
switch entry.GetName() {
case "foo":
// this also verfies that the update value was persisted and
// this also verifies that the update value was persisted and
// the updated values are seen
readService, ok = entry.(*ServiceConfigEntry)
require.True(t, ok)
require.Equal(t, service.Kind, readService.Kind)
require.Equal(t, service.Name, readService.Name)
require.Equal(t, service.Protocol, readService.Protocol)
require.Equal(t, service.Connect.SidecarProxy, readService.Connect.SidecarProxy)
case "bar":
readService, ok = entry.(*ServiceConfigEntry)
require.True(t, ok)

View File

@ -50,7 +50,6 @@ import (
"github.com/hashicorp/consul/command/connect/ca"
caget "github.com/hashicorp/consul/command/connect/ca/get"
caset "github.com/hashicorp/consul/command/connect/ca/set"
connectenable "github.com/hashicorp/consul/command/connect/enable"
"github.com/hashicorp/consul/command/connect/envoy"
"github.com/hashicorp/consul/command/connect/proxy"
"github.com/hashicorp/consul/command/debug"
@ -166,7 +165,6 @@ func init() {
Register("connect ca", func(ui cli.Ui) (cli.Command, error) { return ca.New(), nil })
Register("connect ca get-config", func(ui cli.Ui) (cli.Command, error) { return caget.New(ui), nil })
Register("connect ca set-config", func(ui cli.Ui) (cli.Command, error) { return caset.New(ui), nil })
Register("connect enable", func(ui cli.Ui) (cli.Command, error) { return connectenable.New(ui), nil })
Register("connect proxy", func(ui cli.Ui) (cli.Command, error) { return proxy.New(ui, MakeShutdownCh()), nil })
Register("connect envoy", func(ui cli.Ui) (cli.Command, error) { return envoy.New(ui), nil })
Register("debug", func(ui cli.Ui) (cli.Command, error) { return debug.New(ui, MakeShutdownCh()), nil })

View File

@ -1,101 +0,0 @@
package enable
import (
"flag"
"fmt"
"io"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/command/flags"
"github.com/mitchellh/cli"
)
func New(ui cli.Ui) *cmd {
c := &cmd{UI: ui}
c.init()
return c
}
type cmd struct {
UI cli.Ui
flags *flag.FlagSet
http *flags.HTTPFlags
help string
service string
protocol string
sidecarProxy bool
testStdin io.Reader
}
func (c *cmd) init() {
c.flags = flag.NewFlagSet("", flag.ContinueOnError)
c.http = &flags.HTTPFlags{}
c.flags.BoolVar(&c.sidecarProxy, "sidecar-proxy", false, "Whether the service should have a Sidecar Proxy by default")
c.flags.StringVar(&c.service, "service", "", "The service to enable connect for")
c.flags.StringVar(&c.protocol, "protocol", "", "The protocol spoken by the service")
flags.Merge(c.flags, c.http.ClientFlags())
flags.Merge(c.flags, c.http.ServerFlags())
c.help = flags.Usage(help, c.flags)
}
func (c *cmd) Run(args []string) int {
if err := c.flags.Parse(args); err != nil {
return 1
}
if c.service == "" {
c.UI.Error("Must specify the -service parameter")
return 1
}
entry := &api.ServiceConfigEntry{
Kind: api.ServiceDefaults,
Name: c.service,
Protocol: c.protocol,
Connect: api.ConnectConfiguration{
SidecarProxy: c.sidecarProxy,
},
}
client, err := c.http.APIClient()
if err != nil {
c.UI.Error(fmt.Sprintf("Error connect to Consul agent: %s", err))
return 1
}
written, _, err := client.ConfigEntries().Set(entry, nil)
if err != nil {
c.UI.Error(fmt.Sprintf("Error writing config entry %q / %q: %v", entry.GetKind(), entry.GetName(), err))
return 1
}
if !written {
c.UI.Error(fmt.Sprintf("Config entry %q / %q not updated", entry.GetKind(), entry.GetName()))
return 1
}
// TODO (mkeeler) should we output anything when successful
return 0
}
func (c *cmd) Synopsis() string {
return synopsis
}
func (c *cmd) Help() string {
return flags.Usage(c.help, nil)
}
const synopsis = "Sets some simple Connect related configuration for a service"
const help = `
Usage: consul connect enable -service <service name> [options]
Sets up some Connect related service defaults.
Example:
$ consul connect enable -service web -protocol http -sidecar-proxy true
`

View File

@ -1,64 +0,0 @@
package enable
import (
"testing"
"github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/api"
"github.com/mitchellh/cli"
"github.com/stretchr/testify/require"
)
func TestConnectEnable_noTabs(t *testing.T) {
t.Parallel()
require.NotContains(t, New(cli.NewMockUi()).Help(), "\t")
}
func TestConnectEnable(t *testing.T) {
t.Parallel()
a := agent.NewTestAgent(t, t.Name(), ``)
defer a.Shutdown()
client := a.Client()
ui := cli.NewMockUi()
c := New(ui)
args := []string{
"-http-addr=" + a.HTTPAddr(),
"-service=web",
"-protocol=tcp",
"-sidecar-proxy=true",
}
code := c.Run(args)
require.Equal(t, 0, code)
entry, _, err := client.ConfigEntries().Get(api.ServiceDefaults, "web", nil)
require.NoError(t, err)
svc, ok := entry.(*api.ServiceConfigEntry)
require.True(t, ok)
require.Equal(t, api.ServiceDefaults, svc.Kind)
require.Equal(t, "web", svc.Name)
require.Equal(t, "tcp", svc.Protocol)
require.True(t, svc.Connect.SidecarProxy)
}
func TestConnectEnable_InvalidArgs(t *testing.T) {
t.Parallel()
cases := map[string][]string{
"no service": []string{},
}
for name, tcase := range cases {
t.Run(name, func(t *testing.T) {
ui := cli.NewMockUi()
c := New(ui)
require.NotEqual(t, 0, c.Run(tcase))
require.NotEmpty(t, ui.ErrorWriter.String())
})
}
}

1
go.mod
View File

@ -81,6 +81,7 @@ require (
github.com/hashicorp/vault v0.10.3
github.com/hashicorp/vault-plugin-secrets-kv v0.0.0-20190318174639-195e0e9d07f1 // indirect
github.com/hashicorp/yamux v0.0.0-20180604194846-3520598351bb
github.com/imdario/mergo v0.3.6
github.com/jefferai/jsonx v0.0.0-20160721235117-9cc31c3135ee // indirect
github.com/keybase/go-crypto v0.0.0-20180614160407-5114a9a81e1b // indirect
github.com/kr/pretty v0.1.0 // indirect

View File

@ -5,13 +5,7 @@ set -euo pipefail
# Setup deny intention
docker_consul intention create -deny s1 s2
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -0,0 +1,26 @@
enable_central_service_config = true
config_entries {
bootstrap {
kind = "proxy-defaults"
name = "global"
config {
envoy_prometheus_bind_addr = "0.0.0.0:1234"
}
}
bootstrap {
kind = "service-defaults"
name = "s1"
protocol = "http"
connect {
sidecar_proxy = true
}
}
bootstrap {
kind = "service-defaults"
name = "s2"
protocol = "http"
connect {
sidecar_proxy = true
}
}
}

View File

@ -0,0 +1,16 @@
services {
name = "s1"
port = 8080
connect {
sidecar_service {
proxy {
upstreams = [
{
destination_name = "s2"
local_bind_port = 5000
}
]
}
}
}
}

View File

@ -0,0 +1,17 @@
services {
name = "s2"
port = 8181
connect {
sidecar_service {
proxy {
config {
# We need to override this because both proxies run in same network
# namespace and so it's non-deterministic which one manages to bind
# the 1234 port first. This forces the issue here while still testing
# that s1's proxy is configured from global config.
envoy_prometheus_bind_addr = "0.0.0.0:2345"
}
}
}
}
}

View File

@ -0,0 +1,9 @@
#!/bin/bash
set -euo pipefail
# retry because resolving the central config might race
retry_default gen_envoy_bootstrap s1 19000
retry_default gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -0,0 +1,49 @@
#!/usr/bin/env bats
load helpers
@test "s1 proxy admin is up on :19000" {
retry_default curl -f -s localhost:19000/stats -o /dev/null
}
@test "s2 proxy admin is up on :19001" {
retry_default curl -f -s localhost:19001/stats -o /dev/null
}
@test "s1 proxy listener should be up and have right cert" {
assert_proxy_presents_cert_uri localhost:21000 s1
}
@test "s2 proxy listener should be up and have right cert" {
assert_proxy_presents_cert_uri localhost:21001 s2
}
@test "s1 upstream should be able to connect to s2 with http/1.1" {
run retry_default curl --http1.1 -s -f -d hello localhost:5000
[ "$status" -eq 0 ]
[ "$output" = "hello" ]
}
@test "s1 proxy should be exposing metrics to prometheus from central config" {
# Should have http metrics. This is just a sample one. Require the metric to
# be present not just found in a comment (anchor the regexp).
retry_default \
must_match_in_prometheus_response localhost:1234 \
'^envoy_http_downstream_rq_active'
# Should be labelling with local_cluster.
retry_default \
must_match_in_prometheus_response localhost:1234 \
'[\{,]local_cluster="s1"[,}] '
# Ensure we have http metrics for public listener
retry_default \
must_match_in_prometheus_response localhost:1234 \
'[\{,]envoy_http_conn_manager_prefix="public_listener_http"[,}]'
# Ensure we have http metrics for s2 upstream
retry_default \
must_match_in_prometheus_response localhost:1234 \
'[\{,]envoy_http_conn_manager_prefix="upstream_s2_http"[,}]'
}

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy fake-statsd"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy fake-statsd"

View File

@ -5,13 +5,7 @@ set -euo pipefail
# Setup deny intention
docker_consul intention create -deny s1 s2
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy"

View File

@ -28,17 +28,17 @@ load helpers
@test "s1 proxy should be exposing metrics to prometheus" {
# Should have http metrics. This is just a sample one. Require the metric to
# be present not just found in a comment (anchor the regexp).
run retry_defaults \
retry_default \
must_match_in_prometheus_response localhost:1234 \
'^envoy_http_downstream_rq_active'
# Should be labelling with local_cluster.
run retry_defaults \
retry_default \
must_match_in_prometheus_response localhost:1234 \
'[\{,]local_cluster="s1"[,}] '
# Should be labelling with http listener prefix.
run retry_defaults \
retry_default \
must_match_in_prometheus_response localhost:1234 \
'[\{,]envoy_http_conn_manager_prefix="public_listener_http"[,}]'
}

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy fake-statsd"

View File

@ -2,13 +2,7 @@
set -euo pipefail
docker_consul connect envoy -bootstrap \
-proxy-id s1-sidecar-proxy \
> workdir/envoy/s1-bootstrap.json
docker_consul connect envoy -bootstrap \
-proxy-id s2-sidecar-proxy \
-admin-bind 127.0.0.1:19001 \
> workdir/envoy/s2-bootstrap.json
gen_envoy_bootstrap s1 19000
gen_envoy_bootstrap s2 19001
export REQUIRED_SERVICES="s1 s1-sidecar-proxy s2 s2-sidecar-proxy jaeger"

View File

@ -95,6 +95,8 @@ function must_match_in_prometheus_response {
run curl -f -s $1/metrics
COUNT=$( echo "$output" | grep -Ec $2 )
echo "OUTPUT head -n 10"
echo "$output" | head -n 10
echo "COUNT of '$2' matches: $COUNT"
[ "$status" == 0 ]
@ -129,4 +131,23 @@ function must_fail_http_connection {
# Should fail request with 503
echo "$output" | grep '503 Service Unavailable'
}
function gen_envoy_bootstrap {
SERVICE=$1
ADMIN_PORT=$2
if output=$(docker_consul connect envoy -bootstrap \
-proxy-id $SERVICE-sidecar-proxy \
-admin-bind 0.0.0.0:$ADMIN_PORT 2>&1); then
# All OK, write config to file
echo "$output" > workdir/envoy/$SERVICE-bootstrap.json
else
status=$?
# Command failed, instead of swallowing error (printed on stdout by docker
# it seems) by writing it to file, echo it
echo "$output"
return $status
fi
}

View File

@ -36,24 +36,36 @@ FILTER_TESTS=${FILTER_TESTS:-}
LEAVE_CONSUL_UP=${LEAVE_CONSUL_UP:-}
PROXY_LOGS_ON_FAIL=${PROXY_LOGS_ON_FAIL:-}
mkdir -p workdir/{consul,envoy,bats,statsd,logs}
source helpers.bash
RESULT=1
CLEANED_UP=0
PREV_CMD=""
THIS_CMD=""
function cleanup {
local STATUS="$?"
local CMD="$THIS_CMD"
if [ "$CLEANED_UP" != 0 ] ; then
return
fi
CLEANED_UP=1
# We failed due to set -e catching an error, output some useful info about
# that error.
echo "ERR: command exited with $STATUS"
echo " command: $CMD"
if [ -z "$LEAVE_CONSUL_UP" ] ; then
docker-compose down
fi
}
trap cleanup EXIT
# Magic to capture commands and statuses so we can show them when we exit due to
# set -e This is useful for debugging setup.sh failures.
trap 'PREV_CMD=$THIS_CMD; THIS_CMD=$BASH_COMMAND' DEBUG
# Start the volume container
docker-compose up -d workdir
@ -73,6 +85,8 @@ for c in ./case-*/ ; do
# Wipe state
docker-compose up wipe-volumes
rm -rf workdir/*
mkdir -p workdir/{consul,envoy,bats,statsd,logs}
# Reload consul config from defaults
cp consul-base-cfg/* workdir/consul

2
vendor/modules.txt vendored
View File

@ -120,8 +120,8 @@ github.com/envoyproxy/go-control-plane/envoy/config/filter/network/http_connecti
github.com/envoyproxy/go-control-plane/envoy/config/filter/network/tcp_proxy/v2
github.com/envoyproxy/go-control-plane/envoy/service/auth/v2alpha
github.com/envoyproxy/go-control-plane/envoy/service/discovery/v2
github.com/envoyproxy/go-control-plane/pkg/util
github.com/envoyproxy/go-control-plane/envoy/type
github.com/envoyproxy/go-control-plane/pkg/util
github.com/envoyproxy/go-control-plane/envoy/config/filter/accesslog/v2
# github.com/fatih/color v1.7.0
github.com/fatih/color

View File

@ -15,7 +15,7 @@ var (
//
// Version must conform to the format expected by github.com/hashicorp/go-version
// for tests to work.
Version = "1.4.4"
Version = "1.5.0"
// A pre-release marker for the version. If this is "" (empty string)
// then it means that it is a final release. Otherwise, this is a pre-release