Implement DriverNetwork and Service.AddressMode

Ideally DriverNetwork would be fully populated in Driver.Prestart, but
Docker doesn't assign the container's IP until you start the container.

However, it's important to setup the port env vars before calling
Driver.Start, so Prestart should populate that.
This commit is contained in:
Michael Schurter 2017-06-09 10:29:41 -07:00
parent 9993616e4f
commit b9bfb84b53
18 changed files with 338 additions and 112 deletions

View File

@ -95,11 +95,12 @@ type ServiceCheck struct {
// The Service model represents a Consul service definition
type Service struct {
Id string
Name string
Tags []string
PortLabel string `mapstructure:"port"`
Checks []ServiceCheck
Id string
Name string
Tags []string
PortLabel string `mapstructure:"port"`
AddressMode string `mapstructure:"address_mode"`
Checks []ServiceCheck
}
func (s *Service) Canonicalize(t *Task, tg *TaskGroup, job *Job) {

View File

@ -2,13 +2,14 @@ package client
import (
"github.com/hashicorp/nomad/client/driver"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
// ConsulServiceAPI is the interface the Nomad Client uses to register and
// remove services and checks from Consul.
type ConsulServiceAPI interface {
RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor) error
RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) error
RemoveTask(allocID string, task *structs.Task)
UpdateTask(allocID string, existing, newTask *structs.Task, exec driver.ScriptExecutor) error
}

View File

@ -471,7 +471,7 @@ func (d *DockerDriver) Prestart(ctx *ExecContext, task *structs.Task) (*Prestart
return nil, err
}
// Set state needed by Start()
// Set state needed by Start
d.driverConfig = driverConfig
// Initialize docker API clients
@ -488,12 +488,11 @@ func (d *DockerDriver) Prestart(ctx *ExecContext, task *structs.Task) (*Prestart
resp := NewPrestartResponse()
resp.CreatedResources.Add(dockerImageResKey, id)
resp.PortMap = d.driverConfig.PortMap
d.imageID = id
return resp, nil
}
func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
pluginLogFile := filepath.Join(ctx.TaskDir.Dir, "executor.out")
executorConfig := &dstructs.ExecutorConfig{
@ -560,6 +559,15 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle
pluginClient.Kill()
return nil, fmt.Errorf("Failed to start container %s: %s", container.ID, err)
}
// InspectContainer to get all of the container metadata as
// much of the metadata (eg networking) isn't populated until
// the container is started
if container, err = client.InspectContainer(container.ID); err != nil {
err = fmt.Errorf("failed to inspect started container %s: %s", container.ID, err)
d.logger.Printf("[ERR] driver.docker: %v", err)
pluginClient.Kill()
return nil, structs.NewRecoverableError(err, true)
}
d.logger.Printf("[INFO] driver.docker: started container %s", container.ID)
} else {
d.logger.Printf("[DEBUG] driver.docker: re-attaching to container %s with status %q",
@ -585,7 +593,51 @@ func (d *DockerDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle
}
go h.collectStats()
go h.run()
return h, nil
// Detect container address
ip, autoUse := d.detectIP(container)
// Create a response with the driver handle and container network metadata
resp := &StartResponse{
Handle: h,
Network: &cstructs.DriverNetwork{
PortMap: d.driverConfig.PortMap,
IP: ip,
AutoUseIP: autoUse,
},
}
return resp, nil
}
func (d *DockerDriver) detectIP(c *docker.Container) (string, bool) {
if c.NetworkSettings == nil {
// This should only happen if there's been a coding error (such
// as not calling InspetContainer after CreateContainer). Code
// defensively in case the Docker API changes subtly.
d.logger.Printf("[WARN] driver.docker: no network settings for container %s", c.ID)
return "", false
}
ip, ipName := "", ""
n := 0
auto := false
for name, net := range c.NetworkSettings.Networks {
if net.IPAddress == "" {
// Ignore networks without an IP address
continue
}
n++
ip = net.IPAddress
ipName = name
// Don't auto-advertise bridge IPs
if name != "bridge" {
auto = true
}
}
if n > 1 {
d.logger.Printf("[WARN] driver.docker: multiple (%d) Docker networks for container %q but Nomad only supports 1: choosing %q", n, c.ID, ipName)
}
return ip, auto
}
func (d *DockerDriver) Cleanup(_ *ExecContext, res *CreatedResources) error {

View File

@ -57,10 +57,6 @@ type Factory func(*DriverContext) Driver
type PrestartResponse struct {
// CreatedResources by the driver.
CreatedResources *CreatedResources
// PortMap can be set by drivers to replace ports in environment
// variables with driver-specific mappings.
PortMap map[string]int
}
// NewPrestartResponse creates a new PrestartResponse with CreatedResources
@ -183,6 +179,20 @@ func (r *CreatedResources) Hash() []byte {
return h.Sum(nil)
}
// StartResponse is returned by Driver.Start.
type StartResponse struct {
// Handle to the driver's task executor for controlling the lifecycle
// of the task.
Handle DriverHandle
// DriverNetwork contains driver-specific network parameters such as
// the port map between the host and a container.
//
// Network may be nil as not all drivers or configurations create
// networks.
Network *cstructs.DriverNetwork
}
// Driver is used for execution of tasks. This allows Nomad
// to support many pluggable implementations of task drivers.
// Examples could include LXC, Docker, Qemu, etc.
@ -196,8 +206,11 @@ type Driver interface {
// CreatedResources may be non-nil even when an error occurs.
Prestart(*ExecContext, *structs.Task) (*PrestartResponse, error)
// Start is used to being task execution
Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error)
// Start is used to begin task execution. If error is nil,
// StartResponse.Handle will be the handle to the task's executor.
// StartResponse.Network may be nil if the task doesn't configure a
// network.
Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error)
// Open is used to re-open a handle to a task
Open(ctx *ExecContext, handleID string) (DriverHandle, error)
@ -208,7 +221,7 @@ type Driver interface {
//
// If Cleanup returns a recoverable error it may be retried. On retry
// it will be passed the same CreatedResources, so all successfully
// cleaned up resources should be removed.
// cleaned up resources should be removed or handled idempotently.
Cleanup(*ExecContext, *CreatedResources) error
// Drivers must validate their configuration

View File

@ -8,11 +8,16 @@ import (
"strings"
"sync"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/helper"
hargs "github.com/hashicorp/nomad/helper/args"
"github.com/hashicorp/nomad/nomad/structs"
)
// Network env vars
/*
*/
// A set of environment variables that are exported by each driver.
const (
// AllocDir is the environment variable with the path to the alloc directory
@ -60,12 +65,23 @@ const (
// E.g $NOMAD_ADDR_http=127.0.0.1:80
AddrPrefix = "NOMAD_ADDR_"
// IpPrefix is the prefix for passing the IP of a port allocation to a task.
// HostAddrPrefix is the prefix for passing both dynamic and static
// port allocations to tasks with the host's IP address for cases where
// the task advertises a different address.
HostAddrPrefix = "NOMAD_HOST_ADDR_"
// IpPrefix is the prefix for passing the IP of a port allocation to a
// task. This may not be the host's address depending on task
// configuration.
IpPrefix = "NOMAD_IP_"
// PortPrefix is the prefix for passing the port allocation to a task.
PortPrefix = "NOMAD_PORT_"
// HostIPPrefix is the prefix for passing the host's IP to a task for
// cases where the task advertises a different address.
HostIPPrefix = "NOMAD_HOST_IP_"
// HostPortPrefix is the prefix for passing the host port when a portmap is
// specified.
HostPortPrefix = "NOMAD_HOST_PORT_"
@ -202,7 +218,6 @@ type Builder struct {
region string
allocId string
allocName string
portMap map[string]string
vaultToken string
injectVaultToken bool
jobName string
@ -210,9 +225,13 @@ type Builder struct {
// otherPorts for tasks in the same alloc
otherPorts map[string]string
// driverNetwork is the network defined by the driver (or nil if none
// was defined).
driverNetwork *cstructs.DriverNetwork
// network resources from the task; must be lazily turned into env vars
// because portMaps can change after builder creation and affect
// network env vars.
// because portMaps and advertiseIP can change after builder creation
// and affect network env vars.
networks []*structs.NetworkResource
mu *sync.RWMutex
@ -287,21 +306,8 @@ func (b *Builder) Build() *TaskEnv {
nodeAttrs[nodeRegionKey] = b.region
}
// Build the addrs for this task
for _, network := range b.networks {
for label, intVal := range network.MapLabelToValues(nil) {
value := strconv.Itoa(intVal)
envMap[fmt.Sprintf("%s%s", IpPrefix, label)] = network.IP
envMap[fmt.Sprintf("%s%s", HostPortPrefix, label)] = value
if forwardedPort, ok := b.portMap[label]; ok {
value = forwardedPort
}
envMap[fmt.Sprintf("%s%s", PortPrefix, label)] = value
ipPort := net.JoinHostPort(network.IP, value)
envMap[fmt.Sprintf("%s%s", AddrPrefix, label)] = ipPort
}
}
// Build the network related env vars
buildNetworkEnv(envMap, b.networks, b.driverNetwork)
// Build the addr of the other tasks
for k, v := range b.otherPorts {
@ -455,17 +461,69 @@ func (b *Builder) SetSecretsDir(dir string) *Builder {
return b
}
func (b *Builder) SetPortMap(portMap map[string]int) *Builder {
newPortMap := make(map[string]string, len(portMap))
for k, v := range portMap {
newPortMap[k] = strconv.Itoa(v)
}
// SetDriverNetwork defined by the driver.
func (b *Builder) SetDriverNetwork(n *cstructs.DriverNetwork) *Builder {
ncopy := n.Copy()
b.mu.Lock()
b.portMap = newPortMap
b.driverNetwork = ncopy
b.mu.Unlock()
return b
}
// buildNetworkEnv env vars in the given map.
//
// Auto: NOMAD_{IP,PORT,ADDR}_<label>
// Host: NOMAD_HOST_{IP,PORT,ADDR}_<label>
// Driver: NOMAD_DRIVER_{IP,PORT,ADDR}_<label>
//
// Handled by setAlloc -> otherPorts:
//
// Task: NOMAD_TASK_{IP,PORT,ADDR}_<task>_<label> # Always host values
//
func buildNetworkEnv(envMap map[string]string, nets structs.Networks, driverNet *cstructs.DriverNetwork) {
for _, n := range nets {
for _, p := range n.ReservedPorts {
buildPortEnv(envMap, p, n.IP, driverNet)
}
for _, p := range n.DynamicPorts {
buildPortEnv(envMap, p, n.IP, driverNet)
}
}
}
func buildPortEnv(envMap map[string]string, p structs.Port, ip string, driverNet *cstructs.DriverNetwork) {
// Host IP, PORT, and ADDR
portStr := strconv.Itoa(p.Value)
envMap["NOMAD_HOST_IP_"+p.Label] = ip
envMap["NOMAD_HOST_PORT_"+p.Label] = portStr
envMap["NOMAD_HOST_ADDR_"+p.Label] = net.JoinHostPort(ip, portStr)
// Driver IP, PORT, and ADDR; use host if nil
if driverNet == nil {
envMap["NOMAD_DRIVER_IP_"+p.Label] = ip
envMap["NOMAD_DRIVER_PORT_"+p.Label] = portStr
envMap["NOMAD_DRIVER_ADDR_"+p.Label] = net.JoinHostPort(ip, portStr)
} else {
driverPortStr := strconv.Itoa(driverNet.PortMap[p.Label])
envMap["NOMAD_DRIVER_IP_"+p.Label] = driverNet.IP
envMap["NOMAD_DRIVER_PORT_"+p.Label] = driverPortStr
envMap["NOMAD_DRIVER_ADDR_"+p.Label] = net.JoinHostPort(driverNet.IP, driverPortStr)
}
// Auto IP, PORT, and ADDR
if driverNet.Use() {
// Use driver's
envMap[IpPrefix+p.Label] = envMap["NOMAD_DRIVER_IP_"+p.Label]
envMap[PortPrefix+p.Label] = envMap["NOMAD_DRIVER_PORT_"+p.Label]
envMap[AddrPrefix+p.Label] = envMap["NOMAD_DRIVER_ADDR_"+p.Label]
} else {
// Use host's
envMap[IpPrefix+p.Label] = envMap["NOMAD_HOST_IP_"+p.Label]
envMap[PortPrefix+p.Label] = envMap["NOMAD_HOST_PORT_"+p.Label]
envMap[AddrPrefix+p.Label] = envMap["NOMAD_HOST_ADDR_"+p.Label]
}
}
// SetHostEnvvars adds the host environment variables to the tasks. The filter
// parameter can be use to filter host environment from entering the tasks.
func (b *Builder) SetHostEnvvars(filter []string) *Builder {

View File

@ -102,7 +102,7 @@ func (d *ExecDriver) Prestart(*ExecContext, *structs.Task) (*PrestartResponse, e
return nil, nil
}
func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig ExecDriverConfig
if err := mapstructure.WeakDecode(task.Config, &driverConfig); err != nil {
return nil, err
@ -168,7 +168,7 @@ func (d *ExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle,
taskDir: ctx.TaskDir,
}
go h.run()
return h, nil
return &StartResponse{Handle: h}, nil
}
func (d *ExecDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }

View File

@ -202,7 +202,7 @@ func NewJavaDriverConfig(task *structs.Task, env *env.TaskEnv) (*JavaDriverConfi
return &driverConfig, nil
}
func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
driverConfig, err := NewJavaDriverConfig(task, ctx.TaskEnv)
if err != nil {
return nil, err
@ -296,7 +296,7 @@ func (d *JavaDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle,
waitCh: make(chan *dstructs.WaitResult, 1),
}
go h.run()
return h, nil
return &StartResponse{Handle: h}, nil
}
func (d *JavaDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }

View File

@ -178,7 +178,7 @@ func (d *LxcDriver) Prestart(*ExecContext, *structs.Task) (*PrestartResponse, er
}
// Start starts the LXC Driver
func (d *LxcDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *LxcDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig LxcDriverConfig
if err := mapstructure.WeakDecode(task.Config, &driverConfig); err != nil {
return nil, err
@ -269,7 +269,7 @@ func (d *LxcDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e
return nil, fmt.Errorf("unable to set cpu shares: %v", err)
}
handle := lxcDriverHandle{
h := lxcDriverHandle{
container: c,
initPid: c.InitPid(),
lxcPath: lxcPath,
@ -283,9 +283,9 @@ func (d *LxcDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e
doneCh: make(chan bool, 1),
}
go handle.run()
go h.run()
return &handle, nil
return &StartResponse{Handle: &h}, nil
}
func (d *LxcDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }

View File

@ -89,7 +89,7 @@ func (d *MockDriver) Prestart(*ExecContext, *structs.Task) (*PrestartResponse, e
}
// Start starts the mock driver
func (m *MockDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (m *MockDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig MockDriverConfig
dec, err := mapstructure.NewDecoder(&mapstructure.DecoderConfig{
DecodeHook: mapstructure.StringToTimeDurationHookFunc(),
@ -126,7 +126,7 @@ func (m *MockDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle,
}
m.logger.Printf("[DEBUG] driver.mock: starting task %q", task.Name)
go h.run()
return &h, nil
return &StartResponse{Handle: &h}, nil
}
// Cleanup deletes all keys except for Config.Options["cleanup_fail_on"] for

View File

@ -137,7 +137,7 @@ func (d *QemuDriver) Prestart(*ExecContext, *structs.Task) (*PrestartResponse, e
// Run an existing Qemu image. Start() will pull down an existing, valid Qemu
// image and save it to the Drivers Allocation Dir
func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig QemuDriverConfig
if err := mapstructure.WeakDecode(task.Config, &driverConfig); err != nil {
return nil, err
@ -195,7 +195,7 @@ func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle,
// reserved ports to the ports listenting in the VM
// Ex: hostfwd=tcp::22000-:22,hostfwd=tcp::80-:8080
var forwarding []string
taskPorts := task.Resources.Networks[0].MapLabelToValues(nil)
taskPorts := task.Resources.Networks[0].PortLabels()
for label, guest := range driverConfig.PortMap[0] {
host, ok := taskPorts[label]
if !ok {
@ -276,7 +276,7 @@ func (d *QemuDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle,
waitCh: make(chan *dstructs.WaitResult, 1),
}
go h.run()
return h, nil
return &StartResponse{Handle: h}, nil
}
type qemuId struct {

View File

@ -110,7 +110,7 @@ func (d *RawExecDriver) Prestart(*ExecContext, *structs.Task) (*PrestartResponse
return nil, nil
}
func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig ExecDriverConfig
if err := mapstructure.WeakDecode(task.Config, &driverConfig); err != nil {
return nil, err
@ -173,7 +173,7 @@ func (d *RawExecDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandl
taskDir: ctx.TaskDir,
}
go h.run()
return h, nil
return &StartResponse{Handle: h}, nil
}
func (d *RawExecDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }

View File

@ -236,7 +236,7 @@ func (d *RktDriver) Prestart(ctx *ExecContext, task *structs.Task) (*PrestartRes
}
// Run an existing Rkt image.
func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, error) {
func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (*StartResponse, error) {
var driverConfig RktDriverConfig
if err := mapstructure.WeakDecode(task.Config, &driverConfig); err != nil {
return nil, err
@ -512,7 +512,8 @@ func (d *RktDriver) Start(ctx *ExecContext, task *structs.Task) (DriverHandle, e
waitCh: make(chan *dstructs.WaitResult, 1),
}
go h.run()
return h, nil
//TODO Set Network
return &StartResponse{Handle: h}, nil
}
func (d *RktDriver) Cleanup(*ExecContext, *CreatedResources) error { return nil }

View File

@ -124,3 +124,40 @@ func (f FSIsolation) String() string {
return "INVALID"
}
}
// DriverNetwork is the network created by driver's (eg Docker's bridge
// network) during Prestart.
type DriverNetwork struct {
// PortMap can be set by drivers to replace ports in environment
// variables with driver-specific mappings.
PortMap map[string]int
// IP is the IP address for the task created by the driver.
IP string
// AutoUseIP indicates whether the driver thinks services that choose
// to auto-advertise-addresses should use this IP instead of the host's
AutoUseIP bool
}
// Use returns true if the driver suggests using the IP set. May be called on a
// nil Network in which case it returns false.
func (d *DriverNetwork) Use() bool {
return d != nil && d.AutoUseIP
}
// Copy a Network struct. If it is nil, nil is returned.
func (d *DriverNetwork) Copy() *DriverNetwork {
if d == nil {
return nil
}
pm := make(map[string]int, len(d.PortMap))
for k, v := range d.PortMap {
pm[k] = v
}
return &DriverNetwork{
PortMap: pm,
IP: d.IP,
AutoUseIP: d.AutoUseIP,
}
}

View File

@ -351,7 +351,8 @@ func (r *TaskRunner) RestoreState() (string, error) {
restartReason = pre06ScriptCheckReason
}
if err := r.registerServices(d, handle); err != nil {
//FIXME don't pass nil here
if err := r.registerServices(d, handle, nil); err != nil {
// Don't hard fail here as there's a chance this task
// registered with Consul properly when it initial
// started.
@ -1303,18 +1304,13 @@ func (r *TaskRunner) startTask() error {
// Run prestart
ctx := driver.NewExecContext(r.taskDir, r.envBuilder.Build())
resp, err := drv.Prestart(ctx, r.task)
presp, err := drv.Prestart(ctx, r.task)
// Merge newly created resources into previously created resources
if resp != nil {
if presp != nil {
r.createdResourcesLock.Lock()
r.createdResources.Merge(resp.CreatedResources)
r.createdResources.Merge(presp.CreatedResources)
r.createdResourcesLock.Unlock()
// Update environment with PortMap if it was returned
if len(resp.PortMap) > 0 {
r.envBuilder.SetPortMap(resp.PortMap)
}
}
if err != nil {
@ -1328,7 +1324,7 @@ func (r *TaskRunner) startTask() error {
ctx = driver.NewExecContext(r.taskDir, r.envBuilder.Build())
// Start the job
handle, err := drv.Start(ctx, r.task)
sresp, err := drv.Start(ctx, r.task)
if err != nil {
wrapped := fmt.Sprintf("failed to start task %q for alloc %q: %v",
r.task.Name, r.alloc.ID, err)
@ -1337,13 +1333,16 @@ func (r *TaskRunner) startTask() error {
}
if err := r.registerServices(drv, handle); err != nil {
// Update environment with the network defined by the driver for the task
r.envBuilder.SetDriverNetwork(sresp.Network)
if err := r.registerServices(drv, sresp.Handle, sresp.Network); err != nil {
// All IO is done asynchronously, so errors from registering
// services are hard failures.
r.logger.Printf("[ERR] client: failed to register services and checks for task %q alloc %q: %v", r.task.Name, r.alloc.ID, err)
// Kill the started task
if destroyed, err := r.handleDestroy(handle); !destroyed {
if destroyed, err := r.handleDestroy(sresp.Handle); !destroyed {
r.logger.Printf("[ERR] client: failed to kill task %q alloc %q. Resources may be leaked: %v",
r.task.Name, r.alloc.ID, err)
}
@ -1351,21 +1350,21 @@ func (r *TaskRunner) startTask() error {
}
r.handleLock.Lock()
r.handle = handle
r.handle = sresp.Handle
r.handleLock.Unlock()
return nil
}
// registerServices and checks with Consul.
func (r *TaskRunner) registerServices(d driver.Driver, h driver.ScriptExecutor) error {
func (r *TaskRunner) registerServices(d driver.Driver, h driver.DriverHandle, n *cstructs.DriverNetwork) error {
var exec driver.ScriptExecutor
if d.Abilities().Exec {
// Allow set the script executor if the driver supports it
exec = h
}
interpolateServices(r.envBuilder.Build(), r.task)
return r.consul.RegisterTask(r.alloc.ID, r.task, exec)
return r.consul.RegisterTask(r.alloc.ID, r.task, exec, n)
}
// interpolateServices interpolates tags in a service and checks with values from the

View File

@ -13,6 +13,7 @@ import (
metrics "github.com/armon/go-metrics"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/nomad/client/driver"
cstructs "github.com/hashicorp/nomad/client/structs"
"github.com/hashicorp/nomad/nomad/structs"
)
@ -420,15 +421,32 @@ func (c *ServiceClient) RegisterAgent(role string, services []*structs.Service)
// serviceRegs creates service registrations, check registrations, and script
// checks from a service.
func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *structs.Service,
exec driver.ScriptExecutor, task *structs.Task) error {
task *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) error {
id := makeTaskServiceID(allocID, task.Name, service)
host, port := task.FindHostAndPortFor(service.PortLabel)
addrMode := service.AddressMode
if addrMode == structs.AddressModeAuto {
if net == nil || !net.AutoUseIP {
// No driver network or shouldn't default to driver's network
addrMode = structs.AddressModeHost
} else {
addrMode = structs.AddressModeDriver
}
}
ip, port := task.Resources.Networks.Port(service.PortLabel)
if addrMode == structs.AddressModeDriver {
if net == nil {
//FIXME oof this is a doozy of an error condition... wording?
return fmt.Errorf("service %s cannot use driver's IP as it is unset", service.Name)
}
ip = net.IP
port = net.PortMap[service.PortLabel]
}
serviceReg := &api.AgentServiceRegistration{
ID: id,
Name: service.Name,
Tags: make([]string, len(service.Tags)),
Address: host,
Address: ip,
Port: port,
}
// copy isn't strictly necessary but can avoid bugs especially
@ -452,11 +470,15 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st
}
host, port := serviceReg.Address, serviceReg.Port
if check.PortLabel != "" {
host, port = task.FindHostAndPortFor(check.PortLabel)
// Checks should always use the host ip:port
//FIXME right?!
portLabel := check.PortLabel
if portLabel == "" {
// Default to the service's port label
portLabel = service.PortLabel
}
checkReg, err := createCheckReg(id, checkID, check, host, port)
ip, port := task.Resources.Networks.Port(portLabel)
checkReg, err := createCheckReg(id, checkID, check, ip, port)
if err != nil {
return fmt.Errorf("failed to add check %q: %v", check.Name, err)
}
@ -468,11 +490,14 @@ func (c *ServiceClient) serviceRegs(ops *operations, allocID string, service *st
// RegisterTask with Consul. Adds all sevice entries and checks to Consul. If
// exec is nil and a script check exists an error is returned.
//
// If the service IP is set it used as the address in the service registration.
// Checks will always use the IP from the Task struct (host's IP).
//
// Actual communication with Consul is done asynchrously (see Run).
func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor) error {
func (c *ServiceClient) RegisterTask(allocID string, task *structs.Task, exec driver.ScriptExecutor, net *cstructs.DriverNetwork) error {
ops := &operations{}
for _, service := range task.Services {
if err := c.serviceRegs(ops, allocID, service, exec, task); err != nil {
if err := c.serviceRegs(ops, allocID, service, task, exec, net); err != nil {
return err
}
}
@ -535,7 +560,8 @@ func (c *ServiceClient) UpdateTask(allocID string, existing, newTask *structs.Ta
// Any remaining services should just be enqueued directly
for _, newSvc := range newIDs {
err := c.serviceRegs(ops, allocID, newSvc, exec, newTask)
//FIXME driver.Network needed
err := c.serviceRegs(ops, allocID, newSvc, newTask, exec, nil)
if err != nil {
return err
}

View File

@ -579,9 +579,10 @@ func ApiTaskToStructsTask(apiTask *api.Task, structsTask *structs.Task) {
structsTask.Services = make([]*structs.Service, l)
for i, service := range apiTask.Services {
structsTask.Services[i] = &structs.Service{
Name: service.Name,
PortLabel: service.PortLabel,
Tags: service.Tags,
Name: service.Name,
PortLabel: service.PortLabel,
Tags: service.Tags,
AddressMode: service.AddressMode,
}
if l := len(service.Checks); l != 0 {

View File

@ -905,6 +905,7 @@ func parseServices(jobName string, taskGroupName string, task *api.Task, service
"tags",
"port",
"check",
"address_mode",
}
if err := checkHCLKeys(o.Val, valid); err != nil {
return multierror.Prefix(err, fmt.Sprintf("service (%d) ->", idx))

View File

@ -890,6 +890,26 @@ type NodeListStub struct {
ModifyIndex uint64
}
// Networks defined for a task on the Resources struct.
type Networks []*NetworkResource
// Port assignment and IP for the given label or empty values.
func (ns Networks) Port(label string) (string, int) {
for _, n := range ns {
for _, p := range n.ReservedPorts {
if p.Label == label {
return n.IP, p.Value
}
}
for _, p := range n.DynamicPorts {
if p.Label == label {
return n.IP, p.Value
}
}
}
return "", 0
}
// Resources is used to define the resources available
// on a client
type Resources struct {
@ -897,7 +917,7 @@ type Resources struct {
MemoryMB int
DiskMB int
IOPS int
Networks []*NetworkResource
Networks Networks
}
const (
@ -1054,10 +1074,10 @@ type Port struct {
type NetworkResource struct {
Device string // Name of the device
CIDR string // CIDR block of addresses
IP string // IP address
IP string // Host IP address
MBits int // Throughput
ReservedPorts []Port // Reserved ports
DynamicPorts []Port // Dynamically assigned ports
ReservedPorts []Port // Host Reserved ports
DynamicPorts []Port // Host Dynamically assigned ports
}
func (n *NetworkResource) Canonicalize() {
@ -1113,15 +1133,15 @@ func (n *NetworkResource) GoString() string {
return fmt.Sprintf("*%#v", *n)
}
func (n *NetworkResource) MapLabelToValues(port_map map[string]int) map[string]int {
labelValues := make(map[string]int)
ports := append(n.ReservedPorts, n.DynamicPorts...)
for _, port := range ports {
if mapping, ok := port_map[port.Label]; ok {
labelValues[port.Label] = mapping
} else {
labelValues[port.Label] = port.Value
}
// PortLabels returns a map of port labels to their assigned host ports.
func (n *NetworkResource) PortLabels() map[string]int {
num := len(n.ReservedPorts) + len(n.DynamicPorts)
labelValues := make(map[string]int, num)
for _, port := range n.ReservedPorts {
labelValues[port.Label] = port.Value
}
for _, port := range n.DynamicPorts {
labelValues[port.Label] = port.Value
}
return labelValues
}
@ -2430,6 +2450,12 @@ func (sc *ServiceCheck) Hash(serviceID string) string {
return fmt.Sprintf("%x", h.Sum(nil))
}
const (
AddressModeAuto = "auto"
AddressModeHost = "host"
AddressModeDriver = "driver" //FIXME plugin? I forget what we decided because like 1000 other things have gone wrong since then
)
// Service represents a Consul service definition in Nomad
type Service struct {
// Name of the service registered with Consul. Consul defaults the
@ -2441,8 +2467,13 @@ type Service struct {
// To specify the port number using the host's Consul Advertise
// address, specify an empty host in the PortLabel (e.g. `:port`).
PortLabel string
Tags []string // List of tags for the service
Checks []*ServiceCheck // List of checks associated with the service
// AddressMode specifies whether or not to use the host ip:port for
// this service.
AddressMode string
Tags []string // List of tags for the service
Checks []*ServiceCheck // List of checks associated with the service
}
func (s *Service) Copy() *Service {
@ -2467,6 +2498,11 @@ func (s *Service) Copy() *Service {
// Canonicalize interpolates values of Job, Task Group and Task in the Service
// Name. This also generates check names, service id and check ids.
func (s *Service) Canonicalize(job string, taskGroup string, task string) {
// Default to AddressModeAuto
if s.AddressMode == "" {
s.AddressMode = AddressModeAuto
}
// Ensure empty lists are treated as null to avoid scheduler issues when
// using DeepEquals
if len(s.Tags) == 0 {
@ -2503,6 +2539,13 @@ func (s *Service) Validate() error {
mErr.Errors = append(mErr.Errors, fmt.Errorf("service name must be valid per RFC 1123 and can contain only alphanumeric characters or dashes: %q", s.Name))
}
switch s.AddressMode {
case AddressModeAuto, AddressModeHost, AddressModeDriver:
// OK
default:
mErr.Errors = append(mErr.Errors, fmt.Errorf("service address_mode must be %q, %q, or %q; not %q", AddressModeAuto, AddressModeHost, AddressModeDriver, s.AddressMode))
}
for _, c := range s.Checks {
if s.PortLabel == "" && c.RequiresPort() {
mErr.Errors = append(mErr.Errors, fmt.Errorf("check %s invalid: check requires a port but the service %+q has no port", c.Name, s.Name))
@ -2720,15 +2763,6 @@ func (t *Task) GoString() string {
return fmt.Sprintf("*%#v", *t)
}
func (t *Task) FindHostAndPortFor(portLabel string) (string, int) {
for _, network := range t.Resources.Networks {
if p, ok := network.MapLabelToValues(nil)[portLabel]; ok {
return network.IP, p
}
}
return "", 0
}
// Validate is used to sanity check a task
func (t *Task) Validate(ephemeralDisk *EphemeralDisk) error {
var mErr multierror.Error
@ -2873,7 +2907,7 @@ func validateServices(t *Task) error {
portLabels := make(map[string]struct{})
if t.Resources != nil {
for _, network := range t.Resources.Networks {
ports := network.MapLabelToValues(nil)
ports := network.PortLabels()
for portLabel, _ := range ports {
portLabels[portLabel] = struct{}{}
}
@ -2889,6 +2923,8 @@ func validateServices(t *Task) error {
mErr.Errors = append(mErr.Errors, err)
}
}
// Ensure address mode is valid
return mErr.ErrorOrNil()
}