Sfoglia il codice sorgente

Update libnetwork vendoring

Update now that the changes in https://github.com/docker/libnetwork/pull/1230
have been merged

Signed-off-by: Justin Cormack <justin.cormack@docker.com>
Justin Cormack 9 anni fa
parent
commit
3ae0c66450

+ 1 - 1
hack/vendor.sh

@@ -65,7 +65,7 @@ clone git github.com/RackSec/srslog 259aed10dfa74ea2961eddd1d9847619f6e98837
 clone git github.com/imdario/mergo 0.2.1
 
 #get libnetwork packages
-clone git github.com/docker/libnetwork 377a7337f2387cce3be1df7a4503446147b68ff1
+clone git github.com/docker/libnetwork 6eece7dcc21dcd34d907f3e91dd71cb8640b661c
 clone git github.com/docker/go-events 39718a26497694185f8fb58a7d6f31947f3dc42d
 clone git github.com/armon/go-radix e39d623f12e8e41c7b5529e9a9dd67a1e2261f80
 clone git github.com/armon/go-metrics eb0af217e5e9747e41dd5303755356b62d28e3ec

+ 5 - 2
vendor/src/github.com/docker/libnetwork/Makefile

@@ -7,6 +7,7 @@ docker = docker run --rm -it ${dockerargs} $$EXTRA_ARGS ${container_env} ${build
 ciargs = -e CIRCLECI -e "COVERALLS_TOKEN=$$COVERALLS_TOKEN" -e "INSIDECONTAINER=-incontainer=true"
 cidocker = docker run ${dockerargs} ${ciargs} $$EXTRA_ARGS ${container_env} ${build_image}
 CROSS_PLATFORMS = linux/amd64 linux/386 linux/arm windows/amd64
+export PATH := $(CURDIR)/bin:$(PATH)
 
 all: ${build_image}.created build check integration-tests clean
 
@@ -24,10 +25,11 @@ build: ${build_image}.created
 build-local:
 	@mkdir -p "bin"
 	$(shell which godep) go build -tags experimental -o "bin/dnet" ./cmd/dnet
+	$(shell which godep) go build -o "bin/docker-proxy" ./cmd/proxy
 
 clean:
 	@if [ -d bin ]; then \
-		echo "Removing dnet binaries"; \
+		echo "Removing dnet and proxy binaries"; \
 		rm -rf bin; \
 	fi
 
@@ -41,6 +43,7 @@ cross: ${build_image}.created
 
 cross-local:
 	$(shell which godep) go build -o "bin/dnet-$$GOOS-$$GOARCH" ./cmd/dnet
+	$(shell which godep) go build -o "bin/docker-proxy-$$GOOS-$$GOARCH" ./cmd/proxy
 
 check: ${build_image}.created
 	@${docker} ./wrapmake.sh check-local
@@ -102,4 +105,4 @@ circle-ci-check: ${build_image}.created
 circle-ci-build: ${build_image}.created
 	@${cidocker} make build-local
 
-circle-ci: circle-ci-check circle-ci-cross circle-ci-build integration-tests
+circle-ci: circle-ci-build circle-ci-check circle-ci-cross integration-tests

+ 67 - 0
vendor/src/github.com/docker/libnetwork/cmd/proxy/main.go

@@ -0,0 +1,67 @@
+package main
+
+import (
+	"flag"
+	"fmt"
+	"log"
+	"net"
+	"os"
+	"os/signal"
+	"syscall"
+)
+
+func main() {
+	f := os.NewFile(3, "signal-parent")
+	host, container := parseHostContainerAddrs()
+
+	p, err := NewProxy(host, container)
+	if err != nil {
+		fmt.Fprintf(f, "1\n%s", err)
+		f.Close()
+		os.Exit(1)
+	}
+	go handleStopSignals(p)
+	fmt.Fprint(f, "0\n")
+	f.Close()
+
+	// Run will block until the proxy stops
+	p.Run()
+}
+
+// parseHostContainerAddrs parses the flags passed on reexec to create the TCP or UDP
+// net.Addrs to map the host and container ports
+func parseHostContainerAddrs() (host net.Addr, container net.Addr) {
+	var (
+		proto         = flag.String("proto", "tcp", "proxy protocol")
+		hostIP        = flag.String("host-ip", "", "host ip")
+		hostPort      = flag.Int("host-port", -1, "host port")
+		containerIP   = flag.String("container-ip", "", "container ip")
+		containerPort = flag.Int("container-port", -1, "container port")
+	)
+
+	flag.Parse()
+
+	switch *proto {
+	case "tcp":
+		host = &net.TCPAddr{IP: net.ParseIP(*hostIP), Port: *hostPort}
+		container = &net.TCPAddr{IP: net.ParseIP(*containerIP), Port: *containerPort}
+	case "udp":
+		host = &net.UDPAddr{IP: net.ParseIP(*hostIP), Port: *hostPort}
+		container = &net.UDPAddr{IP: net.ParseIP(*containerIP), Port: *containerPort}
+	default:
+		log.Fatalf("unsupported protocol %s", *proto)
+	}
+
+	return host, container
+}
+
+func handleStopSignals(p Proxy) {
+	s := make(chan os.Signal, 10)
+	signal.Notify(s, os.Interrupt, syscall.SIGTERM)
+
+	for range s {
+		p.Close()
+
+		os.Exit(0)
+	}
+}

+ 37 - 0
vendor/src/github.com/docker/libnetwork/cmd/proxy/proxy.go

@@ -0,0 +1,37 @@
+// docker-proxy provides a network Proxy interface and implementations for TCP
+// and UDP.
+package main
+
+import (
+	"fmt"
+	"net"
+)
+
+// Proxy defines the behavior of a proxy. It forwards traffic back and forth
+// between two endpoints : the frontend and the backend.
+// It can be used to do software port-mapping between two addresses.
+// e.g. forward all traffic between the frontend (host) 127.0.0.1:3000
+// to the backend (container) at 172.17.42.108:4000.
+type Proxy interface {
+	// Run starts forwarding traffic back and forth between the front
+	// and back-end addresses.
+	Run()
+	// Close stops forwarding traffic and close both ends of the Proxy.
+	Close()
+	// FrontendAddr returns the address on which the proxy is listening.
+	FrontendAddr() net.Addr
+	// BackendAddr returns the proxied address.
+	BackendAddr() net.Addr
+}
+
+// NewProxy creates a Proxy according to the specified frontendAddr and backendAddr.
+func NewProxy(frontendAddr, backendAddr net.Addr) (Proxy, error) {
+	switch frontendAddr.(type) {
+	case *net.UDPAddr:
+		return NewUDPProxy(frontendAddr.(*net.UDPAddr), backendAddr.(*net.UDPAddr))
+	case *net.TCPAddr:
+		return NewTCPProxy(frontendAddr.(*net.TCPAddr), backendAddr.(*net.TCPAddr))
+	default:
+		panic(fmt.Errorf("Unsupported protocol"))
+	}
+}

+ 31 - 0
vendor/src/github.com/docker/libnetwork/cmd/proxy/stub_proxy.go

@@ -0,0 +1,31 @@
+package main
+
+import (
+	"net"
+)
+
+// StubProxy is a proxy that is a stub (does nothing).
+type StubProxy struct {
+	frontendAddr net.Addr
+	backendAddr  net.Addr
+}
+
+// Run does nothing.
+func (p *StubProxy) Run() {}
+
+// Close does nothing.
+func (p *StubProxy) Close() {}
+
+// FrontendAddr returns the frontend address.
+func (p *StubProxy) FrontendAddr() net.Addr { return p.frontendAddr }
+
+// BackendAddr returns the backend address.
+func (p *StubProxy) BackendAddr() net.Addr { return p.backendAddr }
+
+// NewStubProxy creates a new StubProxy
+func NewStubProxy(frontendAddr, backendAddr net.Addr) (Proxy, error) {
+	return &StubProxy{
+		frontendAddr: frontendAddr,
+		backendAddr:  backendAddr,
+	}, nil
+}

+ 96 - 0
vendor/src/github.com/docker/libnetwork/cmd/proxy/tcp_proxy.go

@@ -0,0 +1,96 @@
+package main
+
+import (
+	"io"
+	"net"
+	"sync"
+	"syscall"
+
+	"github.com/Sirupsen/logrus"
+)
+
+// TCPProxy is a proxy for TCP connections. It implements the Proxy interface to
+// handle TCP traffic forwarding between the frontend and backend addresses.
+type TCPProxy struct {
+	listener     *net.TCPListener
+	frontendAddr *net.TCPAddr
+	backendAddr  *net.TCPAddr
+}
+
+// NewTCPProxy creates a new TCPProxy.
+func NewTCPProxy(frontendAddr, backendAddr *net.TCPAddr) (*TCPProxy, error) {
+	listener, err := net.ListenTCP("tcp", frontendAddr)
+	if err != nil {
+		return nil, err
+	}
+	// If the port in frontendAddr was 0 then ListenTCP will have a picked
+	// a port to listen on, hence the call to Addr to get that actual port:
+	return &TCPProxy{
+		listener:     listener,
+		frontendAddr: listener.Addr().(*net.TCPAddr),
+		backendAddr:  backendAddr,
+	}, nil
+}
+
+func (proxy *TCPProxy) clientLoop(client *net.TCPConn, quit chan bool) {
+	backend, err := net.DialTCP("tcp", nil, proxy.backendAddr)
+	if err != nil {
+		logrus.Printf("Can't forward traffic to backend tcp/%v: %s\n", proxy.backendAddr, err)
+		client.Close()
+		return
+	}
+
+	var wg sync.WaitGroup
+	var broker = func(to, from *net.TCPConn) {
+		if _, err := io.Copy(to, from); err != nil {
+			// If the socket we are writing to is shutdown with
+			// SHUT_WR, forward it to the other end of the pipe:
+			if err, ok := err.(*net.OpError); ok && err.Err == syscall.EPIPE {
+				from.CloseWrite()
+			}
+		}
+		to.CloseRead()
+		wg.Done()
+	}
+
+	wg.Add(2)
+	go broker(client, backend)
+	go broker(backend, client)
+
+	finish := make(chan struct{})
+	go func() {
+		wg.Wait()
+		close(finish)
+	}()
+
+	select {
+	case <-quit:
+	case <-finish:
+	}
+	client.Close()
+	backend.Close()
+	<-finish
+}
+
+// Run starts forwarding the traffic using TCP.
+func (proxy *TCPProxy) Run() {
+	quit := make(chan bool)
+	defer close(quit)
+	for {
+		client, err := proxy.listener.Accept()
+		if err != nil {
+			logrus.Printf("Stopping proxy on tcp/%v for tcp/%v (%s)", proxy.frontendAddr, proxy.backendAddr, err)
+			return
+		}
+		go proxy.clientLoop(client.(*net.TCPConn), quit)
+	}
+}
+
+// Close stops forwarding the traffic.
+func (proxy *TCPProxy) Close() { proxy.listener.Close() }
+
+// FrontendAddr returns the TCP address on which the proxy is listening.
+func (proxy *TCPProxy) FrontendAddr() net.Addr { return proxy.frontendAddr }
+
+// BackendAddr returns the TCP proxied address.
+func (proxy *TCPProxy) BackendAddr() net.Addr { return proxy.backendAddr }

+ 169 - 0
vendor/src/github.com/docker/libnetwork/cmd/proxy/udp_proxy.go

@@ -0,0 +1,169 @@
+package main
+
+import (
+	"encoding/binary"
+	"net"
+	"strings"
+	"sync"
+	"syscall"
+	"time"
+
+	"github.com/Sirupsen/logrus"
+)
+
+const (
+	// UDPConnTrackTimeout is the timeout used for UDP connection tracking
+	UDPConnTrackTimeout = 90 * time.Second
+	// UDPBufSize is the buffer size for the UDP proxy
+	UDPBufSize = 65507
+)
+
+// A net.Addr where the IP is split into two fields so you can use it as a key
+// in a map:
+type connTrackKey struct {
+	IPHigh uint64
+	IPLow  uint64
+	Port   int
+}
+
+func newConnTrackKey(addr *net.UDPAddr) *connTrackKey {
+	if len(addr.IP) == net.IPv4len {
+		return &connTrackKey{
+			IPHigh: 0,
+			IPLow:  uint64(binary.BigEndian.Uint32(addr.IP)),
+			Port:   addr.Port,
+		}
+	}
+	return &connTrackKey{
+		IPHigh: binary.BigEndian.Uint64(addr.IP[:8]),
+		IPLow:  binary.BigEndian.Uint64(addr.IP[8:]),
+		Port:   addr.Port,
+	}
+}
+
+type connTrackMap map[connTrackKey]*net.UDPConn
+
+// UDPProxy is proxy for which handles UDP datagrams. It implements the Proxy
+// interface to handle UDP traffic forwarding between the frontend and backend
+// addresses.
+type UDPProxy struct {
+	listener       *net.UDPConn
+	frontendAddr   *net.UDPAddr
+	backendAddr    *net.UDPAddr
+	connTrackTable connTrackMap
+	connTrackLock  sync.Mutex
+}
+
+// NewUDPProxy creates a new UDPProxy.
+func NewUDPProxy(frontendAddr, backendAddr *net.UDPAddr) (*UDPProxy, error) {
+	listener, err := net.ListenUDP("udp", frontendAddr)
+	if err != nil {
+		return nil, err
+	}
+	return &UDPProxy{
+		listener:       listener,
+		frontendAddr:   listener.LocalAddr().(*net.UDPAddr),
+		backendAddr:    backendAddr,
+		connTrackTable: make(connTrackMap),
+	}, nil
+}
+
+func (proxy *UDPProxy) replyLoop(proxyConn *net.UDPConn, clientAddr *net.UDPAddr, clientKey *connTrackKey) {
+	defer func() {
+		proxy.connTrackLock.Lock()
+		delete(proxy.connTrackTable, *clientKey)
+		proxy.connTrackLock.Unlock()
+		proxyConn.Close()
+	}()
+
+	readBuf := make([]byte, UDPBufSize)
+	for {
+		proxyConn.SetReadDeadline(time.Now().Add(UDPConnTrackTimeout))
+	again:
+		read, err := proxyConn.Read(readBuf)
+		if err != nil {
+			if err, ok := err.(*net.OpError); ok && err.Err == syscall.ECONNREFUSED {
+				// This will happen if the last write failed
+				// (e.g: nothing is actually listening on the
+				// proxied port on the container), ignore it
+				// and continue until UDPConnTrackTimeout
+				// expires:
+				goto again
+			}
+			return
+		}
+		for i := 0; i != read; {
+			written, err := proxy.listener.WriteToUDP(readBuf[i:read], clientAddr)
+			if err != nil {
+				return
+			}
+			i += written
+		}
+	}
+}
+
+// Run starts forwarding the traffic using UDP.
+func (proxy *UDPProxy) Run() {
+	readBuf := make([]byte, UDPBufSize)
+	for {
+		read, from, err := proxy.listener.ReadFromUDP(readBuf)
+		if err != nil {
+			// NOTE: Apparently ReadFrom doesn't return
+			// ECONNREFUSED like Read do (see comment in
+			// UDPProxy.replyLoop)
+			if !isClosedError(err) {
+				logrus.Printf("Stopping proxy on udp/%v for udp/%v (%s)", proxy.frontendAddr, proxy.backendAddr, err)
+			}
+			break
+		}
+
+		fromKey := newConnTrackKey(from)
+		proxy.connTrackLock.Lock()
+		proxyConn, hit := proxy.connTrackTable[*fromKey]
+		if !hit {
+			proxyConn, err = net.DialUDP("udp", nil, proxy.backendAddr)
+			if err != nil {
+				logrus.Printf("Can't proxy a datagram to udp/%s: %s\n", proxy.backendAddr, err)
+				proxy.connTrackLock.Unlock()
+				continue
+			}
+			proxy.connTrackTable[*fromKey] = proxyConn
+			go proxy.replyLoop(proxyConn, from, fromKey)
+		}
+		proxy.connTrackLock.Unlock()
+		for i := 0; i != read; {
+			written, err := proxyConn.Write(readBuf[i:read])
+			if err != nil {
+				logrus.Printf("Can't proxy a datagram to udp/%s: %s\n", proxy.backendAddr, err)
+				break
+			}
+			i += written
+		}
+	}
+}
+
+// Close stops forwarding the traffic.
+func (proxy *UDPProxy) Close() {
+	proxy.listener.Close()
+	proxy.connTrackLock.Lock()
+	defer proxy.connTrackLock.Unlock()
+	for _, conn := range proxy.connTrackTable {
+		conn.Close()
+	}
+}
+
+// FrontendAddr returns the UDP address on which the proxy is listening.
+func (proxy *UDPProxy) FrontendAddr() net.Addr { return proxy.frontendAddr }
+
+// BackendAddr returns the proxied UDP address.
+func (proxy *UDPProxy) BackendAddr() net.Addr { return proxy.backendAddr }
+
+func isClosedError(err error) bool {
+	/* This comparison is ugly, but unfortunately, net.go doesn't export errClosing.
+	 * See:
+	 * http://golang.org/src/pkg/net/net.go
+	 * https://code.google.com/p/go/issues/detail?id=4337
+	 * https://groups.google.com/forum/#!msg/golang-nuts/0_aaCvBmOcM/SptmDyX1XJMJ
+	 */
+	return strings.HasSuffix(err.Error(), "use of closed network connection")
+}

+ 5 - 0
vendor/src/github.com/docker/libnetwork/controller.go

@@ -193,6 +193,11 @@ func New(cfgOptions ...config.Option) (NetworkController, error) {
 			return nil, err
 		}
 	}
+
+	if err = initIPAMDrivers(drvRegistry, nil, c.getStore(datastore.GlobalScope)); err != nil {
+		return nil, err
+	}
+
 	c.drvRegistry = drvRegistry
 
 	if c.cfg != nil && c.cfg.Cluster.Watcher != nil {

+ 1 - 1
vendor/src/github.com/docker/libnetwork/drivers/bridge/bridge.go

@@ -330,7 +330,7 @@ func (c *networkConfiguration) conflictsWithNetworks(id string, others []*bridge
 		// bridges. This could not be completely caught by the config conflict
 		// check, because networks which config does not specify the AddressIPv4
 		// get their address and subnet selected by the driver (see electBridgeIPv4())
-		if c.AddressIPv4 != nil {
+		if c.AddressIPv4 != nil && nwBridge.bridgeIPv4 != nil {
 			if nwBridge.bridgeIPv4.Contains(c.AddressIPv4.IP) ||
 				c.AddressIPv4.Contains(nwBridge.bridgeIPv4.IP) {
 				return types.ForbiddenErrorf("conflicts with network %s (%s) by ip network", nwID, nwConfig.BridgeName)

+ 23 - 0
vendor/src/github.com/docker/libnetwork/drivers_ipam.go

@@ -0,0 +1,23 @@
+package libnetwork
+
+import (
+	"github.com/docker/libnetwork/drvregistry"
+	"github.com/docker/libnetwork/ipamapi"
+	builtinIpam "github.com/docker/libnetwork/ipams/builtin"
+	nullIpam "github.com/docker/libnetwork/ipams/null"
+	remoteIpam "github.com/docker/libnetwork/ipams/remote"
+)
+
+func initIPAMDrivers(r *drvregistry.DrvRegistry, lDs, gDs interface{}) error {
+	for _, fn := range [](func(ipamapi.Callback, interface{}, interface{}) error){
+		builtinIpam.Init,
+		remoteIpam.Init,
+		nullIpam.Init,
+	} {
+		if err := fn(r, lDs, gDs); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}

+ 0 - 22
vendor/src/github.com/docker/libnetwork/drvregistry/drvregistry.go

@@ -8,10 +8,6 @@ import (
 	"github.com/docker/libnetwork/driverapi"
 	"github.com/docker/libnetwork/ipamapi"
 	"github.com/docker/libnetwork/types"
-
-	builtinIpam "github.com/docker/libnetwork/ipams/builtin"
-	nullIpam "github.com/docker/libnetwork/ipams/null"
-	remoteIpam "github.com/docker/libnetwork/ipams/remote"
 )
 
 type driverData struct {
@@ -64,10 +60,6 @@ func New(lDs, gDs interface{}, dfn DriverNotifyFunc, ifn IPAMNotifyFunc) (*DrvRe
 		ifn:         ifn,
 	}
 
-	if err := r.initIPAMs(lDs, gDs); err != nil {
-		return nil, err
-	}
-
 	return r, nil
 }
 
@@ -157,20 +149,6 @@ func (r *DrvRegistry) IPAMDefaultAddressSpaces(name string) (string, string, err
 	return i.defaultLocalAddressSpace, i.defaultGlobalAddressSpace, nil
 }
 
-func (r *DrvRegistry) initIPAMs(lDs, gDs interface{}) error {
-	for _, fn := range [](func(ipamapi.Callback, interface{}, interface{}) error){
-		builtinIpam.Init,
-		remoteIpam.Init,
-		nullIpam.Init,
-	} {
-		if err := fn(r, nil, gDs); err != nil {
-			return err
-		}
-	}
-
-	return nil
-}
-
 // RegisterDriver registers the network driver when it gets discovered.
 func (r *DrvRegistry) RegisterDriver(ntype string, driver driverapi.Driver, capability driverapi.Capability) error {
 	if strings.TrimSpace(ntype) == "" {

+ 15 - 6
vendor/src/github.com/docker/libnetwork/etchosts/etchosts.go

@@ -119,25 +119,34 @@ func Add(path string, recs []Record) error {
 		return nil
 	}
 
-	f, err := os.Open(path)
+	b, err := mergeRecords(path, recs)
 	if err != nil {
 		return err
 	}
 
-	content := bytes.NewBuffer(nil)
+	return ioutil.WriteFile(path, b, 0644)
+}
 
-	_, err = content.ReadFrom(f)
+func mergeRecords(path string, recs []Record) ([]byte, error) {
+	f, err := os.Open(path)
 	if err != nil {
-		return err
+		return nil, err
+	}
+	defer f.Close()
+
+	content := bytes.NewBuffer(nil)
+
+	if _, err := content.ReadFrom(f); err != nil {
+		return nil, err
 	}
 
 	for _, r := range recs {
 		if _, err := r.WriteTo(content); err != nil {
-			return err
+			return nil, err
 		}
 	}
 
-	return ioutil.WriteFile(path, content.Bytes(), 0644)
+	return content.Bytes(), nil
 }
 
 // Delete deletes an arbitrary number of Records already existing in /etc/hosts file

+ 1 - 1
vendor/src/github.com/docker/libnetwork/networkdb/cluster.go

@@ -14,7 +14,7 @@ import (
 	"github.com/hashicorp/memberlist"
 )
 
-const reapInterval = 2 * time.Second
+const reapInterval = 30 * time.Second
 
 type logWriter struct{}
 

+ 5 - 0
vendor/src/github.com/docker/libnetwork/networkdb/delegate.go

@@ -130,6 +130,11 @@ func (nDB *NetworkDB) handleTableMessage(buf []byte, isBulkSync bool) {
 		return
 	}
 
+	// Ignore messages that this node generated.
+	if tEvent.NodeName == nDB.config.NodeName {
+		return
+	}
+
 	// Do not rebroadcast a bulk sync
 	if rebroadcast := nDB.handleTableEvent(&tEvent); rebroadcast && !isBulkSync {
 		var err error

+ 8 - 2
vendor/src/github.com/docker/libnetwork/portmapper/mapper.go

@@ -90,7 +90,10 @@ func (pm *PortMapper) MapRange(container net.Addr, hostIP net.IP, hostPortStart,
 		}
 
 		if useProxy {
-			m.userlandProxy = newProxy(proto, hostIP, allocatedHostPort, container.(*net.TCPAddr).IP, container.(*net.TCPAddr).Port)
+			m.userlandProxy, err = newProxy(proto, hostIP, allocatedHostPort, container.(*net.TCPAddr).IP, container.(*net.TCPAddr).Port)
+			if err != nil {
+				return nil, err
+			}
 		} else {
 			m.userlandProxy = newDummyProxy(proto, hostIP, allocatedHostPort)
 		}
@@ -107,7 +110,10 @@ func (pm *PortMapper) MapRange(container net.Addr, hostIP net.IP, hostPortStart,
 		}
 
 		if useProxy {
-			m.userlandProxy = newProxy(proto, hostIP, allocatedHostPort, container.(*net.UDPAddr).IP, container.(*net.UDPAddr).Port)
+			m.userlandProxy, err = newProxy(proto, hostIP, allocatedHostPort, container.(*net.UDPAddr).IP, container.(*net.UDPAddr).Port)
+			if err != nil {
+				return nil, err
+			}
 		} else {
 			m.userlandProxy = newDummyProxy(proto, hostIP, allocatedHostPort)
 		}

+ 2 - 2
vendor/src/github.com/docker/libnetwork/portmapper/mock_proxy.go

@@ -2,8 +2,8 @@ package portmapper
 
 import "net"
 
-func newMockProxyCommand(proto string, hostIP net.IP, hostPort int, containerIP net.IP, containerPort int) userlandProxy {
-	return &mockProxyCommand{}
+func newMockProxyCommand(proto string, hostIP net.IP, hostPort int, containerIP net.IP, containerPort int) (userlandProxy, error) {
+	return &mockProxyCommand{}, nil
 }
 
 type mockProxyCommand struct {

+ 6 - 67
vendor/src/github.com/docker/libnetwork/portmapper/proxy.go

@@ -1,29 +1,19 @@
 package portmapper
 
 import (
-	"flag"
 	"fmt"
 	"io"
 	"io/ioutil"
-	"log"
 	"net"
 	"os"
 	"os/exec"
-	"os/signal"
 	"strconv"
 	"syscall"
 	"time"
-
-	"github.com/docker/docker/pkg/proxy"
-	"github.com/docker/docker/pkg/reexec"
 )
 
 const userlandProxyCommandName = "docker-proxy"
 
-func init() {
-	reexec.Register(userlandProxyCommandName, execProxy)
-}
-
 type userlandProxy interface {
 	Start() error
 	Stop() error
@@ -35,66 +25,15 @@ type proxyCommand struct {
 	cmd *exec.Cmd
 }
 
-// execProxy is the reexec function that is registered to start the userland proxies
-func execProxy() {
-	f := os.NewFile(3, "signal-parent")
-	host, container := parseHostContainerAddrs()
+func newProxyCommand(proto string, hostIP net.IP, hostPort int, containerIP net.IP, containerPort int) (userlandProxy, error) {
+	cmd, err := exec.LookPath(userlandProxyCommandName)
 
-	p, err := proxy.NewProxy(host, container)
 	if err != nil {
-		fmt.Fprintf(f, "1\n%s", err)
-		f.Close()
-		os.Exit(1)
-	}
-	go handleStopSignals(p)
-	fmt.Fprint(f, "0\n")
-	f.Close()
-
-	// Run will block until the proxy stops
-	p.Run()
-}
-
-// parseHostContainerAddrs parses the flags passed on reexec to create the TCP or UDP
-// net.Addrs to map the host and container ports
-func parseHostContainerAddrs() (host net.Addr, container net.Addr) {
-	var (
-		proto         = flag.String("proto", "tcp", "proxy protocol")
-		hostIP        = flag.String("host-ip", "", "host ip")
-		hostPort      = flag.Int("host-port", -1, "host port")
-		containerIP   = flag.String("container-ip", "", "container ip")
-		containerPort = flag.Int("container-port", -1, "container port")
-	)
-
-	flag.Parse()
-
-	switch *proto {
-	case "tcp":
-		host = &net.TCPAddr{IP: net.ParseIP(*hostIP), Port: *hostPort}
-		container = &net.TCPAddr{IP: net.ParseIP(*containerIP), Port: *containerPort}
-	case "udp":
-		host = &net.UDPAddr{IP: net.ParseIP(*hostIP), Port: *hostPort}
-		container = &net.UDPAddr{IP: net.ParseIP(*containerIP), Port: *containerPort}
-	default:
-		log.Fatalf("unsupported protocol %s", *proto)
-	}
-
-	return host, container
-}
-
-func handleStopSignals(p proxy.Proxy) {
-	s := make(chan os.Signal, 10)
-	signal.Notify(s, os.Interrupt, syscall.SIGTERM, syscall.SIGSTOP)
-
-	for range s {
-		p.Close()
-
-		os.Exit(0)
+		return nil, err
 	}
-}
 
-func newProxyCommand(proto string, hostIP net.IP, hostPort int, containerIP net.IP, containerPort int) userlandProxy {
 	args := []string{
-		userlandProxyCommandName,
+		cmd,
 		"-proto", proto,
 		"-host-ip", hostIP.String(),
 		"-host-port", strconv.Itoa(hostPort),
@@ -104,13 +43,13 @@ func newProxyCommand(proto string, hostIP net.IP, hostPort int, containerIP net.
 
 	return &proxyCommand{
 		cmd: &exec.Cmd{
-			Path: reexec.Self(),
+			Path: cmd,
 			Args: args,
 			SysProcAttr: &syscall.SysProcAttr{
 				Pdeathsig: syscall.SIGTERM, // send a sigterm to the proxy if the daemon process dies
 			},
 		},
-	}
+	}, nil
 }
 
 func (p *proxyCommand) Start() error {

+ 9 - 1
vendor/src/github.com/docker/libnetwork/sandbox_dns_unix.go

@@ -275,7 +275,15 @@ func (sb *sandbox) updateDNS(ipv6Enabled bool) error {
 	if err != nil {
 		return err
 	}
-	if err = ioutil.WriteFile(tmpHashFile.Name(), []byte(newRC.Hash), filePerm); err != nil {
+	if err = tmpHashFile.Chmod(filePerm); err != nil {
+		tmpHashFile.Close()
+		return err
+	}
+	_, err = tmpHashFile.Write([]byte(newRC.Hash))
+	if err1 := tmpHashFile.Close(); err == nil {
+		err = err1
+	}
+	if err != nil {
 		return err
 	}
 	return os.Rename(tmpHashFile.Name(), hashFile)

+ 2 - 0
vendor/src/github.com/docker/libnetwork/sandbox_externalkey_unix.go

@@ -135,6 +135,8 @@ func (c *controller) acceptClientConnections(sock string, l net.Listener) {
 			continue
 		}
 		go func() {
+			defer conn.Close()
+
 			err := c.processExternalKey(conn)
 			ret := success
 			if err != nil {