consul: plubming for specifying consul namespace in job/group

This PR adds the common OSS changes for adding support for Consul Namespaces,
which is going to be a Nomad Enterprise feature. There is no new functionality
provided by this changeset and hopefully no new bugs.
This commit is contained in:
Seth Hoenig
2021-03-16 13:22:21 -05:00
parent 5c3399853d
commit a97254fa20
73 changed files with 2078 additions and 529 deletions

View File

@@ -312,8 +312,9 @@ func TestTaskRunner_ConnectNativeHook_Ok(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -376,8 +377,9 @@ func TestTaskRunner_ConnectNativeHook_with_SI_token(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -452,8 +454,9 @@ func TestTaskRunner_ConnectNativeHook_shareTLS(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -571,8 +574,9 @@ func TestTaskRunner_ConnectNativeHook_shareTLS_override(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))

View File

@@ -49,9 +49,10 @@ func newConsulTransportConfig(consul *config.ConsulConfig) consulTransportConfig
}
type envoyBootstrapHookConfig struct {
consul consulTransportConfig
alloc *structs.Allocation
logger hclog.Logger
alloc *structs.Allocation
consul consulTransportConfig
consulNamespace string
logger hclog.Logger
}
func decodeTriState(b *bool) string {
@@ -65,11 +66,12 @@ func decodeTriState(b *bool) string {
}
}
func newEnvoyBootstrapHookConfig(alloc *structs.Allocation, consul *config.ConsulConfig, logger hclog.Logger) *envoyBootstrapHookConfig {
func newEnvoyBootstrapHookConfig(alloc *structs.Allocation, consul *config.ConsulConfig, consulNamespace string, logger hclog.Logger) *envoyBootstrapHookConfig {
return &envoyBootstrapHookConfig{
alloc: alloc,
logger: logger,
consul: newConsulTransportConfig(consul),
alloc: alloc,
consul: newConsulTransportConfig(consul),
consulNamespace: consulNamespace,
logger: logger,
}
}
@@ -95,18 +97,36 @@ type envoyBootstrapHook struct {
// before contacting Consul.
consulConfig consulTransportConfig
// consulNamespace is the Consul namespace as set by in the job
consulNamespace string
// logger is used to log things
logger hclog.Logger
}
func newEnvoyBootstrapHook(c *envoyBootstrapHookConfig) *envoyBootstrapHook {
return &envoyBootstrapHook{
alloc: c.alloc,
consulConfig: c.consul,
logger: c.logger.Named(envoyBootstrapHookName),
alloc: c.alloc,
consulConfig: c.consul,
consulNamespace: c.consulNamespace,
logger: c.logger.Named(envoyBootstrapHookName),
}
}
// getConsulNamespace will resolve the Consul namespace, choosing between
// - agent config (low precedence)
// - task group config (high precedence)
func (h *envoyBootstrapHook) getConsulNamespace() string {
var namespace string
if h.consulConfig.Namespace != "" {
namespace = h.consulConfig.Namespace
}
if h.consulNamespace != "" {
namespace = h.consulNamespace
}
return namespace
}
func (envoyBootstrapHook) Name() string {
return envoyBootstrapHookName
}
@@ -355,8 +375,11 @@ func (h *envoyBootstrapHook) newEnvoyBootstrapArgs(
sidecarForID string // sidecar only
gateway string // gateway only
proxyID string // gateway only
namespace string
)
namespace = h.getConsulNamespace()
switch {
case service.Connect.HasSidecar():
sidecarForID = h.proxyServiceID(group, service)
@@ -372,7 +395,7 @@ func (h *envoyBootstrapHook) newEnvoyBootstrapArgs(
"sidecar_for", service.Name, "bootstrap_file", filepath,
"sidecar_for_id", sidecarForID, "grpc_addr", grpcAddr,
"admin_bind", envoyAdminBind, "gateway", gateway,
"proxy_id", proxyID,
"proxy_id", proxyID, "namespace", namespace,
)
return envoyBootstrapArgs{
@@ -383,6 +406,7 @@ func (h *envoyBootstrapHook) newEnvoyBootstrapArgs(
siToken: siToken,
gateway: gateway,
proxyID: proxyID,
namespace: namespace,
}
}
@@ -397,6 +421,7 @@ type envoyBootstrapArgs struct {
siToken string
gateway string // gateways only
proxyID string // gateways only
namespace string
}
// args returns the CLI arguments consul needs in the correct order, with the
@@ -439,7 +464,7 @@ func (e envoyBootstrapArgs) args() []string {
arguments = append(arguments, "-client-key", v)
}
if v := e.consulConfig.Namespace; v != "" {
if v := e.namespace; v != "" {
arguments = append(arguments, "-namespace", v)
}
@@ -462,7 +487,7 @@ func (e envoyBootstrapArgs) env(env []string) []string {
if v := e.consulConfig.VerifySSL; v != "" {
env = append(env, fmt.Sprintf("%s=%s", "CONSUL_HTTP_SSL_VERIFY", v))
}
if v := e.consulConfig.Namespace; v != "" {
if v := e.namespace; v != "" {
env = append(env, fmt.Sprintf("%s=%s", "CONSUL_NAMESPACE", v))
}
return env

View File

@@ -32,6 +32,12 @@ import (
var _ interfaces.TaskPrestartHook = (*envoyBootstrapHook)(nil)
const (
// consulNamespace is empty string in OSS, because Consul OSS does not like
// having even the default namespace set.
consulNamespace = ""
)
func writeTmp(t *testing.T, s string, fm os.FileMode) string {
dir, err := ioutil.TempDir("", "envoy-")
require.NoError(t, err)
@@ -299,8 +305,9 @@ func TestEnvoyBootstrapHook_with_SI_token(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -308,7 +315,7 @@ func TestEnvoyBootstrapHook_with_SI_token(t *testing.T) {
// Run Connect bootstrap Hook
h := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, &config.ConsulConfig{
Addr: consulConfig.Address,
}, logger))
}, consulNamespace, logger))
req := &interfaces.TaskPrestartRequest{
Task: sidecarTask,
TaskDir: allocDir.NewTaskDir(sidecarTask.Name),
@@ -399,8 +406,9 @@ func TestTaskRunner_EnvoyBootstrapHook_sidecar_ok(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
consulClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
require.NoError(t, consulClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -408,7 +416,7 @@ func TestTaskRunner_EnvoyBootstrapHook_sidecar_ok(t *testing.T) {
// Run Connect bootstrap Hook
h := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, &config.ConsulConfig{
Addr: consulConfig.Address,
}, logger))
}, consulNamespace, logger))
req := &interfaces.TaskPrestartRequest{
Task: sidecarTask,
TaskDir: allocDir.NewTaskDir(sidecarTask.Name),
@@ -463,9 +471,10 @@ func TestTaskRunner_EnvoyBootstrapHook_gateway_ok(t *testing.T) {
consulConfig.Address = testConsul.HTTPAddr
consulAPIClient, err := consulapi.NewClient(consulConfig)
require.NoError(t, err)
namespacesClient := agentconsul.NewNamespacesClient(consulAPIClient.Namespaces())
// Register Group Services
serviceClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), logger, true)
serviceClient := agentconsul.NewServiceClient(consulAPIClient.Agent(), namespacesClient, logger, true)
go serviceClient.Run()
defer serviceClient.Shutdown()
require.NoError(t, serviceClient.RegisterWorkload(agentconsul.BuildAllocServices(mock.Node(), alloc, agentconsul.NoopRestarter())))
@@ -489,7 +498,7 @@ func TestTaskRunner_EnvoyBootstrapHook_gateway_ok(t *testing.T) {
// Run Connect bootstrap hook
h := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, &config.ConsulConfig{
Addr: consulConfig.Address,
}, logger))
}, consulNamespace, logger))
req := &interfaces.TaskPrestartRequest{
Task: alloc.Job.TaskGroups[0].Tasks[0],
@@ -542,7 +551,7 @@ func TestTaskRunner_EnvoyBootstrapHook_Noop(t *testing.T) {
// not get hit.
h := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, &config.ConsulConfig{
Addr: "http://127.0.0.2:1",
}, logger))
}, consulNamespace, logger))
req := &interfaces.TaskPrestartRequest{
Task: task,
TaskDir: allocDir.NewTaskDir(task.Name),
@@ -615,7 +624,7 @@ func TestTaskRunner_EnvoyBootstrapHook_RecoverableError(t *testing.T) {
// Run Connect bootstrap Hook
h := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, &config.ConsulConfig{
Addr: testConsul.HTTPAddr,
}, logger))
}, consulNamespace, logger))
req := &interfaces.TaskPrestartRequest{
Task: sidecarTask,
TaskDir: allocDir.NewTaskDir(sidecarTask.Name),
@@ -677,12 +686,14 @@ func TestTaskRunner_EnvoyBootstrapHook_grpcAddress(t *testing.T) {
bridgeH := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(
mock.ConnectIngressGatewayAlloc("bridge"),
new(config.ConsulConfig),
consulNamespace,
testlog.HCLogger(t),
))
hostH := newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(
mock.ConnectIngressGatewayAlloc("host"),
new(config.ConsulConfig),
consulNamespace,
testlog.HCLogger(t),
))

View File

@@ -34,12 +34,13 @@ type scriptCheckHookConfig struct {
// scriptCheckHook implements a task runner hook for running script
// checks in the context of a task
type scriptCheckHook struct {
consul consul.ConsulServiceAPI
alloc *structs.Allocation
task *structs.Task
logger log.Logger
shutdownWait time.Duration // max time to wait for scripts to shutdown
shutdownCh chan struct{} // closed when all scripts should shutdown
consul consul.ConsulServiceAPI
consulNamespace string
alloc *structs.Allocation
task *structs.Task
logger log.Logger
shutdownWait time.Duration // max time to wait for scripts to shutdown
shutdownCh chan struct{} // closed when all scripts should shutdown
// The following fields can be changed by Update()
driverExec tinterfaces.ScriptExecutor
@@ -59,13 +60,14 @@ type scriptCheckHook struct {
// in Poststart() or Update()
func newScriptCheckHook(c scriptCheckHookConfig) *scriptCheckHook {
h := &scriptCheckHook{
consul: c.consul,
alloc: c.alloc,
task: c.task,
scripts: make(map[string]*scriptCheck),
runningScripts: make(map[string]*taskletHandle),
shutdownWait: defaultShutdownWait,
shutdownCh: make(chan struct{}),
consul: c.consul,
consulNamespace: c.alloc.Job.LookupTaskGroup(c.alloc.TaskGroup).Consul.GetNamespace(),
alloc: c.alloc,
task: c.task,
scripts: make(map[string]*scriptCheck),
runningScripts: make(map[string]*taskletHandle),
shutdownWait: defaultShutdownWait,
shutdownCh: make(chan struct{}),
}
if c.shutdownWait != 0 {
@@ -183,11 +185,12 @@ func (h *scriptCheckHook) newScriptChecks() map[string]*scriptCheck {
serviceID := agentconsul.MakeAllocServiceID(
h.alloc.ID, h.task.Name, service)
sc := newScriptCheck(&scriptCheckConfig{
namespace: h.consulNamespace,
allocID: h.alloc.ID,
taskName: h.task.Name,
check: check,
serviceID: serviceID,
agent: h.consul,
ttlUpdater: h.consul,
driverExec: h.driverExec,
taskEnv: h.taskEnv,
logger: h.logger,
@@ -226,7 +229,7 @@ func (h *scriptCheckHook) newScriptChecks() map[string]*scriptCheck {
taskName: groupTaskName,
check: check,
serviceID: serviceID,
agent: h.consul,
ttlUpdater: h.consul,
driverExec: h.driverExec,
taskEnv: h.taskEnv,
logger: h.logger,
@@ -255,19 +258,20 @@ func (*scriptCheckHook) associated(task, serviceTask, checkTask string) bool {
return false
}
// heartbeater is the subset of consul agent functionality needed by script
// TTLUpdater is the subset of consul agent functionality needed by script
// checks to heartbeat
type heartbeater interface {
UpdateTTL(id, output, status string) error
type TTLUpdater interface {
UpdateTTL(id, namespace, output, status string) error
}
// scriptCheck runs script checks via a interfaces.ScriptExecutor and updates the
// appropriate check's TTL when the script succeeds.
type scriptCheck struct {
id string
agent heartbeater
check *structs.ServiceCheck
lastCheckOk bool // true if the last check was ok; otherwise false
id string
consulNamespace string
ttlUpdater TTLUpdater
check *structs.ServiceCheck
lastCheckOk bool // true if the last check was ok; otherwise false
tasklet
}
@@ -276,8 +280,9 @@ type scriptCheckConfig struct {
allocID string
taskName string
serviceID string
namespace string // consul namespace (TODO: SET)
check *structs.ServiceCheck
agent heartbeater
ttlUpdater TTLUpdater
driverExec tinterfaces.ScriptExecutor
taskEnv *taskenv.TaskEnv
logger log.Logger
@@ -299,7 +304,7 @@ func newScriptCheck(config *scriptCheckConfig) *scriptCheck {
orig := config.check
sc := &scriptCheck{
agent: config.agent,
ttlUpdater: config.ttlUpdater,
check: config.check.Copy(),
lastCheckOk: true, // start logging on first failure
}
@@ -325,6 +330,7 @@ func newScriptCheck(config *scriptCheckConfig) *scriptCheck {
} else {
sc.id = agentconsul.MakeCheckID(config.serviceID, sc.check)
}
sc.consulNamespace = config.namespace
return sc
}
@@ -394,7 +400,7 @@ const (
// service registration and the first check.
func (s *scriptCheck) updateTTL(ctx context.Context, msg, state string) error {
for attempts := 0; ; attempts++ {
err := s.agent.UpdateTTL(s.id, msg, state)
err := s.ttlUpdater.UpdateTTL(s.id, s.consulNamespace, msg, state)
if err == nil {
return nil
}

View File

@@ -19,7 +19,7 @@ import (
"github.com/stretchr/testify/require"
)
func newScriptMock(hb heartbeater, exec interfaces.ScriptExecutor, logger hclog.Logger, interval, timeout time.Duration) *scriptCheck {
func newScriptMock(hb TTLUpdater, exec interfaces.ScriptExecutor, logger hclog.Logger, interval, timeout time.Duration) *scriptCheck {
script := newScriptCheck(&scriptCheckConfig{
allocID: "allocid",
taskName: "testtask",
@@ -28,7 +28,7 @@ func newScriptMock(hb heartbeater, exec interfaces.ScriptExecutor, logger hclog.
Interval: interval,
Timeout: timeout,
},
agent: hb,
ttlUpdater: hb,
driverExec: exec,
taskEnv: &taskenv.TaskEnv{},
logger: logger,
@@ -39,13 +39,13 @@ func newScriptMock(hb heartbeater, exec interfaces.ScriptExecutor, logger hclog.
return script
}
// fakeHeartbeater implements the heartbeater interface to allow mocking out
// fakeHeartbeater implements the TTLUpdater interface to allow mocking out
// Consul in script executor tests.
type fakeHeartbeater struct {
heartbeats chan heartbeat
}
func (f *fakeHeartbeater) UpdateTTL(checkID, output, status string) error {
func (f *fakeHeartbeater) UpdateTTL(checkID, namespace, output, status string) error {
f.heartbeats <- heartbeat{checkID: checkID, output: output, status: status}
return nil
}
@@ -67,7 +67,7 @@ func TestScript_Exec_Cancel(t *testing.T) {
defer cancel()
logger := testlog.HCLogger(t)
script := newScriptMock(nil, // heartbeater should never be called
script := newScriptMock(nil, // TTLUpdater should never be called
exec, logger, time.Hour, time.Hour)
handle := script.run()
@@ -242,10 +242,10 @@ func TestScript_TaskEnvInterpolation(t *testing.T) {
map[string]string{"SVC_NAME": "frontend"}).Build()
svcHook := newServiceHook(serviceHookConfig{
alloc: alloc,
task: task,
consul: consulClient,
logger: logger,
alloc: alloc,
task: task,
consulServices: consulClient,
logger: logger,
})
// emulate prestart having been fired
svcHook.taskEnv = env
@@ -255,7 +255,7 @@ func TestScript_TaskEnvInterpolation(t *testing.T) {
task: task,
consul: consulClient,
logger: logger,
shutdownWait: time.Hour, // heartbeater will never be called
shutdownWait: time.Hour, // TTLUpdater will never be called
})
// emulate prestart having been fired
scHook.taskEnv = env

View File

@@ -21,9 +21,10 @@ var _ interfaces.TaskExitedHook = &serviceHook{}
var _ interfaces.TaskStopHook = &serviceHook{}
type serviceHookConfig struct {
alloc *structs.Allocation
task *structs.Task
consul consul.ConsulServiceAPI
alloc *structs.Allocation
task *structs.Task
consulServices consul.ConsulServiceAPI
consulNamespace string
// Restarter is a subset of the TaskLifecycle interface
restarter agentconsul.WorkloadRestarter
@@ -32,11 +33,12 @@ type serviceHookConfig struct {
}
type serviceHook struct {
consul consul.ConsulServiceAPI
allocID string
taskName string
restarter agentconsul.WorkloadRestarter
logger log.Logger
allocID string
taskName string
consulNamespace string
consulServices consul.ConsulServiceAPI
restarter agentconsul.WorkloadRestarter
logger log.Logger
// The following fields may be updated
driverExec tinterfaces.ScriptExecutor
@@ -58,12 +60,13 @@ type serviceHook struct {
func newServiceHook(c serviceHookConfig) *serviceHook {
h := &serviceHook{
consul: c.consul,
allocID: c.alloc.ID,
taskName: c.task.Name,
services: c.task.Services,
restarter: c.restarter,
ports: c.alloc.AllocatedResources.Shared.Ports,
allocID: c.alloc.ID,
taskName: c.task.Name,
consulServices: c.consulServices,
consulNamespace: c.consulNamespace,
services: c.task.Services,
restarter: c.restarter,
ports: c.alloc.AllocatedResources.Shared.Ports,
}
if res := c.alloc.AllocatedResources.Tasks[c.task.Name]; res != nil {
@@ -95,7 +98,7 @@ func (h *serviceHook) Poststart(ctx context.Context, req *interfaces.TaskPoststa
// Create task services struct with request's driver metadata
workloadServices := h.getWorkloadServices()
return h.consul.RegisterWorkload(workloadServices)
return h.consulServices.RegisterWorkload(workloadServices)
}
func (h *serviceHook) Update(ctx context.Context, req *interfaces.TaskUpdateRequest, _ *interfaces.TaskUpdateResponse) error {
@@ -118,7 +121,7 @@ func (h *serviceHook) Update(ctx context.Context, req *interfaces.TaskUpdateRequ
// Create new task services struct with those new values
newWorkloadServices := h.getWorkloadServices()
return h.consul.UpdateWorkload(oldWorkloadServices, newWorkloadServices)
return h.consulServices.UpdateWorkload(oldWorkloadServices, newWorkloadServices)
}
func (h *serviceHook) updateHookFields(req *interfaces.TaskUpdateRequest) error {
@@ -168,12 +171,12 @@ func (h *serviceHook) Exited(context.Context, *interfaces.TaskExitedRequest, *in
// deregister services from Consul.
func (h *serviceHook) deregister() {
workloadServices := h.getWorkloadServices()
h.consul.RemoveWorkload(workloadServices)
h.consulServices.RemoveWorkload(workloadServices)
// Canary flag may be getting flipped when the alloc is being
// destroyed, so remove both variations of the service
workloadServices.Canary = !workloadServices.Canary
h.consul.RemoveWorkload(workloadServices)
h.consulServices.RemoveWorkload(workloadServices)
h.initialRegistration = false
}
@@ -190,14 +193,15 @@ func (h *serviceHook) getWorkloadServices() *agentconsul.WorkloadServices {
// Create task services struct with request's driver metadata
return &agentconsul.WorkloadServices{
AllocID: h.allocID,
Task: h.taskName,
Restarter: h.restarter,
Services: interpolatedServices,
DriverExec: h.driverExec,
DriverNetwork: h.driverNet,
Networks: h.networks,
Canary: h.canary,
Ports: h.ports,
AllocID: h.allocID,
Task: h.taskName,
ConsulNamespace: h.consulNamespace,
Restarter: h.restarter,
Services: interpolatedServices,
DriverExec: h.driverExec,
DriverNetwork: h.driverNet,
Networks: h.networks,
Canary: h.canary,
Ports: h.ports,
}
}

View File

@@ -23,10 +23,10 @@ func TestUpdate_beforePoststart(t *testing.T) {
c := consul.NewMockConsulServiceClient(t, logger)
hook := newServiceHook(serviceHookConfig{
alloc: alloc,
task: alloc.LookupTask("web"),
consul: c,
logger: logger,
alloc: alloc,
task: alloc.LookupTask("web"),
consulServices: c,
logger: logger,
})
require.NoError(t, hook.Update(context.Background(), &interfaces.TaskUpdateRequest{Alloc: alloc}, &interfaces.TaskUpdateResponse{}))
require.Len(t, c.GetOps(), 0)

View File

@@ -89,26 +89,31 @@ func (tr *TaskRunner) initHooks() {
}))
}
// Get the consul namespace for the TG of the allocation
consulNamespace := tr.alloc.ConsulNamespace()
// If there are templates is enabled, add the hook
if len(task.Templates) != 0 {
tr.runnerHooks = append(tr.runnerHooks, newTemplateHook(&templateHookConfig{
logger: hookLogger,
lifecycle: tr,
events: tr,
templates: task.Templates,
clientConfig: tr.clientConfig,
envBuilder: tr.envBuilder,
logger: hookLogger,
lifecycle: tr,
events: tr,
templates: task.Templates,
clientConfig: tr.clientConfig,
envBuilder: tr.envBuilder,
consulNamespace: consulNamespace,
}))
}
// Always add the service hook. A task with no services on initial registration
// may be updated to include services, which must be handled with this hook.
tr.runnerHooks = append(tr.runnerHooks, newServiceHook(serviceHookConfig{
alloc: tr.Alloc(),
task: tr.Task(),
consul: tr.consulServiceClient,
restarter: tr,
logger: hookLogger,
alloc: tr.Alloc(),
task: tr.Task(),
consulServices: tr.consulServiceClient,
consulNamespace: consulNamespace,
restarter: tr,
logger: hookLogger,
}))
// If this is a Connect sidecar proxy (or a Connect Native) service,
@@ -129,7 +134,7 @@ func (tr *TaskRunner) initHooks() {
if task.UsesConnectSidecar() {
tr.runnerHooks = append(tr.runnerHooks,
newEnvoyVersionHook(newEnvoyVersionHookConfig(alloc, tr.consulProxiesClient, hookLogger)),
newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, tr.clientConfig.ConsulConfig, hookLogger)),
newEnvoyBootstrapHook(newEnvoyBootstrapHookConfig(alloc, tr.clientConfig.ConsulConfig, consulNamespace, hookLogger)),
)
} else if task.Kind.IsConnectNative() {
tr.runnerHooks = append(tr.runnerHooks, newConnectNativeHook(

View File

@@ -1108,7 +1108,8 @@ func TestTaskRunner_CheckWatcher_Restart(t *testing.T) {
// backed by a mock consul whose checks are always unhealthy.
consulAgent := agentconsul.NewMockAgent()
consulAgent.SetStatus("critical")
consulClient := agentconsul.NewServiceClient(consulAgent, conf.Logger, true)
namespacesClient := agentconsul.NewNamespacesClient(agentconsul.NewMockNamespaces(nil))
consulClient := agentconsul.NewServiceClient(consulAgent, namespacesClient, conf.Logger, true)
go consulClient.Run()
defer consulClient.Shutdown()
@@ -1786,7 +1787,8 @@ func TestTaskRunner_DriverNetwork(t *testing.T) {
// Use a mock agent to test for services
consulAgent := agentconsul.NewMockAgent()
consulClient := agentconsul.NewServiceClient(consulAgent, conf.Logger, true)
namespacesClient := agentconsul.NewNamespacesClient(agentconsul.NewMockNamespaces(nil))
consulClient := agentconsul.NewServiceClient(consulAgent, namespacesClient, conf.Logger, true)
defer consulClient.Shutdown()
go consulClient.Run()
@@ -1801,7 +1803,7 @@ func TestTaskRunner_DriverNetwork(t *testing.T) {
testWaitForTaskToStart(t, tr)
testutil.WaitForResult(func() (bool, error) {
services, _ := consulAgent.Services()
services, _ := consulAgent.ServicesWithFilterOpts("", nil)
if n := len(services); n != 2 {
return false, fmt.Errorf("expected 2 services, but found %d", n)
}
@@ -1852,7 +1854,7 @@ func TestTaskRunner_DriverNetwork(t *testing.T) {
return true, nil
}, func(err error) {
services, _ := consulAgent.Services()
services, _ := consulAgent.ServicesWithFilterOpts("", nil)
for _, s := range services {
t.Logf(pretty.Sprint("Service: ", s))
}

View File

@@ -85,6 +85,9 @@ type TaskTemplateManagerConfig struct {
// ClientConfig is the Nomad Client configuration
ClientConfig *config.Config
// ConsulNamespace is the Consul namespace for the task
ConsulNamespace string
// VaultToken is the Vault token for the task.
VaultToken string
@@ -642,7 +645,12 @@ func newRunnerConfig(config *TaskTemplateManagerConfig,
if cc.ConsulConfig != nil {
conf.Consul.Address = &cc.ConsulConfig.Addr
conf.Consul.Token = &cc.ConsulConfig.Token
conf.Consul.Namespace = &cc.ConsulConfig.Namespace
// Get the Consul namespace from agent config. This is the lower level
// of precedence (beyond default).
if cc.ConsulConfig.Namespace != "" {
conf.Consul.Namespace = &cc.ConsulConfig.Namespace
}
if cc.ConsulConfig.EnableSSL != nil && *cc.ConsulConfig.EnableSSL {
verify := cc.ConsulConfig.VerifySSL != nil && *cc.ConsulConfig.VerifySSL
@@ -669,6 +677,12 @@ func newRunnerConfig(config *TaskTemplateManagerConfig,
}
}
// Get the Consul namespace from job/group config. This is the higher level
// of precedence if set (above agent config).
if config.ConsulNamespace != "" {
conf.Consul.Namespace = &config.ConsulNamespace
}
// Setup the Vault config
// Always set these to ensure nothing is picked up from the environment
emptyStr := ""

View File

@@ -14,6 +14,10 @@ import (
"github.com/hashicorp/nomad/nomad/structs"
)
const (
templateHookName = "template"
)
type templateHookConfig struct {
// logger is used to log
logger log.Logger
@@ -32,6 +36,9 @@ type templateHookConfig struct {
// envBuilder is the environment variable builder for the task.
envBuilder *taskenv.Builder
// consulNamespace is the current Consul namespace
consulNamespace string
}
type templateHook struct {
@@ -44,6 +51,9 @@ type templateHook struct {
templateManager *template.TaskTemplateManager
managerLock sync.Mutex
// consulNamespace is the current Consul namespace
consulNamespace string
// vaultToken is the current Vault token
vaultToken string
@@ -55,15 +65,15 @@ type templateHook struct {
}
func newTemplateHook(config *templateHookConfig) *templateHook {
h := &templateHook{
config: config,
return &templateHook{
config: config,
consulNamespace: config.consulNamespace,
logger: config.logger.Named(templateHookName),
}
h.logger = config.logger.Named(h.Name())
return h
}
func (*templateHook) Name() string {
return "template"
return templateHookName
}
func (h *templateHook) Prestart(ctx context.Context, req *interfaces.TaskPrestartRequest, resp *interfaces.TaskPrestartResponse) error {
@@ -106,6 +116,7 @@ func (h *templateHook) newManager() (unblock chan struct{}, err error) {
Events: h.config.events,
Templates: h.config.templates,
ClientConfig: h.config.clientConfig,
ConsulNamespace: h.config.consulNamespace,
VaultToken: h.vaultToken,
VaultNamespace: h.vaultNamespace,
TaskDir: h.taskDir,