Split advertised address from listen address

There are currently problems with "swarm init" and "swarm join" when an
explicit --listen-addr flag is not provided. swarmkit defaults to
finding the IP address associated with the default route, and in cloud
setups this is often the wrong choice.

Introduce a notion of "advertised address", with the client flag
--advertise-addr, and the daemon flag --swarm-default-advertise-addr to
provide a default. The default listening address is now 0.0.0.0, but a
valid advertised address must be detected or specified.

If no explicit advertised address is specified, error out if there is
more than one usable candidate IP address on the system. This requires a
user to explicitly choose instead of letting swarmkit make the wrong
choice. For the purposes of this autodetection, we ignore certain
interfaces that are unlikely to be relevant (currently docker*).

The user is also required to choose a listen address on swarm init if
they specify an explicit advertise address that is a hostname or an IP
address that's not local to the system. This is a requirement for
overlay networking.

Also support specifying interface names to --listen-addr,
--advertise-addr, and the daemon flag --swarm-default-advertise-addr.
This will fail if the interface has multiple IP addresses (unless it has
a single IPv4 address and a single IPv6 address - then we resolve the
tie in favor of IPv4).

This change also exposes the node's externally-reachable address in
docker info, as requested by #24017.

Make corresponding API and CLI docs changes.

Signed-off-by: Aaron Lehmann <aaron.lehmann@docker.com>
(cherry picked from commit a0ccd0d42f)
Signed-off-by: Tibor Vass <tibor@docker.com>
This commit is contained in:
Aaron Lehmann 2016-06-30 18:07:35 -07:00 committed by Tibor Vass
parent 38beb3d9ab
commit 762a73bf7f
20 changed files with 792 additions and 69 deletions

View file

@ -79,6 +79,7 @@ func (cli *DockerCli) CmdInfo(args ...string) error {
fmt.Fprintf(cli.out, " Managers: %d\n", info.Swarm.Managers)
fmt.Fprintf(cli.out, " Nodes: %d\n", info.Swarm.Nodes)
}
fmt.Fprintf(cli.out, " Node Address: %s\n", info.Swarm.NodeAddr)
}
if len(info.Runtimes) > 0 {

View file

@ -1,7 +1,9 @@
package swarm
import (
"errors"
"fmt"
"strings"
"golang.org/x/net/context"
@ -21,7 +23,9 @@ const (
type initOptions struct {
swarmOptions
listenAddr NodeAddrOption
listenAddr NodeAddrOption
// Not a NodeAddrOption because it has no default port.
advertiseAddr string
forceNewCluster bool
}
@ -40,7 +44,8 @@ func newInitCommand(dockerCli *client.DockerCli) *cobra.Command {
}
flags := cmd.Flags()
flags.Var(&opts.listenAddr, "listen-addr", "Listen address")
flags.Var(&opts.listenAddr, flagListenAddr, "Listen address (format: <ip|hostname|interface>[:port])")
flags.StringVar(&opts.advertiseAddr, flagAdvertiseAddr, "", "Advertised address (format: <ip|hostname|interface>[:port])")
flags.BoolVar(&opts.forceNewCluster, "force-new-cluster", false, "Force create a new cluster from current state.")
addSwarmFlags(flags, &opts.swarmOptions)
return cmd
@ -52,12 +57,16 @@ func runInit(dockerCli *client.DockerCli, flags *pflag.FlagSet, opts initOptions
req := swarm.InitRequest{
ListenAddr: opts.listenAddr.String(),
AdvertiseAddr: opts.advertiseAddr,
ForceNewCluster: opts.forceNewCluster,
Spec: opts.swarmOptions.ToSpec(),
}
nodeID, err := client.SwarmInit(ctx, req)
if err != nil {
if strings.Contains(err.Error(), "could not choose an IP address to advertise") || strings.Contains(err.Error(), "could not find the system's IP address") {
return errors.New(err.Error() + " - specify one with --advertise-addr")
}
return err
}

View file

@ -14,7 +14,9 @@ import (
type joinOptions struct {
remote string
listenAddr NodeAddrOption
token string
// Not a NodeAddrOption because it has no default port.
advertiseAddr string
token string
}
func newJoinCommand(dockerCli *client.DockerCli) *cobra.Command {
@ -33,7 +35,8 @@ func newJoinCommand(dockerCli *client.DockerCli) *cobra.Command {
}
flags := cmd.Flags()
flags.Var(&opts.listenAddr, flagListenAddr, "Listen address")
flags.Var(&opts.listenAddr, flagListenAddr, "Listen address (format: <ip|hostname|interface>[:port])")
flags.StringVar(&opts.advertiseAddr, flagAdvertiseAddr, "", "Advertised address (format: <ip|hostname|interface>[:port])")
flags.StringVar(&opts.token, flagToken, "", "Token for entry into the swarm")
return cmd
}
@ -43,9 +46,10 @@ func runJoin(dockerCli *client.DockerCli, opts joinOptions) error {
ctx := context.Background()
req := swarm.JoinRequest{
JoinToken: opts.token,
ListenAddr: opts.listenAddr.String(),
RemoteAddrs: []string{opts.remote},
JoinToken: opts.token,
ListenAddr: opts.listenAddr.String(),
AdvertiseAddr: opts.advertiseAddr,
RemoteAddrs: []string{opts.remote},
}
err := client.SwarmJoin(ctx, req)
if err != nil {

View file

@ -18,6 +18,7 @@ const (
flagCertExpiry = "cert-expiry"
flagDispatcherHeartbeat = "dispatcher-heartbeat"
flagListenAddr = "listen-addr"
flagAdvertiseAddr = "advertise-addr"
flagToken = "token"
flagTaskHistoryLimit = "task-history-limit"
flagExternalCA = "external-ca"

203
api/client/system/info.go Normal file
View file

@ -0,0 +1,203 @@
package system
import (
"fmt"
"strings"
"golang.org/x/net/context"
"github.com/docker/docker/api/client"
"github.com/docker/docker/cli"
"github.com/docker/docker/pkg/ioutils"
"github.com/docker/docker/utils"
"github.com/docker/engine-api/types/swarm"
"github.com/docker/go-units"
"github.com/spf13/cobra"
)
// NewInfoCommand creates a new cobra.Command for `docker info`
func NewInfoCommand(dockerCli *client.DockerCli) *cobra.Command {
cmd := &cobra.Command{
Use: "info",
Short: "Display system-wide information",
Args: cli.ExactArgs(0),
RunE: func(cmd *cobra.Command, args []string) error {
return runInfo(dockerCli)
},
}
return cmd
}
func runInfo(dockerCli *client.DockerCli) error {
info, err := dockerCli.Client().Info(context.Background())
if err != nil {
return err
}
fmt.Fprintf(dockerCli.Out(), "Containers: %d\n", info.Containers)
fmt.Fprintf(dockerCli.Out(), " Running: %d\n", info.ContainersRunning)
fmt.Fprintf(dockerCli.Out(), " Paused: %d\n", info.ContainersPaused)
fmt.Fprintf(dockerCli.Out(), " Stopped: %d\n", info.ContainersStopped)
fmt.Fprintf(dockerCli.Out(), "Images: %d\n", info.Images)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Server Version: %s\n", info.ServerVersion)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Storage Driver: %s\n", info.Driver)
if info.DriverStatus != nil {
for _, pair := range info.DriverStatus {
fmt.Fprintf(dockerCli.Out(), " %s: %s\n", pair[0], pair[1])
// print a warning if devicemapper is using a loopback file
if pair[0] == "Data loop file" {
fmt.Fprintln(dockerCli.Err(), " WARNING: Usage of loopback devices is strongly discouraged for production use. Use `--storage-opt dm.thinpooldev` to specify a custom block storage device.")
}
}
}
if info.SystemStatus != nil {
for _, pair := range info.SystemStatus {
fmt.Fprintf(dockerCli.Out(), "%s: %s\n", pair[0], pair[1])
}
}
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Logging Driver: %s\n", info.LoggingDriver)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Cgroup Driver: %s\n", info.CgroupDriver)
fmt.Fprintf(dockerCli.Out(), "Plugins: \n")
fmt.Fprintf(dockerCli.Out(), " Volume:")
fmt.Fprintf(dockerCli.Out(), " %s", strings.Join(info.Plugins.Volume, " "))
fmt.Fprintf(dockerCli.Out(), "\n")
fmt.Fprintf(dockerCli.Out(), " Network:")
fmt.Fprintf(dockerCli.Out(), " %s", strings.Join(info.Plugins.Network, " "))
fmt.Fprintf(dockerCli.Out(), "\n")
if len(info.Plugins.Authorization) != 0 {
fmt.Fprintf(dockerCli.Out(), " Authorization:")
fmt.Fprintf(dockerCli.Out(), " %s", strings.Join(info.Plugins.Authorization, " "))
fmt.Fprintf(dockerCli.Out(), "\n")
}
fmt.Fprintf(dockerCli.Out(), "Swarm: %v\n", info.Swarm.LocalNodeState)
if info.Swarm.LocalNodeState != swarm.LocalNodeStateInactive {
fmt.Fprintf(dockerCli.Out(), " NodeID: %s\n", info.Swarm.NodeID)
if info.Swarm.Error != "" {
fmt.Fprintf(dockerCli.Out(), " Error: %v\n", info.Swarm.Error)
}
fmt.Fprintf(dockerCli.Out(), " Is Manager: %v\n", info.Swarm.ControlAvailable)
if info.Swarm.ControlAvailable {
fmt.Fprintf(dockerCli.Out(), " Managers: %d\n", info.Swarm.Managers)
fmt.Fprintf(dockerCli.Out(), " Nodes: %d\n", info.Swarm.Nodes)
}
fmt.Fprintf(dockerCli.Out(), " Node Address: %s\n", info.Swarm.NodeAddr)
}
if len(info.Runtimes) > 0 {
fmt.Fprintf(dockerCli.Out(), "Runtimes:")
for name := range info.Runtimes {
fmt.Fprintf(dockerCli.Out(), " %s", name)
}
fmt.Fprint(dockerCli.Out(), "\n")
fmt.Fprintf(dockerCli.Out(), "Default Runtime: %s\n", info.DefaultRuntime)
}
fmt.Fprintf(dockerCli.Out(), "Security Options:")
ioutils.FprintfIfNotEmpty(dockerCli.Out(), " %s", strings.Join(info.SecurityOptions, " "))
fmt.Fprintf(dockerCli.Out(), "\n")
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Kernel Version: %s\n", info.KernelVersion)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Operating System: %s\n", info.OperatingSystem)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "OSType: %s\n", info.OSType)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Architecture: %s\n", info.Architecture)
fmt.Fprintf(dockerCli.Out(), "CPUs: %d\n", info.NCPU)
fmt.Fprintf(dockerCli.Out(), "Total Memory: %s\n", units.BytesSize(float64(info.MemTotal)))
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Name: %s\n", info.Name)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "ID: %s\n", info.ID)
fmt.Fprintf(dockerCli.Out(), "Docker Root Dir: %s\n", info.DockerRootDir)
fmt.Fprintf(dockerCli.Out(), "Debug Mode (client): %v\n", utils.IsDebugEnabled())
fmt.Fprintf(dockerCli.Out(), "Debug Mode (server): %v\n", info.Debug)
if info.Debug {
fmt.Fprintf(dockerCli.Out(), " File Descriptors: %d\n", info.NFd)
fmt.Fprintf(dockerCli.Out(), " Goroutines: %d\n", info.NGoroutines)
fmt.Fprintf(dockerCli.Out(), " System Time: %s\n", info.SystemTime)
fmt.Fprintf(dockerCli.Out(), " EventsListeners: %d\n", info.NEventsListener)
}
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Http Proxy: %s\n", info.HTTPProxy)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "Https Proxy: %s\n", info.HTTPSProxy)
ioutils.FprintfIfNotEmpty(dockerCli.Out(), "No Proxy: %s\n", info.NoProxy)
if info.IndexServerAddress != "" {
u := dockerCli.ConfigFile().AuthConfigs[info.IndexServerAddress].Username
if len(u) > 0 {
fmt.Fprintf(dockerCli.Out(), "Username: %v\n", u)
}
fmt.Fprintf(dockerCli.Out(), "Registry: %v\n", info.IndexServerAddress)
}
// Only output these warnings if the server does not support these features
if info.OSType != "windows" {
if !info.MemoryLimit {
fmt.Fprintln(dockerCli.Err(), "WARNING: No memory limit support")
}
if !info.SwapLimit {
fmt.Fprintln(dockerCli.Err(), "WARNING: No swap limit support")
}
if !info.KernelMemory {
fmt.Fprintln(dockerCli.Err(), "WARNING: No kernel memory limit support")
}
if !info.OomKillDisable {
fmt.Fprintln(dockerCli.Err(), "WARNING: No oom kill disable support")
}
if !info.CPUCfsQuota {
fmt.Fprintln(dockerCli.Err(), "WARNING: No cpu cfs quota support")
}
if !info.CPUCfsPeriod {
fmt.Fprintln(dockerCli.Err(), "WARNING: No cpu cfs period support")
}
if !info.CPUShares {
fmt.Fprintln(dockerCli.Err(), "WARNING: No cpu shares support")
}
if !info.CPUSet {
fmt.Fprintln(dockerCli.Err(), "WARNING: No cpuset support")
}
if !info.IPv4Forwarding {
fmt.Fprintln(dockerCli.Err(), "WARNING: IPv4 forwarding is disabled")
}
if !info.BridgeNfIptables {
fmt.Fprintln(dockerCli.Err(), "WARNING: bridge-nf-call-iptables is disabled")
}
if !info.BridgeNfIP6tables {
fmt.Fprintln(dockerCli.Err(), "WARNING: bridge-nf-call-ip6tables is disabled")
}
}
if info.Labels != nil {
fmt.Fprintln(dockerCli.Out(), "Labels:")
for _, attribute := range info.Labels {
fmt.Fprintf(dockerCli.Out(), " %s\n", attribute)
}
}
ioutils.FprintfIfTrue(dockerCli.Out(), "Experimental: %v\n", info.ExperimentalBuild)
if info.ClusterStore != "" {
fmt.Fprintf(dockerCli.Out(), "Cluster Store: %s\n", info.ClusterStore)
}
if info.ClusterAdvertise != "" {
fmt.Fprintf(dockerCli.Out(), "Cluster Advertise: %s\n", info.ClusterAdvertise)
}
if info.RegistryConfig != nil && (len(info.RegistryConfig.InsecureRegistryCIDRs) > 0 || len(info.RegistryConfig.IndexConfigs) > 0) {
fmt.Fprintln(dockerCli.Out(), "Insecure Registries:")
for _, registry := range info.RegistryConfig.IndexConfigs {
if registry.Secure == false {
fmt.Fprintf(dockerCli.Out(), " %s\n", registry.Name)
}
}
for _, registry := range info.RegistryConfig.InsecureRegistryCIDRs {
mask, _ := registry.Mask.Size()
fmt.Fprintf(dockerCli.Out(), " %s/%d\n", registry.IP.String(), mask)
}
}
return nil
}

View file

@ -274,9 +274,11 @@ func (cli *DaemonCli) start() (err error) {
name, _ := os.Hostname()
c, err := cluster.New(cluster.Config{
Root: cli.Config.Root,
Name: name,
Backend: d,
Root: cli.Config.Root,
Name: name,
Backend: d,
NetworkSubnetsProvider: d,
DefaultAdvertiseAddr: cli.Config.SwarmDefaultAdvertiseAddr,
})
if err != nil {
logrus.Fatalf("Error creating cluster component: %v", err)

View file

@ -1839,11 +1839,17 @@ _docker_swarm_init() {
fi
return
;;
--advertise-addr)
if [[ $cur == *: ]] ; then
COMPREPLY=( $( compgen -W "2377" -- "${cur##*:}" ) )
fi
return
;;
esac
case "$cur" in
-*)
COMPREPLY=( $( compgen -W "--force-new-cluster --help --listen-addr" -- "$cur" ) )
COMPREPLY=( $( compgen -W "--advertise-addr --force-new-cluster --help --listen-addr" -- "$cur" ) )
;;
esac
}
@ -1873,11 +1879,17 @@ _docker_swarm_join() {
fi
return
;;
--advertise-addr)
if [[ $cur == *: ]] ; then
COMPREPLY=( $( compgen -W "2377" -- "${cur##*:}" ) )
fi
return
;;
esac
case "$cur" in
-*)
COMPREPLY=( $( compgen -W "--help --listen-addr --token" -- "$cur" ) )
COMPREPLY=( $( compgen -W "--adveritse-addr --help --listen-addr --token" -- "$cur" ) )
;;
*:)
COMPREPLY=( $( compgen -W "2377" -- "${cur##*:}" ) )

View file

@ -1204,6 +1204,7 @@ __docker_swarm_subcommand() {
(init)
_arguments $(__docker_arguments) \
$opts_help \
"($help)--advertise-addr[Advertised address]:ip\:port: " \
"($help)*--external-ca=[Specifications of one or more certificate signing endpoints]:endpoint: " \
"($help)--force-new-cluster[Force create a new cluster from current state]" \
"($help)--listen-addr=[Listen address]:ip\:port: " && ret=0
@ -1216,6 +1217,7 @@ __docker_swarm_subcommand() {
(join)
_arguments $(__docker_arguments) \
$opts_help \
"($help)--advertise-addr[Advertised address]:ip\:port: " \
"($help)--listen-addr=[Listen address]:ip\:port: " \
"($help)--token=[Token for entry into the swarm]:secret: " \
"($help -):host\:port: " && ret=0

View file

@ -4,6 +4,7 @@ import (
"encoding/json"
"fmt"
"io/ioutil"
"net"
"os"
"path/filepath"
"strings"
@ -73,14 +74,35 @@ var defaultSpec = types.Spec{
}
type state struct {
// LocalAddr is this machine's local IP or hostname, if specified.
LocalAddr string
// RemoteAddr is the address that was given to "swarm join. It is used
// to find LocalAddr if necessary.
RemoteAddr string
// ListenAddr is the address we bind to, including a port.
ListenAddr string
// AdvertiseAddr is the address other nodes should connect to,
// including a port.
AdvertiseAddr string
}
// NetworkSubnetsProvider exposes functions for retrieving the subnets
// of networks managed by Docker, so they can be filtered.
type NetworkSubnetsProvider interface {
V4Subnets() []net.IPNet
V6Subnets() []net.IPNet
}
// Config provides values for Cluster.
type Config struct {
Root string
Name string
Backend executorpkg.Backend
Root string
Name string
Backend executorpkg.Backend
NetworkSubnetsProvider NetworkSubnetsProvider
// DefaultAdvertiseAddr is the default host/IP or network interface to use
// if no AdvertiseAddr value is specified.
DefaultAdvertiseAddr string
}
// Cluster provides capabilities to participate in a cluster as a worker or a
@ -88,13 +110,17 @@ type Config struct {
type Cluster struct {
sync.RWMutex
*node
root string
config Config
configEvent chan struct{} // todo: make this array and goroutine safe
listenAddr string
stop bool
err error
cancelDelay func()
root string
config Config
configEvent chan struct{} // todo: make this array and goroutine safe
localAddr string
actualLocalAddr string // after resolution, not persisted
remoteAddr string
listenAddr string
advertiseAddr string
stop bool
err error
cancelDelay func()
}
type node struct {
@ -126,7 +152,7 @@ func New(config Config) (*Cluster, error) {
return nil, err
}
n, err := c.startNewNode(false, st.ListenAddr, "", "")
n, err := c.startNewNode(false, st.LocalAddr, st.RemoteAddr, st.ListenAddr, st.AdvertiseAddr, "", "")
if err != nil {
return nil, err
}
@ -162,7 +188,12 @@ func (c *Cluster) loadState() (*state, error) {
}
func (c *Cluster) saveState() error {
dt, err := json.Marshal(state{ListenAddr: c.listenAddr})
dt, err := json.Marshal(state{
LocalAddr: c.localAddr,
RemoteAddr: c.remoteAddr,
ListenAddr: c.listenAddr,
AdvertiseAddr: c.advertiseAddr,
})
if err != nil {
return err
}
@ -195,7 +226,7 @@ func (c *Cluster) reconnectOnFailure(n *node) {
return
}
var err error
n, err = c.startNewNode(false, c.listenAddr, c.getRemoteAddress(), "")
n, err = c.startNewNode(false, c.localAddr, c.getRemoteAddress(), c.listenAddr, c.advertiseAddr, c.getRemoteAddress(), "")
if err != nil {
c.err = err
close(n.done)
@ -204,24 +235,55 @@ func (c *Cluster) reconnectOnFailure(n *node) {
}
}
func (c *Cluster) startNewNode(forceNewCluster bool, listenAddr, joinAddr, joinToken string) (*node, error) {
func (c *Cluster) startNewNode(forceNewCluster bool, localAddr, remoteAddr, listenAddr, advertiseAddr, joinAddr, joinToken string) (*node, error) {
if err := c.config.Backend.IsSwarmCompatible(); err != nil {
return nil, err
}
actualLocalAddr := localAddr
if actualLocalAddr == "" {
// If localAddr was not specified, resolve it automatically
// based on the route to joinAddr. localAddr can only be left
// empty on "join".
listenHost, _, err := net.SplitHostPort(listenAddr)
if err != nil {
return nil, fmt.Errorf("could not parse listen address: %v", err)
}
listenAddrIP := net.ParseIP(listenHost)
if listenAddrIP == nil || !listenAddrIP.IsUnspecified() {
actualLocalAddr = listenHost
} else {
if remoteAddr == "" {
// Should never happen except using swarms created by
// old versions that didn't save remoteAddr.
remoteAddr = "8.8.8.8:53"
}
conn, err := net.Dial("udp", remoteAddr)
if err != nil {
return nil, fmt.Errorf("could not find local IP address: %v", err)
}
localHostPort := conn.LocalAddr().String()
actualLocalAddr, _, _ = net.SplitHostPort(localHostPort)
conn.Close()
}
}
c.node = nil
c.cancelDelay = nil
c.stop = false
n, err := swarmagent.NewNode(&swarmagent.NodeConfig{
Hostname: c.config.Name,
ForceNewCluster: forceNewCluster,
ListenControlAPI: filepath.Join(c.root, controlSocket),
ListenRemoteAPI: listenAddr,
JoinAddr: joinAddr,
StateDir: c.root,
JoinToken: joinToken,
Executor: container.NewExecutor(c.config.Backend),
HeartbeatTick: 1,
ElectionTick: 3,
Hostname: c.config.Name,
ForceNewCluster: forceNewCluster,
ListenControlAPI: filepath.Join(c.root, controlSocket),
ListenRemoteAPI: listenAddr,
AdvertiseRemoteAPI: advertiseAddr,
JoinAddr: joinAddr,
StateDir: c.root,
JoinToken: joinToken,
Executor: container.NewExecutor(c.config.Backend),
HeartbeatTick: 1,
ElectionTick: 3,
})
if err != nil {
return nil, err
@ -236,8 +298,13 @@ func (c *Cluster) startNewNode(forceNewCluster bool, listenAddr, joinAddr, joinT
reconnectDelay: initialReconnectDelay,
}
c.node = node
c.localAddr = localAddr
c.actualLocalAddr = actualLocalAddr // not saved
c.remoteAddr = remoteAddr
c.listenAddr = listenAddr
c.advertiseAddr = advertiseAddr
c.saveState()
c.config.Backend.SetClusterProvider(c)
go func() {
err := n.Err(ctx)
@ -301,8 +368,49 @@ func (c *Cluster) Init(req types.InitRequest) (string, error) {
return "", err
}
listenHost, listenPort, err := resolveListenAddr(req.ListenAddr)
if err != nil {
c.Unlock()
return "", err
}
advertiseHost, advertisePort, err := c.resolveAdvertiseAddr(req.AdvertiseAddr, listenPort)
if err != nil {
c.Unlock()
return "", err
}
localAddr := listenHost
// If the advertise address is not one of the system's
// addresses, we also require a listen address.
listenAddrIP := net.ParseIP(listenHost)
if listenAddrIP != nil && listenAddrIP.IsUnspecified() {
advertiseIP := net.ParseIP(advertiseHost)
if advertiseIP == nil {
// not an IP
c.Unlock()
return "", errMustSpecifyListenAddr
}
systemIPs := listSystemIPs()
found := false
for _, systemIP := range systemIPs {
if systemIP.Equal(advertiseIP) {
found = true
break
}
}
if !found {
c.Unlock()
return "", errMustSpecifyListenAddr
}
localAddr = advertiseIP.String()
}
// todo: check current state existing
n, err := c.startNewNode(req.ForceNewCluster, req.ListenAddr, "", "")
n, err := c.startNewNode(req.ForceNewCluster, localAddr, "", net.JoinHostPort(listenHost, listenPort), net.JoinHostPort(advertiseHost, advertisePort), "", "")
if err != nil {
c.Unlock()
return "", err
@ -339,8 +447,23 @@ func (c *Cluster) Join(req types.JoinRequest) error {
c.Unlock()
return err
}
listenHost, listenPort, err := resolveListenAddr(req.ListenAddr)
if err != nil {
c.Unlock()
return err
}
var advertiseAddr string
advertiseHost, advertisePort, err := c.resolveAdvertiseAddr(req.AdvertiseAddr, listenPort)
// For joining, we don't need to provide an advertise address,
// since the remote side can detect it.
if err == nil {
advertiseAddr = net.JoinHostPort(advertiseHost, advertisePort)
}
// todo: check current state existing
n, err := c.startNewNode(false, req.ListenAddr, req.RemoteAddrs[0], req.JoinToken)
n, err := c.startNewNode(false, "", req.RemoteAddrs[0], net.JoinHostPort(listenHost, listenPort), advertiseAddr, req.RemoteAddrs[0], req.JoinToken)
if err != nil {
c.Unlock()
return err
@ -530,15 +653,22 @@ func (c *Cluster) IsAgent() bool {
return c.node != nil && c.ready
}
// GetListenAddress returns the listening address for current manager's
// consensus and dispatcher APIs.
func (c *Cluster) GetListenAddress() string {
// GetLocalAddress returns the local address.
func (c *Cluster) GetLocalAddress() string {
c.RLock()
defer c.RUnlock()
if c.isActiveManager() {
return c.listenAddr
return c.actualLocalAddr
}
// GetAdvertiseAddress returns the remotely reachable address of this node.
func (c *Cluster) GetAdvertiseAddress() string {
c.RLock()
defer c.RUnlock()
if c.advertiseAddr != "" {
advertiseHost, _, _ := net.SplitHostPort(c.advertiseAddr)
return advertiseHost
}
return ""
return c.actualLocalAddr
}
// GetRemoteAddress returns a known advertise address of a remote manager if
@ -572,7 +702,10 @@ func (c *Cluster) ListenClusterEvents() <-chan struct{} {
// Info returns information about the current cluster state.
func (c *Cluster) Info() types.Info {
var info types.Info
info := types.Info{
NodeAddr: c.GetAdvertiseAddress(),
}
c.RLock()
defer c.RUnlock()

View file

@ -0,0 +1,250 @@
package cluster
import (
"errors"
"fmt"
"net"
)
var (
errNoSuchInterface = errors.New("no such interface")
errMultipleIPs = errors.New("could not choose an IP address to advertise since this system has multiple addresses")
errNoIP = errors.New("could not find the system's IP address")
errMustSpecifyListenAddr = errors.New("must specify a listening address because the address to advertise is not recognized as a system address")
)
func resolveListenAddr(specifiedAddr string) (string, string, error) {
specifiedHost, specifiedPort, err := net.SplitHostPort(specifiedAddr)
if err != nil {
return "", "", fmt.Errorf("could not parse listen address %s", specifiedAddr)
}
// Does the host component match any of the interface names on the
// system? If so, use the address from that interface.
interfaceAddr, err := resolveInterfaceAddr(specifiedHost)
if err == nil {
return interfaceAddr.String(), specifiedPort, nil
}
if err != errNoSuchInterface {
return "", "", err
}
return specifiedHost, specifiedPort, nil
}
func (c *Cluster) resolveAdvertiseAddr(advertiseAddr, listenAddrPort string) (string, string, error) {
// Approach:
// - If an advertise address is specified, use that. Resolve the
// interface's address if an interface was specified in
// advertiseAddr. Fill in the port from listenAddrPort if necessary.
// - If DefaultAdvertiseAddr is not empty, use that with the port from
// listenAddrPort. Resolve the interface's address from
// if an interface name was specified in DefaultAdvertiseAddr.
// - Otherwise, try to autodetect the system's address. Use the port in
// listenAddrPort with this address if autodetection succeeds.
if advertiseAddr != "" {
advertiseHost, advertisePort, err := net.SplitHostPort(advertiseAddr)
if err != nil {
// Not a host:port specification
advertiseHost = advertiseAddr
advertisePort = listenAddrPort
}
// Does the host component match any of the interface names on the
// system? If so, use the address from that interface.
interfaceAddr, err := resolveInterfaceAddr(advertiseHost)
if err == nil {
return interfaceAddr.String(), advertisePort, nil
}
if err != errNoSuchInterface {
return "", "", err
}
return advertiseHost, advertisePort, nil
}
if c.config.DefaultAdvertiseAddr != "" {
// Does the default advertise address component match any of the
// interface names on the system? If so, use the address from
// that interface.
interfaceAddr, err := resolveInterfaceAddr(c.config.DefaultAdvertiseAddr)
if err == nil {
return interfaceAddr.String(), listenAddrPort, nil
}
if err != errNoSuchInterface {
return "", "", err
}
return c.config.DefaultAdvertiseAddr, listenAddrPort, nil
}
systemAddr, err := c.resolveSystemAddr()
if err != nil {
return "", "", err
}
return systemAddr.String(), listenAddrPort, nil
}
func resolveInterfaceAddr(specifiedInterface string) (net.IP, error) {
// Use a specific interface's IP address.
intf, err := net.InterfaceByName(specifiedInterface)
if err != nil {
return nil, errNoSuchInterface
}
addrs, err := intf.Addrs()
if err != nil {
return nil, err
}
var interfaceAddr4, interfaceAddr6 net.IP
for _, addr := range addrs {
ipAddr, ok := addr.(*net.IPNet)
if ok {
if ipAddr.IP.To4() != nil {
// IPv4
if interfaceAddr4 != nil {
return nil, fmt.Errorf("interface %s has more than one IPv4 address", specifiedInterface)
}
interfaceAddr4 = ipAddr.IP
} else {
// IPv6
if interfaceAddr6 != nil {
return nil, fmt.Errorf("interface %s has more than one IPv6 address", specifiedInterface)
}
interfaceAddr6 = ipAddr.IP
}
}
}
if interfaceAddr4 == nil && interfaceAddr6 == nil {
return nil, fmt.Errorf("interface %s has no usable IPv4 or IPv6 address", specifiedInterface)
}
// In the case that there's exactly one IPv4 address
// and exactly one IPv6 address, favor IPv4 over IPv6.
if interfaceAddr4 != nil {
return interfaceAddr4, nil
}
return interfaceAddr6, nil
}
func (c *Cluster) resolveSystemAddr() (net.IP, error) {
// Use the system's only IP address, or fail if there are
// multiple addresses to choose from.
interfaces, err := net.Interfaces()
if err != nil {
return nil, err
}
var systemAddr net.IP
// List Docker-managed subnets
v4Subnets := c.config.NetworkSubnetsProvider.V4Subnets()
v6Subnets := c.config.NetworkSubnetsProvider.V6Subnets()
ifaceLoop:
for _, intf := range interfaces {
// Skip inactive interfaces and loopback interfaces
if (intf.Flags&net.FlagUp == 0) || (intf.Flags&net.FlagLoopback) != 0 {
continue
}
addrs, err := intf.Addrs()
if err != nil {
continue
}
var interfaceAddr4, interfaceAddr6 net.IP
for _, addr := range addrs {
ipAddr, ok := addr.(*net.IPNet)
// Skip loopback and link-local addresses
if !ok || !ipAddr.IP.IsGlobalUnicast() {
continue
}
if ipAddr.IP.To4() != nil {
// IPv4
// Ignore addresses in subnets that are managed by Docker.
for _, subnet := range v4Subnets {
if subnet.Contains(ipAddr.IP) {
continue ifaceLoop
}
}
if interfaceAddr4 != nil {
return nil, errMultipleIPs
}
interfaceAddr4 = ipAddr.IP
} else {
// IPv6
// Ignore addresses in subnets that are managed by Docker.
for _, subnet := range v6Subnets {
if subnet.Contains(ipAddr.IP) {
continue ifaceLoop
}
}
if interfaceAddr6 != nil {
return nil, errMultipleIPs
}
interfaceAddr6 = ipAddr.IP
}
}
// In the case that this interface has exactly one IPv4 address
// and exactly one IPv6 address, favor IPv4 over IPv6.
if interfaceAddr4 != nil {
if systemAddr != nil {
return nil, errMultipleIPs
}
systemAddr = interfaceAddr4
} else if interfaceAddr6 != nil {
if systemAddr != nil {
return nil, errMultipleIPs
}
systemAddr = interfaceAddr6
}
}
if systemAddr == nil {
return nil, errNoIP
}
return systemAddr, nil
}
func listSystemIPs() []net.IP {
interfaces, err := net.Interfaces()
if err != nil {
return nil
}
var systemAddrs []net.IP
for _, intf := range interfaces {
addrs, err := intf.Addrs()
if err != nil {
continue
}
for _, addr := range addrs {
ipAddr, ok := addr.(*net.IPNet)
if ok {
systemAddrs = append(systemAddrs, ipAddr.IP)
}
}
}
return systemAddrs
}

View file

@ -127,6 +127,13 @@ type CommonConfig struct {
// Embedded structs that allow config
// deserialization without the full struct.
CommonTLSOptions
// SwarmDefaultAdvertiseAddr is the default host/IP or network interface
// to use if a wildcard address is specified in the ListenAddr value
// given to the /swarm/init endpoint and no advertise address is
// specified.
SwarmDefaultAdvertiseAddr string `json:"swarm-default-advertise-addr"`
LogConfig
bridgeConfig // bridgeConfig holds bridge network specific configuration.
registry.ServiceOptions
@ -167,6 +174,8 @@ func (config *Config) InstallCommonFlags(cmd *flag.FlagSet, usageFn func(string)
cmd.IntVar(&maxConcurrentDownloads, []string{"-max-concurrent-downloads"}, defaultMaxConcurrentDownloads, usageFn("Set the max concurrent downloads for each pull"))
cmd.IntVar(&maxConcurrentUploads, []string{"-max-concurrent-uploads"}, defaultMaxConcurrentUploads, usageFn("Set the max concurrent uploads for each push"))
cmd.StringVar(&config.SwarmDefaultAdvertiseAddr, []string{"-swarm-default-advertise-addr"}, "", usageFn("Set default address or interface for swarm advertised address"))
config.MaxConcurrentDownloads = &maxConcurrentDownloads
config.MaxConcurrentUploads = &maxConcurrentUploads
}

View file

@ -730,6 +730,42 @@ func (daemon *Daemon) Unmount(container *container.Container) error {
return nil
}
// V4Subnets returns the IPv4 subnets of networks that are managed by Docker.
func (daemon *Daemon) V4Subnets() []net.IPNet {
var subnets []net.IPNet
managedNetworks := daemon.netController.Networks()
for _, managedNetwork := range managedNetworks {
v4Infos, _ := managedNetwork.Info().IpamInfo()
for _, v4Info := range v4Infos {
if v4Info.IPAMData.Pool != nil {
subnets = append(subnets, *v4Info.IPAMData.Pool)
}
}
}
return subnets
}
// V6Subnets returns the IPv6 subnets of networks that are managed by Docker.
func (daemon *Daemon) V6Subnets() []net.IPNet {
var subnets []net.IPNet
managedNetworks := daemon.netController.Networks()
for _, managedNetwork := range managedNetworks {
_, v6Infos := managedNetwork.Info().IpamInfo()
for _, v6Info := range v6Infos {
if v6Info.IPAMData.Pool != nil {
subnets = append(subnets, *v6Info.IPAMData.Pool)
}
}
}
return subnets
}
func writeDistributionProgress(cancelFunc func(), outStream io.Writer, progressChan <-chan progress.Progress) {
progressOutput := streamformatter.NewJSONStreamFormatter().NewProgressOutput(outStream, false)
operationCancelled := false

View file

@ -3615,8 +3615,11 @@ Initialize a new Swarm
JSON Parameters:
- **ListenAddr** Listen address used for inter-manager communication, as well as determining.
the networking interface used for the VXLAN Tunnel Endpoint (VTEP).
- **ListenAddr** Listen address used for inter-manager communication, as well as determining
the networking interface used for the VXLAN Tunnel Endpoint (VTEP). This can either be an
address/port combination in the form `192.168.1.1:4567`, or an interface followed by a port
number, like `eth0:4567`. If the port number is omitted, the default swarm listening port is
used.
- **ForceNewCluster** Force creating a new Swarm even if already part of one.
- **Spec** Configuration settings of the new Swarm.
- **Orchestration** Configuration settings for the orchestration aspects of the Swarm.

View file

@ -69,6 +69,7 @@ Options:
-s, --storage-driver Storage driver to use
--selinux-enabled Enable selinux support
--storage-opt=[] Storage driver options
--swarm-default-advertise-addr Set default address or interface for swarm advertised address
--tls Use TLS; implied by --tlsverify
--tlscacert=~/.docker/ca.pem Trust certs signed only by this CA
--tlscert=~/.docker/cert.pem Path to TLS certificate file
@ -1042,6 +1043,7 @@ This is a full example of the allowed configuration options on Linux:
"tlscacert": "",
"tlscert": "",
"tlskey": "",
"swarm-default-advertise-addr": "",
"api-cors-header": "",
"selinux-enabled": false,
"userns-remap": "",
@ -1112,6 +1114,7 @@ This is a full example of the allowed configuration options on Windows:
"tlscacert": "",
"tlscert": "",
"tlskey": "",
"swarm-default-advertise-addr": "",
"group": "",
"default-ulimits": {},
"bridge": "",

View file

@ -18,12 +18,13 @@ Usage: docker swarm init [OPTIONS]
Initialize a swarm
Options:
--advertise-addr value Advertised address (format: <ip|hostname|interface>[:port])
--cert-expiry duration Validity period for node certificates (default 2160h0m0s)
--dispatcher-heartbeat duration Dispatcher heartbeat period (default 5s)
--external-ca value Specifications of one or more certificate signing endpoints
--force-new-cluster Force create a new cluster from current state.
--help Print usage
--listen-addr value Listen address (default 0.0.0.0:2377)
--listen-addr value Listen address (format: <ip|hostname|interface>[:port])
--task-history-limit int Task history retention limit (default 5)
```
@ -32,7 +33,7 @@ in the newly created one node swarm cluster.
```bash
$ docker swarm init --listen-addr 192.168.99.121:2377
$ docker swarm init --advertise-addr 192.168.99.121
Swarm initialized: current node (bvz81updecsj6wjz393c09vti) is now a manager.
To add a worker to this swarm, run the following command:
@ -71,11 +72,31 @@ The URL specifies the endpoint where signing requests should be submitted.
### `--force-new-cluster`
This flag forces an existing node that was part of a quorum that was lost to restart as a single node Manager without losing its data
This flag forces an existing node that was part of a quorum that was lost to restart as a single node Manager without losing its data.
### `--listen-addr value`
The node listens for inbound swarm manager traffic on this IP:PORT
The node listens for inbound Swarm manager traffic on this address. The default is to listen on
0.0.0.0:2377. It is also possible to specify a network interface to listen on that interface's
address; for example `--listen-addr eth0:2377`.
Specifying a port is optional. If the value is a bare IP address, hostname, or interface
name, the default port 2377 will be used.
### `--advertise-addr value`
This flag specifies the address that will be advertised to other members of the
swarm for API access and overlay networking. If unspecified, Docker will check
if the system has a single IP address, and use that IP address with with the
listening port (see `--listen-addr`). If the system has multiple IP addresses,
`--advertise-addr` must be specified so that the correct address is chosen for
inter-manager communication and overlay networking.
It is also possible to specify a network interface to advertise that interface's address;
for example `--advertise-addr eth0:2377`.
Specifying a port is optional. If the value is a bare IP address, hostname, or interface
name, the default port 2377 will be used.
### `--task-history-limit`

View file

@ -18,9 +18,10 @@ Usage: docker swarm join [OPTIONS] HOST:PORT
Join a swarm as a node and/or manager
Options:
--help Print usage
--listen-addr value Listen address (default 0.0.0.0:2377)
--token string Token for entry into the swarm
--advertise-addr value Advertised address (format: <ip|hostname|interface>[:port])
--help Print usage
--listen-addr value Listen address
--token string Token for entry into the swarm
```
Join a node to a swarm. The node joins as a manager node or worker node based upon the token you
@ -32,7 +33,7 @@ pass a worker token, the node joins as a worker.
The example below demonstrates joining a manager node using a manager token.
```bash
$ docker swarm join --token SWMTKN-1-3pu6hszjas19xyp7ghgosyx9k8atbfcr8p2is99znpy26u2lkl-7p73s1dx5in4tatdymyhg9hu2 --listen-addr 192.168.99.122:2377 192.168.99.121:2377
$ docker swarm join --token SWMTKN-1-3pu6hszjas19xyp7ghgosyx9k8atbfcr8p2is99znpy26u2lkl-7p73s1dx5in4tatdymyhg9hu2 192.168.99.121:2377
This node joined a swarm as a manager.
$ docker node ls
ID HOSTNAME STATUS AVAILABILITY MANAGER STATUS
@ -49,7 +50,7 @@ should join as workers instead. Managers should be stable hosts that have static
The example below demonstrates joining a worker node using a worker token.
```bash
$ docker swarm join --token SWMTKN-1-3pu6hszjas19xyp7ghgosyx9k8atbfcr8p2is99znpy26u2lkl-1awxwuwd3z9j1z3puu7rcgdbx --listen-addr 192.168.99.123:2377 192.168.99.121:2377
$ docker swarm join --token SWMTKN-1-3pu6hszjas19xyp7ghgosyx9k8atbfcr8p2is99znpy26u2lkl-1awxwuwd3z9j1z3puu7rcgdbx 192.168.99.121:2377
This node joined a swarm as a worker.
$ docker node ls
ID HOSTNAME STATUS AVAILABILITY MANAGER STATUS
@ -60,7 +61,36 @@ dvfxp4zseq4s0rih1selh0d20 * manager1 Ready Active Leader
### `--listen-addr value`
The node listens for inbound swarm manager traffic on this IP:PORT
If the node is a manager, it will listen for inbound Swarm manager traffic on this
address. The default is to listen on 0.0.0.0:2377. It is also possible to specify a
network interface to listen on that interface's address; for example `--listen-addr eth0:2377`.
Specifying a port is optional. If the value is a bare IP address, hostname, or interface
name, the default port 2377 will be used.
This flag is generally not necessary when joining an existing swarm.
### `--advertise-addr value`
This flag specifies the address that will be advertised to other members of the
swarm for API access. If unspecified, Docker will check if the system has a
single IP address, and use that IP address with with the listening port (see
`--listen-addr`). If the system has multiple IP addresses, `--advertise-addr`
must be specified so that the correct address is chosen for inter-manager
communication and overlay networking.
It is also possible to specify a network interface to advertise that interface's address;
for example `--advertise-addr eth0:2377`.
Specifying a port is optional. If the value is a bare IP address, hostname, or interface
name, the default port 2377 will be used.
This flag is generally not necessary when joining an existing swarm.
### `--manager`
Joins the node as a manager
>>>>>>> 22565e1... Split advertised address from listen address
### `--token string`

View file

@ -23,14 +23,14 @@ node. For example, the tutorial uses a machine named `manager1`.
2. Run the following command to create a new swarm:
```bash
docker swarm init --listen-addr <MANAGER-IP>:<PORT>
docker swarm init --advertise-addr <MANAGER-IP>
```
In the tutorial, the following command creates a swarm on the `manager1`
machine:
```bash
$ docker swarm init --listen-addr 192.168.99.100:2377
$ docker swarm init --advertise-addr 192.168.99.100
Swarm initialized: current node (dxn1zf6l61qsb1josjja83ngz) is now a manager.
To add a worker to this swarm, run the following command:
@ -44,9 +44,9 @@ node. For example, the tutorial uses a machine named `manager1`.
192.168.99.100:2377
```
The `--listen-addr` flag configures the manager node to listen on port
`2377`. The other nodes in the swarm must be able to access the manager at
the IP address.
The `--advertise-addr` flag configures the manager node to publish its
address as `192.168.99.100`. The other nodes in the swarm must be able
to access the manager at the IP address.
The output incudes the commands to join new nodes to the swarm. Nodes will
join as managers or workers depending on the value for the `--swarm-token`

View file

@ -211,7 +211,7 @@ func (s *DockerSwarmSuite) AddDaemon(c *check.C, joinSwarm, manager bool) *Swarm
port: defaultSwarmPort + s.portIndex,
}
d.listenAddr = fmt.Sprintf("0.0.0.0:%d", d.port)
err := d.StartWithBusybox("--iptables=false") // avoid networking conflicts
err := d.StartWithBusybox("--iptables=false", "--swarm-default-advertise-addr=lo") // avoid networking conflicts
c.Assert(err, check.IsNil)
if joinSwarm == true {

View file

@ -599,11 +599,9 @@ func (s *DockerSwarmSuite) TestApiSwarmLeaveOnPendingJoin(c *check.C) {
go d2.Join(swarm.JoinRequest{
RemoteAddrs: []string{"nosuchhost:1234"},
}) // will block on pending state
})
waitAndAssert(c, defaultReconciliationTimeout, d2.checkLocalNodeState, checker.Equals, swarm.LocalNodeStatePending)
c.Assert(d2.Leave(true), checker.IsNil)
waitAndAssert(c, defaultReconciliationTimeout, d2.checkLocalNodeState, checker.Equals, swarm.LocalNodeStateInactive)
waitAndAssert(c, defaultReconciliationTimeout, d2.checkActiveContainerCount, checker.Equals, 1)
@ -617,9 +615,9 @@ func (s *DockerSwarmSuite) TestApiSwarmRestoreOnPendingJoin(c *check.C) {
d := s.AddDaemon(c, false, false)
go d.Join(swarm.JoinRequest{
RemoteAddrs: []string{"nosuchhost:1234"},
}) // will block on pending state
})
waitAndAssert(c, defaultReconciliationTimeout, d.checkLocalNodeState, checker.Equals, swarm.LocalNodeStatePending)
waitAndAssert(c, defaultReconciliationTimeout, d.checkLocalNodeState, checker.Equals, swarm.LocalNodeStateInactive)
c.Assert(d.Stop(), checker.IsNil)
c.Assert(d.Start(), checker.IsNil)

View file

@ -55,6 +55,7 @@ dockerd - Enable daemon mode
[**-s**|**--storage-driver**[=*STORAGE-DRIVER*]]
[**--selinux-enabled**]
[**--storage-opt**[=*[]*]]
[**--swarm-default-advertise-addr**[=*IP|HOSTNAME|INTERFACE*]]
[**--tls**]
[**--tlscacert**[=*~/.docker/ca.pem*]]
[**--tlscert**[=*~/.docker/cert.pem*]]
@ -239,6 +240,11 @@ output otherwise.
**--storage-opt**=[]
Set storage driver options. See STORAGE DRIVER OPTIONS.
**--swarm-default-advertise-addr**=*IP|HOSTNAME|INTERFACE*
Set default address or interface for swarm to advertise as its externally-reachable address to other cluster
members. This can be a hostname, an IP address, or an interface such as `eth0`. A port cannot be specified with
this option.
**--tls**=*true*|*false*
Use TLS; implied by --tlsverify. Default is false.