diff --git a/container.go b/container.go index 6898ada24f..a0f14ed810 100644 --- a/container.go +++ b/container.go @@ -7,6 +7,7 @@ import ( "github.com/dotcloud/docker/archive" "github.com/dotcloud/docker/execdriver" "github.com/dotcloud/docker/graphdriver" + "github.com/dotcloud/docker/networkdriver/ipallocator" "github.com/dotcloud/docker/pkg/mount" "github.com/dotcloud/docker/pkg/term" "github.com/dotcloud/docker/utils" @@ -1041,8 +1042,9 @@ func (container *Container) allocateNetwork() error { manager: manager, } if iface != nil && iface.IPNet.IP != nil { - ipNum := ipToInt(iface.IPNet.IP) - manager.ipAllocator.inUse[ipNum] = struct{}{} + if _, err := ipallocator.RequestIP(manager.bridgeNetwork, &iface.IPNet.IP); err != nil { + return err + } } else { iface, err = container.runtime.networkManager.Allocate() if err != nil { diff --git a/network.go b/network.go index 1139f6ad37..7414b348a2 100644 --- a/network.go +++ b/network.go @@ -1,9 +1,8 @@ package docker import ( - "encoding/binary" - "errors" "fmt" + "github.com/dotcloud/docker/networkdriver/ipallocator" "github.com/dotcloud/docker/pkg/iptables" "github.com/dotcloud/docker/pkg/netlink" "github.com/dotcloud/docker/proxy" @@ -25,76 +24,6 @@ const ( siocBRADDBR = 0x89a0 ) -// Calculates the first and last IP addresses in an IPNet -func networkRange(network *net.IPNet) (net.IP, net.IP) { - netIP := network.IP.To4() - firstIP := netIP.Mask(network.Mask) - lastIP := net.IPv4(0, 0, 0, 0).To4() - for i := 0; i < len(lastIP); i++ { - lastIP[i] = netIP[i] | ^network.Mask[i] - } - return firstIP, lastIP -} - -// Detects overlap between one IPNet and another -func networkOverlaps(netX *net.IPNet, netY *net.IPNet) bool { - firstIP, _ := networkRange(netX) - if netY.Contains(firstIP) { - return true - } - firstIP, _ = networkRange(netY) - if netX.Contains(firstIP) { - return true - } - return false -} - -// Converts a 4 bytes IP into a 32 bit integer -func ipToInt(ip net.IP) int32 { - return int32(binary.BigEndian.Uint32(ip.To4())) -} - -// Converts 32 bit integer into a 4 bytes IP address -func intToIP(n int32) net.IP { - b := make([]byte, 4) - binary.BigEndian.PutUint32(b, uint32(n)) - return net.IP(b) -} - -// Given a netmask, calculates the number of available hosts -func networkSize(mask net.IPMask) int32 { - m := net.IPv4Mask(0, 0, 0, 0) - for i := 0; i < net.IPv4len; i++ { - m[i] = ^mask[i] - } - - return int32(binary.BigEndian.Uint32(m)) + 1 -} - -func checkRouteOverlaps(networks []netlink.Route, dockerNetwork *net.IPNet) error { - for _, network := range networks { - if network.IPNet != nil && networkOverlaps(dockerNetwork, network.IPNet) { - return fmt.Errorf("Network %s is already routed: '%s'", dockerNetwork, network) - } - } - return nil -} - -func checkNameserverOverlaps(nameservers []string, dockerNetwork *net.IPNet) error { - if len(nameservers) > 0 { - for _, ns := range nameservers { - _, nsNetwork, err := net.ParseCIDR(ns) - if err != nil { - return err - } - if networkOverlaps(dockerNetwork, nsNetwork) { - return fmt.Errorf("%s overlaps nameserver %s", dockerNetwork, nsNetwork) - } - } - } - return nil -} - // CreateBridgeIface creates a network bridge interface on the host system with the name `ifaceName`, // and attempts to configure it with an address which doesn't conflict with any other interface on the host. // If it can't find an address which doesn't conflict, it will return an error. @@ -131,10 +60,13 @@ func CreateBridgeIface(config *DaemonConfig) error { var ifaceAddr string if len(config.BridgeIp) != 0 { - _, _, err := net.ParseCIDR(config.BridgeIp) + _, dockerNetwork, err := net.ParseCIDR(config.BridgeIp) if err != nil { return err } + if err := ipallocator.RegisterNetwork(dockerNetwork, nameservers); err != nil { + return err + } ifaceAddr = config.BridgeIp } else { for _, addr := range addrs { @@ -142,20 +74,16 @@ func CreateBridgeIface(config *DaemonConfig) error { if err != nil { return err } - routes, err := netlink.NetworkGetRoutes() - if err != nil { - return err - } - if err := checkRouteOverlaps(routes, dockerNetwork); err == nil { - if err := checkNameserverOverlaps(nameservers, dockerNetwork); err == nil { - ifaceAddr = addr - break - } + + if err := ipallocator.RegisterNetwork(dockerNetwork, nameservers); err == nil { + ifaceAddr = addr + break } else { utils.Debugf("%s: %s", addr, err) } } } + if ifaceAddr == "" { return fmt.Errorf("Could not find a free IP address range for interface '%s'. Please configure its address manually and run 'docker -b %s'", config.BridgeIface, config.BridgeIface) } @@ -426,115 +354,6 @@ func newPortAllocator() (*PortAllocator, error) { return allocator, nil } -// IP allocator: Automatically allocate and release networking ports -type IPAllocator struct { - network *net.IPNet - queueAlloc chan allocatedIP - queueReleased chan net.IP - inUse map[int32]struct{} - quit chan bool -} - -type allocatedIP struct { - ip net.IP - err error -} - -func (alloc *IPAllocator) run() { - firstIP, _ := networkRange(alloc.network) - ipNum := ipToInt(firstIP) - ownIP := ipToInt(alloc.network.IP) - size := networkSize(alloc.network.Mask) - - pos := int32(1) - max := size - 2 // -1 for the broadcast address, -1 for the gateway address - for { - var ( - newNum int32 - inUse bool - ) - - // Find first unused IP, give up after one whole round - for attempt := int32(0); attempt < max; attempt++ { - newNum = ipNum + pos - - pos = pos%max + 1 - - // The network's IP is never okay to use - if newNum == ownIP { - continue - } - - if _, inUse = alloc.inUse[newNum]; !inUse { - // We found an unused IP - break - } - } - - ip := allocatedIP{ip: intToIP(newNum)} - if inUse { - ip.err = errors.New("No unallocated IP available") - } - - select { - case quit := <-alloc.quit: - if quit { - return - } - case alloc.queueAlloc <- ip: - alloc.inUse[newNum] = struct{}{} - case released := <-alloc.queueReleased: - r := ipToInt(released) - delete(alloc.inUse, r) - - if inUse { - // If we couldn't allocate a new IP, the released one - // will be the only free one now, so instantly use it - // next time - pos = r - ipNum - } else { - // Use same IP as last time - if pos == 1 { - pos = max - } else { - pos-- - } - } - } - } -} - -func (alloc *IPAllocator) Acquire() (net.IP, error) { - ip := <-alloc.queueAlloc - return ip.ip, ip.err -} - -func (alloc *IPAllocator) Release(ip net.IP) { - alloc.queueReleased <- ip -} - -func (alloc *IPAllocator) Close() error { - alloc.quit <- true - close(alloc.quit) - close(alloc.queueAlloc) - close(alloc.queueReleased) - return nil -} - -func newIPAllocator(network *net.IPNet) *IPAllocator { - alloc := &IPAllocator{ - network: network, - queueAlloc: make(chan allocatedIP), - queueReleased: make(chan net.IP), - inUse: make(map[int32]struct{}), - quit: make(chan bool), - } - - go alloc.run() - - return alloc -} - // Network interface represents the networking stack of a container type NetworkInterface struct { IPNet net.IPNet @@ -639,7 +458,9 @@ func (iface *NetworkInterface) Release() { } } - iface.manager.ipAllocator.Release(iface.IPNet.IP) + if err := ipallocator.ReleaseIP(iface.manager.bridgeNetwork, &iface.IPNet.IP); err != nil { + log.Printf("Unable to release ip %s\n", err) + } } // Network Manager manages a set of network interfaces @@ -648,7 +469,6 @@ type NetworkManager struct { bridgeIface string bridgeNetwork *net.IPNet - ipAllocator *IPAllocator tcpPortAllocator *PortAllocator udpPortAllocator *PortAllocator portMapper *PortMapper @@ -663,27 +483,31 @@ func (manager *NetworkManager) Allocate() (*NetworkInterface, error) { return &NetworkInterface{disabled: true}, nil } - var ip net.IP + var ip *net.IP var err error - ip, err = manager.ipAllocator.Acquire() + ip, err = ipallocator.RequestIP(manager.bridgeNetwork, nil) if err != nil { return nil, err } - // avoid duplicate IP - ipNum := ipToInt(ip) - firstIP := manager.ipAllocator.network.IP.To4().Mask(manager.ipAllocator.network.Mask) - firstIPNum := ipToInt(firstIP) + 1 - if firstIPNum == ipNum { - ip, err = manager.ipAllocator.Acquire() - if err != nil { - return nil, err + // TODO: @crosbymichael why are we doing this ? + /* + // avoid duplicate IP + ipNum := ipToInt(ip) + firstIP := manager.ipAllocator.network.IP.To4().Mask(manager.ipAllocator.network.Mask) + firstIPNum := ipToInt(firstIP) + 1 + + if firstIPNum == ipNum { + ip, err = manager.ipAllocator.Acquire() + if err != nil { + return nil, err + } } - } + */ iface := &NetworkInterface{ - IPNet: net.IPNet{IP: ip, Mask: manager.bridgeNetwork.Mask}, + IPNet: net.IPNet{IP: *ip, Mask: manager.bridgeNetwork.Mask}, Gateway: manager.bridgeNetwork.IP, manager: manager, } @@ -696,14 +520,13 @@ func (manager *NetworkManager) Close() error { } err1 := manager.tcpPortAllocator.Close() err2 := manager.udpPortAllocator.Close() - err3 := manager.ipAllocator.Close() if err1 != nil { return err1 } if err2 != nil { return err2 } - return err3 + return nil } func newNetworkManager(config *DaemonConfig) (*NetworkManager, error) { @@ -714,6 +537,7 @@ func newNetworkManager(config *DaemonConfig) (*NetworkManager, error) { return manager, nil } + var network *net.IPNet addr, err := getIfaceAddr(config.BridgeIface) if err != nil { // If the iface is not found, try to create it @@ -724,8 +548,13 @@ func newNetworkManager(config *DaemonConfig) (*NetworkManager, error) { if err != nil { return nil, err } + network = addr.(*net.IPNet) + } else { + network = addr.(*net.IPNet) + if err := ipallocator.RegisterExistingNetwork(network); err != nil { + return nil, err + } } - network := addr.(*net.IPNet) // Configure iptables for link support if config.EnableIptables { @@ -790,8 +619,6 @@ func newNetworkManager(config *DaemonConfig) (*NetworkManager, error) { } } - ipAllocator := newIPAllocator(network) - tcpPortAllocator, err := newPortAllocator() if err != nil { return nil, err @@ -810,7 +637,6 @@ func newNetworkManager(config *DaemonConfig) (*NetworkManager, error) { manager := &NetworkManager{ bridgeIface: config.BridgeIface, bridgeNetwork: network, - ipAllocator: ipAllocator, tcpPortAllocator: tcpPortAllocator, udpPortAllocator: udpPortAllocator, portMapper: portMapper, diff --git a/network_test.go b/network_test.go index 0b6857ba76..0d25ccb158 100644 --- a/network_test.go +++ b/network_test.go @@ -2,9 +2,7 @@ package docker import ( "github.com/dotcloud/docker/pkg/iptables" - "github.com/dotcloud/docker/pkg/netlink" "github.com/dotcloud/docker/proxy" - "net" "testing" ) @@ -53,277 +51,6 @@ func TestPortAllocation(t *testing.T) { } } -func TestNetworkRange(t *testing.T) { - // Simple class C test - _, network, _ := net.ParseCIDR("192.168.0.1/24") - first, last := networkRange(network) - if !first.Equal(net.ParseIP("192.168.0.0")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("192.168.0.255")) { - t.Error(last.String()) - } - if size := networkSize(network.Mask); size != 256 { - t.Error(size) - } - - // Class A test - _, network, _ = net.ParseCIDR("10.0.0.1/8") - first, last = networkRange(network) - if !first.Equal(net.ParseIP("10.0.0.0")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("10.255.255.255")) { - t.Error(last.String()) - } - if size := networkSize(network.Mask); size != 16777216 { - t.Error(size) - } - - // Class A, random IP address - _, network, _ = net.ParseCIDR("10.1.2.3/8") - first, last = networkRange(network) - if !first.Equal(net.ParseIP("10.0.0.0")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("10.255.255.255")) { - t.Error(last.String()) - } - - // 32bit mask - _, network, _ = net.ParseCIDR("10.1.2.3/32") - first, last = networkRange(network) - if !first.Equal(net.ParseIP("10.1.2.3")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("10.1.2.3")) { - t.Error(last.String()) - } - if size := networkSize(network.Mask); size != 1 { - t.Error(size) - } - - // 31bit mask - _, network, _ = net.ParseCIDR("10.1.2.3/31") - first, last = networkRange(network) - if !first.Equal(net.ParseIP("10.1.2.2")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("10.1.2.3")) { - t.Error(last.String()) - } - if size := networkSize(network.Mask); size != 2 { - t.Error(size) - } - - // 26bit mask - _, network, _ = net.ParseCIDR("10.1.2.3/26") - first, last = networkRange(network) - if !first.Equal(net.ParseIP("10.1.2.0")) { - t.Error(first.String()) - } - if !last.Equal(net.ParseIP("10.1.2.63")) { - t.Error(last.String()) - } - if size := networkSize(network.Mask); size != 64 { - t.Error(size) - } -} - -func TestConversion(t *testing.T) { - ip := net.ParseIP("127.0.0.1") - i := ipToInt(ip) - if i == 0 { - t.Fatal("converted to zero") - } - conv := intToIP(i) - if !ip.Equal(conv) { - t.Error(conv.String()) - } -} - -func TestIPAllocator(t *testing.T) { - expectedIPs := []net.IP{ - 0: net.IPv4(127, 0, 0, 2), - 1: net.IPv4(127, 0, 0, 3), - 2: net.IPv4(127, 0, 0, 4), - 3: net.IPv4(127, 0, 0, 5), - 4: net.IPv4(127, 0, 0, 6), - } - - gwIP, n, _ := net.ParseCIDR("127.0.0.1/29") - alloc := newIPAllocator(&net.IPNet{IP: gwIP, Mask: n.Mask}) - // Pool after initialisation (f = free, u = used) - // 2(f) - 3(f) - 4(f) - 5(f) - 6(f) - // ↑ - - // Check that we get 5 IPs, from 127.0.0.2–127.0.0.6, in that - // order. - for i := 0; i < 5; i++ { - ip, err := alloc.Acquire() - if err != nil { - t.Fatal(err) - } - - assertIPEquals(t, expectedIPs[i], ip) - } - // Before loop begin - // 2(f) - 3(f) - 4(f) - 5(f) - 6(f) - // ↑ - - // After i = 0 - // 2(u) - 3(f) - 4(f) - 5(f) - 6(f) - // ↑ - - // After i = 1 - // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) - // ↑ - - // After i = 2 - // 2(u) - 3(u) - 4(u) - 5(f) - 6(f) - // ↑ - - // After i = 3 - // 2(u) - 3(u) - 4(u) - 5(u) - 6(f) - // ↑ - - // After i = 4 - // 2(u) - 3(u) - 4(u) - 5(u) - 6(u) - // ↑ - - // Check that there are no more IPs - _, err := alloc.Acquire() - if err == nil { - t.Fatal("There shouldn't be any IP addresses at this point") - } - - // Release some IPs in non-sequential order - alloc.Release(expectedIPs[3]) - // 2(u) - 3(u) - 4(u) - 5(f) - 6(u) - // ↑ - - alloc.Release(expectedIPs[2]) - // 2(u) - 3(u) - 4(f) - 5(f) - 6(u) - // ↑ - - alloc.Release(expectedIPs[4]) - // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) - // ↑ - - // Make sure that IPs are reused in sequential order, starting - // with the first released IP - newIPs := make([]net.IP, 3) - for i := 0; i < 3; i++ { - ip, err := alloc.Acquire() - if err != nil { - t.Fatal(err) - } - - newIPs[i] = ip - } - // Before loop begin - // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) - // ↑ - - // After i = 0 - // 2(u) - 3(u) - 4(f) - 5(u) - 6(f) - // ↑ - - // After i = 1 - // 2(u) - 3(u) - 4(f) - 5(u) - 6(u) - // ↑ - - // After i = 2 - // 2(u) - 3(u) - 4(u) - 5(u) - 6(u) - // ↑ - - assertIPEquals(t, expectedIPs[3], newIPs[0]) - assertIPEquals(t, expectedIPs[4], newIPs[1]) - assertIPEquals(t, expectedIPs[2], newIPs[2]) - - _, err = alloc.Acquire() - if err == nil { - t.Fatal("There shouldn't be any IP addresses at this point") - } -} - -func assertIPEquals(t *testing.T, ip1, ip2 net.IP) { - if !ip1.Equal(ip2) { - t.Fatalf("Expected IP %s, got %s", ip1, ip2) - } -} - -func AssertOverlap(CIDRx string, CIDRy string, t *testing.T) { - _, netX, _ := net.ParseCIDR(CIDRx) - _, netY, _ := net.ParseCIDR(CIDRy) - if !networkOverlaps(netX, netY) { - t.Errorf("%v and %v should overlap", netX, netY) - } -} - -func AssertNoOverlap(CIDRx string, CIDRy string, t *testing.T) { - _, netX, _ := net.ParseCIDR(CIDRx) - _, netY, _ := net.ParseCIDR(CIDRy) - if networkOverlaps(netX, netY) { - t.Errorf("%v and %v should not overlap", netX, netY) - } -} - -func TestNetworkOverlaps(t *testing.T) { - //netY starts at same IP and ends within netX - AssertOverlap("172.16.0.1/24", "172.16.0.1/25", t) - //netY starts within netX and ends at same IP - AssertOverlap("172.16.0.1/24", "172.16.0.128/25", t) - //netY starts and ends within netX - AssertOverlap("172.16.0.1/24", "172.16.0.64/25", t) - //netY starts at same IP and ends outside of netX - AssertOverlap("172.16.0.1/24", "172.16.0.1/23", t) - //netY starts before and ends at same IP of netX - AssertOverlap("172.16.1.1/24", "172.16.0.1/23", t) - //netY starts before and ends outside of netX - AssertOverlap("172.16.1.1/24", "172.16.0.1/23", t) - //netY starts and ends before netX - AssertNoOverlap("172.16.1.1/25", "172.16.0.1/24", t) - //netX starts and ends before netY - AssertNoOverlap("172.16.1.1/25", "172.16.2.1/24", t) -} - -func TestCheckRouteOverlaps(t *testing.T) { - routesData := []string{"10.0.2.0/32", "10.0.3.0/24", "10.0.42.0/24", "172.16.42.0/24", "192.168.142.0/24"} - - routes := []netlink.Route{} - for _, addr := range routesData { - _, netX, _ := net.ParseCIDR(addr) - routes = append(routes, netlink.Route{IPNet: netX}) - } - - _, netX, _ := net.ParseCIDR("172.16.0.1/24") - if err := checkRouteOverlaps(routes, netX); err != nil { - t.Fatal(err) - } - - _, netX, _ = net.ParseCIDR("10.0.2.0/24") - if err := checkRouteOverlaps(routes, netX); err == nil { - t.Fatalf("10.0.2.0/24 and 10.0.2.0 should overlap but it doesn't") - } -} - -func TestCheckNameserverOverlaps(t *testing.T) { - nameservers := []string{"10.0.2.3/32", "192.168.102.1/32"} - - _, netX, _ := net.ParseCIDR("10.0.2.3/32") - - if err := checkNameserverOverlaps(nameservers, netX); err == nil { - t.Fatalf("%s should overlap 10.0.2.3/32 but doesn't", netX) - } - - _, netX, _ = net.ParseCIDR("192.168.102.2/32") - - if err := checkNameserverOverlaps(nameservers, netX); err != nil { - t.Fatalf("%s should not overlap %v but it does", netX, nameservers) - } -} - type StubProxy struct { frontendAddr *net.Addr backendAddr *net.Addr diff --git a/networkdriver/ipallocator/allocator.go b/networkdriver/ipallocator/allocator.go new file mode 100644 index 0000000000..cca8cdb05a --- /dev/null +++ b/networkdriver/ipallocator/allocator.go @@ -0,0 +1,293 @@ +package ipallocator + +import ( + "encoding/binary" + "errors" + "github.com/dotcloud/docker/pkg/netlink" + "net" + "sync" +) + +type networkSet map[iPNet]*iPSet + +type iPNet struct { + IP string + Mask string +} + +var ( + ErrNetworkAlreadyAllocated = errors.New("requested network overlaps with existing network") + ErrNetworkAlreadyRegisterd = errors.New("requested network is already registered") + ErrNetworkOverlapsWithNameservers = errors.New("requested network overlaps with nameserver") + ErrNoAvailableIPs = errors.New("no available ip addresses on network") + ErrIPAlreadyAllocated = errors.New("ip already allocated") + ErrNetworkNotRegistered = errors.New("network not registered") + + lock = sync.Mutex{} + allocatedIPs = networkSet{} + availableIPS = networkSet{} +) + +// RegisterNetwork registers a new network with the allocator +// and validates that it contains a valid ip that does not overlap +// with existing routes and nameservers +func RegisterNetwork(network *net.IPNet, nameservers []string) error { + lock.Lock() + defer lock.Unlock() + + if err := checkExistingNetworkOverlaps(network); err != nil { + return err + } + + routes, err := netlink.NetworkGetRoutes() + if err != nil { + return err + } + + if err := checkRouteOverlaps(routes, network); err != nil { + return err + } + + if err := checkNameserverOverlaps(nameservers, network); err != nil { + return err + } + return RegisterExistingNetwork(network) +} + +// RegisterExistingNetwork registers an exising network created +// for use with the allocator but does not perform any validation +func RegisterExistingNetwork(network *net.IPNet) error { + n := newIPNet(network) + + if _, exists := allocatedIPs[n]; !exists { + allocatedIPs[n] = &iPSet{} + } + if _, exists := availableIPS[n]; !exists { + availableIPS[n] = &iPSet{} + } + + return nil +} + +// RequestIP requests an available ip from the given network. It +// will return the next available ip if the ip provided is nil. If the +// ip provided is not nil it will validate that the provided ip is available +// for use or return an error +func RequestIP(network *net.IPNet, ip *net.IP) (*net.IP, error) { + lock.Lock() + defer lock.Unlock() + + if !networkExists(network) { + return nil, ErrNetworkNotRegistered + } + + if ip == nil { + next, err := getNextIp(network) + if err != nil { + return nil, err + } + return next, nil + } + + if err := registerIP(network, ip); err != nil { + return nil, err + } + return ip, nil +} + +// ReleaseIP adds the provided ip back into the pool of +// available ips to be returned for use. +func ReleaseIP(network *net.IPNet, ip *net.IP) error { + lock.Lock() + defer lock.Unlock() + + if !networkExists(network) { + return ErrNetworkNotRegistered + } + + var ( + n = newIPNet(network) + existing = allocatedIPs[n] + available = availableIPS[n] + pos = getPosition(network, ip) + ) + + existing.Remove(int(pos)) + available.Push(int(pos)) + + return nil +} + +// convert the ip into the position in the subnet. Only +// position are saved in the set +func getPosition(network *net.IPNet, ip *net.IP) int32 { + var ( + first, _ = networkRange(network) + base = ipToInt(&first) + i = ipToInt(ip) + ) + return i - base +} + +// return an available ip if one is currently available. If not, +// return the next available ip for the nextwork +func getNextIp(network *net.IPNet) (*net.IP, error) { + var ( + n = newIPNet(network) + ownIP = ipToInt(&network.IP) + available = availableIPS[n] + allocated = allocatedIPs[n] + first, _ = networkRange(network) + base = ipToInt(&first) + size = int(networkSize(network.Mask)) + max = int32(size - 2) // size -1 for the broadcast address, -1 for the gateway address + pos = int32(available.Pop()) + ) + + // We pop and push the position not the ip + if pos != 0 { + ip := intToIP(int32(base + pos)) + allocated.Push(int(pos)) + + return ip, nil + } + + pos = int32(allocated.PullBack()) + for i := int32(0); i < max; i++ { + pos = pos%max + 1 + next := int32(base + pos) + + if next == ownIP { + continue + } + + if !allocated.Exists(int(pos)) { + ip := intToIP(next) + allocated.Push(int(pos)) + return ip, nil + } + } + return nil, ErrNoAvailableIPs +} + +func registerIP(network *net.IPNet, ip *net.IP) error { + var ( + n = newIPNet(network) + existing = allocatedIPs[n] + available = availableIPS[n] + pos = getPosition(network, ip) + ) + + if existing.Exists(int(pos)) { + return ErrIPAlreadyAllocated + } + available.Remove(int(pos)) + + return nil +} + +func checkRouteOverlaps(networks []netlink.Route, toCheck *net.IPNet) error { + for _, network := range networks { + if network.IPNet != nil && networkOverlaps(toCheck, network.IPNet) { + return ErrNetworkAlreadyAllocated + } + } + return nil +} + +// Detects overlap between one IPNet and another +func networkOverlaps(netX *net.IPNet, netY *net.IPNet) bool { + if firstIP, _ := networkRange(netX); netY.Contains(firstIP) { + return true + } + if firstIP, _ := networkRange(netY); netX.Contains(firstIP) { + return true + } + return false +} + +func checkExistingNetworkOverlaps(network *net.IPNet) error { + for existing := range allocatedIPs { + if newIPNet(network) == existing { + return ErrNetworkAlreadyRegisterd + } + + ex := newNetIPNet(existing) + if networkOverlaps(network, ex) { + return ErrNetworkAlreadyAllocated + } + } + return nil +} + +// Calculates the first and last IP addresses in an IPNet +func networkRange(network *net.IPNet) (net.IP, net.IP) { + var ( + netIP = network.IP.To4() + firstIP = netIP.Mask(network.Mask) + lastIP = net.IPv4(0, 0, 0, 0).To4() + ) + + for i := 0; i < len(lastIP); i++ { + lastIP[i] = netIP[i] | ^network.Mask[i] + } + return firstIP, lastIP +} + +func newIPNet(network *net.IPNet) iPNet { + return iPNet{ + IP: string(network.IP), + Mask: string(network.Mask), + } +} + +func newNetIPNet(network iPNet) *net.IPNet { + return &net.IPNet{ + IP: []byte(network.IP), + Mask: []byte(network.Mask), + } +} + +// Converts a 4 bytes IP into a 32 bit integer +func ipToInt(ip *net.IP) int32 { + return int32(binary.BigEndian.Uint32(ip.To4())) +} + +// Converts 32 bit integer into a 4 bytes IP address +func intToIP(n int32) *net.IP { + b := make([]byte, 4) + binary.BigEndian.PutUint32(b, uint32(n)) + ip := net.IP(b) + return &ip +} + +// Given a netmask, calculates the number of available hosts +func networkSize(mask net.IPMask) int32 { + m := net.IPv4Mask(0, 0, 0, 0) + for i := 0; i < net.IPv4len; i++ { + m[i] = ^mask[i] + } + + return int32(binary.BigEndian.Uint32(m)) + 1 +} + +func checkNameserverOverlaps(nameservers []string, toCheck *net.IPNet) error { + if len(nameservers) > 0 { + for _, ns := range nameservers { + _, nsNetwork, err := net.ParseCIDR(ns) + if err != nil { + return err + } + if networkOverlaps(toCheck, nsNetwork) { + return ErrNetworkOverlapsWithNameservers + } + } + } + return nil +} + +func networkExists(network *net.IPNet) bool { + n := newIPNet(network) + _, exists := allocatedIPs[n] + return exists +} diff --git a/networkdriver/ipallocator/allocator_test.go b/networkdriver/ipallocator/allocator_test.go new file mode 100644 index 0000000000..f574dfda70 --- /dev/null +++ b/networkdriver/ipallocator/allocator_test.go @@ -0,0 +1,476 @@ +package ipallocator + +import ( + "fmt" + "github.com/dotcloud/docker/pkg/netlink" + "net" + "testing" +) + +func reset() { + allocatedIPs = networkSet{} + availableIPS = networkSet{} +} + +func TestRegisterNetwork(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + n := newIPNet(network) + if _, exists := allocatedIPs[n]; !exists { + t.Fatal("IPNet should exist in allocated IPs") + } + + if _, exists := availableIPS[n]; !exists { + t.Fatal("IPNet should exist in available IPs") + } +} + +func TestRegisterTwoNetworks(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + network2 := &net.IPNet{ + IP: []byte{10, 1, 42, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network2, nil); err != nil { + t.Fatal(err) + } +} + +func TestRegisterNetworkThatExists(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + if err := RegisterNetwork(network, nil); err != ErrNetworkAlreadyRegisterd { + t.Fatalf("Expected error of %s got %s", ErrNetworkAlreadyRegisterd, err) + } +} + +func TestRequestNewIps(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + for i := 2; i < 10; i++ { + ip, err := RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + if expected := fmt.Sprintf("192.168.0.%d", i); ip.String() != expected { + t.Fatalf("Expected ip %s got %s", expected, ip.String()) + } + } +} + +func TestReleaseIp(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + ip, err := RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + if err := ReleaseIP(network, ip); err != nil { + t.Fatal(err) + } +} + +func TestGetReleasedIp(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + ip, err := RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + value := ip.String() + if err := ReleaseIP(network, ip); err != nil { + t.Fatal(err) + } + + ip, err = RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + if ip.String() != value { + t.Fatalf("Expected to receive same ip %s got %s", value, ip.String()) + } +} + +func TestRequesetSpecificIp(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + + ip := net.ParseIP("192.168.1.5") + + if _, err := RequestIP(network, &ip); err != nil { + t.Fatal(err) + } +} + +func TestNonOverlapingNameservers(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + nameservers := []string{ + "127.0.0.1/32", + } + + if err := RegisterNetwork(network, nameservers); err != nil { + t.Fatal(err) + } +} + +func TestOverlapingNameservers(t *testing.T) { + defer reset() + network := &net.IPNet{ + IP: []byte{192, 168, 0, 1}, + Mask: []byte{255, 255, 255, 0}, + } + nameservers := []string{ + "192.168.0.1/32", + } + + if err := RegisterNetwork(network, nameservers); err != ErrNetworkOverlapsWithNameservers { + t.Fatalf("Expectecd error of %s got %s", ErrNetworkOverlapsWithNameservers, err) + } +} + +func TestNetworkRange(t *testing.T) { + // Simple class C test + _, network, _ := net.ParseCIDR("192.168.0.1/24") + first, last := networkRange(network) + if !first.Equal(net.ParseIP("192.168.0.0")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("192.168.0.255")) { + t.Error(last.String()) + } + if size := networkSize(network.Mask); size != 256 { + t.Error(size) + } + + // Class A test + _, network, _ = net.ParseCIDR("10.0.0.1/8") + first, last = networkRange(network) + if !first.Equal(net.ParseIP("10.0.0.0")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("10.255.255.255")) { + t.Error(last.String()) + } + if size := networkSize(network.Mask); size != 16777216 { + t.Error(size) + } + + // Class A, random IP address + _, network, _ = net.ParseCIDR("10.1.2.3/8") + first, last = networkRange(network) + if !first.Equal(net.ParseIP("10.0.0.0")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("10.255.255.255")) { + t.Error(last.String()) + } + + // 32bit mask + _, network, _ = net.ParseCIDR("10.1.2.3/32") + first, last = networkRange(network) + if !first.Equal(net.ParseIP("10.1.2.3")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("10.1.2.3")) { + t.Error(last.String()) + } + if size := networkSize(network.Mask); size != 1 { + t.Error(size) + } + + // 31bit mask + _, network, _ = net.ParseCIDR("10.1.2.3/31") + first, last = networkRange(network) + if !first.Equal(net.ParseIP("10.1.2.2")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("10.1.2.3")) { + t.Error(last.String()) + } + if size := networkSize(network.Mask); size != 2 { + t.Error(size) + } + + // 26bit mask + _, network, _ = net.ParseCIDR("10.1.2.3/26") + first, last = networkRange(network) + if !first.Equal(net.ParseIP("10.1.2.0")) { + t.Error(first.String()) + } + if !last.Equal(net.ParseIP("10.1.2.63")) { + t.Error(last.String()) + } + if size := networkSize(network.Mask); size != 64 { + t.Error(size) + } +} + +func TestConversion(t *testing.T) { + ip := net.ParseIP("127.0.0.1") + i := ipToInt(&ip) + if i == 0 { + t.Fatal("converted to zero") + } + conv := intToIP(i) + if !ip.Equal(*conv) { + t.Error(conv.String()) + } +} + +func TestIPAllocator(t *testing.T) { + expectedIPs := []net.IP{ + 0: net.IPv4(127, 0, 0, 2), + 1: net.IPv4(127, 0, 0, 3), + 2: net.IPv4(127, 0, 0, 4), + 3: net.IPv4(127, 0, 0, 5), + 4: net.IPv4(127, 0, 0, 6), + } + + gwIP, n, _ := net.ParseCIDR("127.0.0.1/29") + network := &net.IPNet{IP: gwIP, Mask: n.Mask} + if err := RegisterNetwork(network, nil); err != nil { + t.Fatal(err) + } + // Pool after initialisation (f = free, u = used) + // 2(f) - 3(f) - 4(f) - 5(f) - 6(f) + // ↑ + + // Check that we get 5 IPs, from 127.0.0.2–127.0.0.6, in that + // order. + for i := 0; i < 5; i++ { + ip, err := RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + assertIPEquals(t, &expectedIPs[i], ip) + } + // Before loop begin + // 2(f) - 3(f) - 4(f) - 5(f) - 6(f) + // ↑ + + // After i = 0 + // 2(u) - 3(f) - 4(f) - 5(f) - 6(f) + // ↑ + + // After i = 1 + // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) + // ↑ + + // After i = 2 + // 2(u) - 3(u) - 4(u) - 5(f) - 6(f) + // ↑ + + // After i = 3 + // 2(u) - 3(u) - 4(u) - 5(u) - 6(f) + // ↑ + + // After i = 4 + // 2(u) - 3(u) - 4(u) - 5(u) - 6(u) + // ↑ + + // Check that there are no more IPs + ip, err := RequestIP(network, nil) + if err == nil { + t.Fatalf("There shouldn't be any IP addresses at this point, got %s\n", ip) + } + + // Release some IPs in non-sequential order + if err := ReleaseIP(network, &expectedIPs[3]); err != nil { + t.Fatal(err) + } + // 2(u) - 3(u) - 4(u) - 5(f) - 6(u) + // ↑ + + if err := ReleaseIP(network, &expectedIPs[2]); err != nil { + t.Fatal(err) + } + // 2(u) - 3(u) - 4(f) - 5(f) - 6(u) + // ↑ + + if err := ReleaseIP(network, &expectedIPs[4]); err != nil { + t.Fatal(err) + } + // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) + // ↑ + + // Make sure that IPs are reused in sequential order, starting + // with the first released IP + newIPs := make([]*net.IP, 3) + for i := 0; i < 3; i++ { + ip, err := RequestIP(network, nil) + if err != nil { + t.Fatal(err) + } + + newIPs[i] = ip + } + // Before loop begin + // 2(u) - 3(u) - 4(f) - 5(f) - 6(f) + // ↑ + + // After i = 0 + // 2(u) - 3(u) - 4(f) - 5(u) - 6(f) + // ↑ + + // After i = 1 + // 2(u) - 3(u) - 4(f) - 5(u) - 6(u) + // ↑ + + // After i = 2 + // 2(u) - 3(u) - 4(u) - 5(u) - 6(u) + // ↑ + + // Reordered these because the new set will always return the + // lowest ips first and not in the order that they were released + assertIPEquals(t, &expectedIPs[2], newIPs[0]) + assertIPEquals(t, &expectedIPs[3], newIPs[1]) + assertIPEquals(t, &expectedIPs[4], newIPs[2]) + + _, err = RequestIP(network, nil) + if err == nil { + t.Fatal("There shouldn't be any IP addresses at this point") + } +} + +func assertIPEquals(t *testing.T, ip1, ip2 *net.IP) { + if !ip1.Equal(*ip2) { + t.Fatalf("Expected IP %s, got %s", ip1, ip2) + } +} + +func AssertOverlap(CIDRx string, CIDRy string, t *testing.T) { + _, netX, _ := net.ParseCIDR(CIDRx) + _, netY, _ := net.ParseCIDR(CIDRy) + if !networkOverlaps(netX, netY) { + t.Errorf("%v and %v should overlap", netX, netY) + } +} + +func AssertNoOverlap(CIDRx string, CIDRy string, t *testing.T) { + _, netX, _ := net.ParseCIDR(CIDRx) + _, netY, _ := net.ParseCIDR(CIDRy) + if networkOverlaps(netX, netY) { + t.Errorf("%v and %v should not overlap", netX, netY) + } +} + +func TestNetworkOverlaps(t *testing.T) { + //netY starts at same IP and ends within netX + AssertOverlap("172.16.0.1/24", "172.16.0.1/25", t) + //netY starts within netX and ends at same IP + AssertOverlap("172.16.0.1/24", "172.16.0.128/25", t) + //netY starts and ends within netX + AssertOverlap("172.16.0.1/24", "172.16.0.64/25", t) + //netY starts at same IP and ends outside of netX + AssertOverlap("172.16.0.1/24", "172.16.0.1/23", t) + //netY starts before and ends at same IP of netX + AssertOverlap("172.16.1.1/24", "172.16.0.1/23", t) + //netY starts before and ends outside of netX + AssertOverlap("172.16.1.1/24", "172.16.0.1/23", t) + //netY starts and ends before netX + AssertNoOverlap("172.16.1.1/25", "172.16.0.1/24", t) + //netX starts and ends before netY + AssertNoOverlap("172.16.1.1/25", "172.16.2.1/24", t) +} + +func TestCheckRouteOverlaps(t *testing.T) { + routesData := []string{"10.0.2.0/32", "10.0.3.0/24", "10.0.42.0/24", "172.16.42.0/24", "192.168.142.0/24"} + + routes := []netlink.Route{} + for _, addr := range routesData { + _, netX, _ := net.ParseCIDR(addr) + routes = append(routes, netlink.Route{IPNet: netX}) + } + + _, netX, _ := net.ParseCIDR("172.16.0.1/24") + if err := checkRouteOverlaps(routes, netX); err != nil { + t.Fatal(err) + } + + _, netX, _ = net.ParseCIDR("10.0.2.0/24") + if err := checkRouteOverlaps(routes, netX); err == nil { + t.Fatalf("10.0.2.0/24 and 10.0.2.0 should overlap but it doesn't") + } +} + +func TestCheckNameserverOverlaps(t *testing.T) { + nameservers := []string{"10.0.2.3/32", "192.168.102.1/32"} + + _, netX, _ := net.ParseCIDR("10.0.2.3/32") + + if err := checkNameserverOverlaps(nameservers, netX); err == nil { + t.Fatalf("%s should overlap 10.0.2.3/32 but doesn't", netX) + } + + _, netX, _ = net.ParseCIDR("192.168.102.2/32") + + if err := checkNameserverOverlaps(nameservers, netX); err != nil { + t.Fatalf("%s should not overlap %v but it does", netX, nameservers) + } +} diff --git a/networkdriver/ipallocator/ipset.go b/networkdriver/ipallocator/ipset.go new file mode 100644 index 0000000000..43d54691d1 --- /dev/null +++ b/networkdriver/ipallocator/ipset.go @@ -0,0 +1,84 @@ +package ipallocator + +import ( + "sort" + "sync" +) + +// iPSet is a thread-safe sorted set and a stack. +type iPSet struct { + sync.RWMutex + set []int +} + +// Push takes a string and adds it to the set. If the elem aready exists, it has no effect. +func (s *iPSet) Push(elem int) { + s.RLock() + for _, e := range s.set { + if e == elem { + s.RUnlock() + return + } + } + s.RUnlock() + + s.Lock() + s.set = append(s.set, elem) + // Make sure the list is always sorted + sort.Ints(s.set) + s.Unlock() +} + +// Pop is an alias to PopFront() +func (s *iPSet) Pop() int { + return s.PopFront() +} + +// Pop returns the first elemen from the list and removes it. +// If the list is empty, it returns 0 +func (s *iPSet) PopFront() int { + s.RLock() + + for i, e := range s.set { + ret := e + s.RUnlock() + s.Lock() + s.set = append(s.set[:i], s.set[i+1:]...) + s.Unlock() + return ret + } + s.RUnlock() + + return 0 +} + +// PullBack retrieve the last element of the list. +// The element is not removed. +// If the list is empty, an empty element is returned. +func (s *iPSet) PullBack() int { + if len(s.set) == 0 { + return 0 + } + return s.set[len(s.set)-1] +} + +// Exists checks if the given element present in the list. +func (s *iPSet) Exists(elem int) bool { + for _, e := range s.set { + if e == elem { + return true + } + } + return false +} + +// Remove removes an element from the list. +// If the element is not found, it has no effect. +func (s *iPSet) Remove(elem int) { + for i, e := range s.set { + if e == elem { + s.set = append(s.set[:i], s.set[i+1:]...) + return + } + } +} diff --git a/networkdriver/network.go b/networkdriver/network.go new file mode 100644 index 0000000000..e74734d55f --- /dev/null +++ b/networkdriver/network.go @@ -0,0 +1 @@ +package networkdriver