Merge "ref: Change/remove veyron issues referred to in comments."
diff --git a/cmd/servicerunner/main.go b/cmd/servicerunner/main.go
index d141af5..397747d 100644
--- a/cmd/servicerunner/main.go
+++ b/cmd/servicerunner/main.go
@@ -21,6 +21,7 @@
 
 	"v.io/v23"
 	"v.io/v23/options"
+	"v.io/v23/rpc"
 
 	"v.io/x/ref/envvar"
 	"v.io/x/ref/lib/signals"
@@ -124,7 +125,9 @@
 	sh.SetVar(envvar.NamespacePrefix, vars["MT_NAME"])
 	v23.GetNamespace(ctx).SetRoots(vars["MT_NAME"])
 
-	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, "ws", "127.0.0.1:0", "", "test/proxy")
+	lspec := v23.GetListenSpec(ctx)
+	lspec.Addrs = rpc.ListenAddrs{{"ws", "127.0.0.1:0"}}
+	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, lspec, "test/proxy")
 	defer proxyShutdown()
 	vars["PROXY_NAME"] = proxyEndpoint.Name()
 
diff --git a/cmd/vdl/arith_test.go b/cmd/vdl/arith_test.go
index 7918765..f997fda 100644
--- a/cmd/vdl/arith_test.go
+++ b/cmd/vdl/arith_test.go
@@ -23,7 +23,7 @@
 	"v.io/x/ref/lib/vdl/testdata/base"
 	"v.io/x/ref/test"
 
-	_ "v.io/x/ref/profiles/static"
+	_ "v.io/x/ref/profiles"
 )
 
 var generatedError = errors.New("generated error")
diff --git a/profiles/gce/init.go b/profiles/gce/init.go
index 15c96d3..73a675f 100644
--- a/profiles/gce/init.go
+++ b/profiles/gce/init.go
@@ -58,8 +58,8 @@
 	if ip, err := gce.ExternalIPAddress(); err != nil {
 		return nil, nil, nil, err
 	} else {
-		listenSpec.AddressChooser = func(network string, addrs []rpc.Address) ([]rpc.Address, error) {
-			return []rpc.Address{&netstate.AddrIfc{&net.IPAddr{IP: ip}, "gce-nat", nil}}, nil
+		listenSpec.AddressChooser = func(network string, addrs []net.Addr) ([]net.Addr, error) {
+			return []net.Addr{netstate.NewNetAddr("wsh", ip.String())}, nil
 		}
 	}
 
diff --git a/profiles/internal/lib/publisher/publisher.go b/profiles/internal/lib/publisher/publisher.go
index acdd006..6c87030 100644
--- a/profiles/internal/lib/publisher/publisher.go
+++ b/profiles/internal/lib/publisher/publisher.go
@@ -335,7 +335,7 @@
 	if status.LastUnmountErr != nil {
 		vlog.Errorf("rpc pub: couldn't unmount(%v, %v): %v", name, server, status.LastUnmountErr)
 	} else {
-		vlog.Infof("rpc pub: unmount(%v, %v)", name, server)
+		vlog.VI(1).Infof("rpc pub: unmount(%v, %v)", name, server)
 		delete(ps.mounts, mountKey{name, server})
 	}
 }
diff --git a/profiles/internal/rpc/full_test.go b/profiles/internal/rpc/full_test.go
index 2e47921..e978fc0 100644
--- a/profiles/internal/rpc/full_test.go
+++ b/profiles/internal/rpc/full_test.go
@@ -1520,10 +1520,8 @@
 	sm := imanager.InternalNew(naming.FixedRoutingID(0x555555555))
 	defer sm.Shutdown()
 	ns := tnaming.NewSimpleNamespace()
-	pa := func(string, []rpc.Address) ([]rpc.Address, error) {
-		a := &net.IPAddr{}
-		a.IP = net.ParseIP("1.1.1.1")
-		return []rpc.Address{&netstate.AddrIfc{Addr: a}}, nil
+	pa := func(string, []net.Addr) ([]net.Addr, error) {
+		return []net.Addr{netstate.NewNetAddr("tcp", "1.1.1.1")}, nil
 	}
 	server, err := testInternalNewServer(ctx, sm, ns, testutil.NewPrincipal("server"))
 	if err != nil {
@@ -1565,7 +1563,7 @@
 	sm := imanager.InternalNew(naming.FixedRoutingID(0x555555555))
 	defer sm.Shutdown()
 	ns := tnaming.NewSimpleNamespace()
-	paerr := func(_ string, a []rpc.Address) ([]rpc.Address, error) {
+	paerr := func(_ string, a []net.Addr) ([]net.Addr, error) {
 		return nil, fmt.Errorf("oops")
 	}
 	server, err := testInternalNewServer(ctx, sm, ns, testutil.NewPrincipal("server"))
@@ -1578,17 +1576,16 @@
 		AddressChooser: paerr,
 	}
 	eps, err := server.Listen(spec)
-	iep := eps[0].(*inaming.Endpoint)
-	host, _, err := net.SplitHostPort(iep.Address)
-	if err != nil {
-		t.Errorf("unexpected error: %s", err)
+
+	if got, want := len(eps), 0; got != want {
+		t.Errorf("got %q, want %q", got, want)
 	}
-	ip := net.ParseIP(host)
-	if ip == nil {
-		t.Fatalf("failed to parse IP address: %q", host)
+	status := server.Status()
+	if got, want := len(status.Errors), 1; got != want {
+		t.Errorf("got %q, want %q", got, want)
 	}
-	if !ip.IsUnspecified() {
-		t.Errorf("IP: %q is not unspecified", ip)
+	if got, want := status.Errors[0].Error(), "oops"; got != want {
+		t.Errorf("got %q, want %q", got, want)
 	}
 }
 
diff --git a/profiles/internal/rpc/protocols/tcp/init.go b/profiles/internal/rpc/protocols/tcp/init.go
index a6067b9..08f95e6 100644
--- a/profiles/internal/rpc/protocols/tcp/init.go
+++ b/profiles/internal/rpc/protocols/tcp/init.go
@@ -22,6 +22,7 @@
 }
 
 func tcpDial(network, address string, timeout time.Duration) (net.Conn, error) {
+	vlog.Infof("tcp.Dial %v", address)
 	conn, err := net.DialTimeout(network, address, timeout)
 	if err != nil {
 		return nil, err
diff --git a/profiles/internal/rpc/server.go b/profiles/internal/rpc/server.go
index 91b3a1a..d41aa64 100644
--- a/profiles/internal/rpc/server.go
+++ b/profiles/internal/rpc/server.go
@@ -14,6 +14,9 @@
 	"sync"
 	"time"
 
+	"v.io/x/lib/netstate"
+	"v.io/x/lib/vlog"
+
 	"v.io/v23/config"
 	"v.io/v23/context"
 	"v.io/v23/namespace"
@@ -26,8 +29,7 @@
 	"v.io/v23/verror"
 	"v.io/v23/vom"
 	"v.io/v23/vtrace"
-	"v.io/x/lib/netstate"
-	"v.io/x/lib/vlog"
+
 	"v.io/x/ref/lib/stats"
 	"v.io/x/ref/profiles/internal/lib/publisher"
 	inaming "v.io/x/ref/profiles/internal/naming"
@@ -316,46 +318,10 @@
 	return "", verror.New(errFailedToResolveToEndpoint, s.ctx, address)
 }
 
-// getPossbileAddrs returns an appropriate set of addresses that could be used
-// to contact the supplied protocol, host, port parameters using the supplied
-// chooser function. It returns an indication of whether the supplied address
-// was fully specified or not, returning false if the address was fully
-// specified, and true if it was not.
-func getPossibleAddrs(protocol, host, port string, chooser rpc.AddressChooser) ([]rpc.Address, bool, error) {
-
-	ip := net.ParseIP(host)
-	if ip == nil {
-		return nil, false, verror.New(errFailedToParseIP, nil, host)
-	}
-
-	addrFromIP := func(ip net.IP) rpc.Address {
-		return &netstate.AddrIfc{
-			Addr: &net.IPAddr{IP: ip},
-		}
-	}
-
-	if ip.IsUnspecified() {
-		if chooser != nil {
-			// Need to find a usable IP address since the call to listen
-			// didn't specify one.
-			if addrs, err := netstate.GetAccessibleIPs(); err == nil {
-				a, err := chooser(protocol, addrs)
-				if err == nil && len(a) > 0 {
-					return a, true, nil
-				}
-			}
-		}
-		// We don't have a chooser, so we just return the address the
-		// underlying system has chosen.
-		return []rpc.Address{addrFromIP(ip)}, true, nil
-	}
-	return []rpc.Address{addrFromIP(ip)}, false, nil
-}
-
 // createEndpoints creates appropriate inaming.Endpoint instances for
 // all of the externally accessible network addresses that can be used
 // to reach this server.
-func (s *server) createEndpoints(lep naming.Endpoint, chooser rpc.AddressChooser) ([]*inaming.Endpoint, string, bool, error) {
+func (s *server) createEndpoints(lep naming.Endpoint, chooser netstate.AddressChooser) ([]*inaming.Endpoint, string, bool, error) {
 	iep, ok := lep.(*inaming.Endpoint)
 	if !ok {
 		return nil, "", false, verror.New(errInternalTypeConversion, nil, fmt.Sprintf("%T", lep))
@@ -365,15 +331,15 @@
 		// If not tcp, ws, or wsh, just return the endpoint we were given.
 		return []*inaming.Endpoint{iep}, "", false, nil
 	}
-
 	host, port, err := net.SplitHostPort(iep.Address)
 	if err != nil {
 		return nil, "", false, err
 	}
-	addrs, unspecified, err := getPossibleAddrs(iep.Protocol, host, port, chooser)
+	addrs, unspecified, err := netstate.PossibleAddresses(iep.Protocol, host, chooser)
 	if err != nil {
 		return nil, port, false, err
 	}
+
 	ieps := make([]*inaming.Endpoint, 0, len(addrs))
 	for _, addr := range addrs {
 		n, err := inaming.NewEndpoint(lep.String())
@@ -381,7 +347,7 @@
 			return nil, port, false, err
 		}
 		n.IsMountTable = s.servesMountTable
-		n.Address = net.JoinHostPort(addr.Address().String(), port)
+		n.Address = net.JoinHostPort(addr.String(), port)
 		ieps = append(ieps, n)
 	}
 	return ieps, port, unspecified, nil
@@ -652,7 +618,7 @@
 			return
 		}
 		switch v := setting.Value().(type) {
-		case []rpc.Address:
+		case []net.Addr:
 			s.Lock()
 			if s.isStopState() {
 				s.Unlock()
@@ -687,20 +653,20 @@
 	}
 }
 
-func getHost(address rpc.Address) string {
-	host, _, err := net.SplitHostPort(address.Address().String())
+func getHost(address net.Addr) string {
+	host, _, err := net.SplitHostPort(address.String())
 	if err == nil {
 		return host
 	}
-	return address.Address().String()
+	return address.String()
 
 }
 
 // Remove all endpoints that have the same host address as the supplied
 // address parameter.
-func (s *server) removeAddresses(addresses []rpc.Address) ([]naming.Endpoint, error) {
+func (s *server) removeAddresses(addrs []net.Addr) ([]naming.Endpoint, error) {
 	var removed []naming.Endpoint
-	for _, address := range addresses {
+	for _, address := range addrs {
 		host := getHost(address)
 		for ls, _ := range s.listenState {
 			if ls != nil && ls.roaming && len(ls.ieps) > 0 {
@@ -735,13 +701,16 @@
 // externally accessible.
 // This places the onus on the dhcp/roaming code that sends us addresses
 // to ensure that those addresses are externally reachable.
-func (s *server) addAddresses(addresses []rpc.Address) []naming.Endpoint {
+func (s *server) addAddresses(addrs []net.Addr) []naming.Endpoint {
 	var added []naming.Endpoint
-	for _, address := range addresses {
+	vlog.Infof("HERE WITH %v -> %v", addrs, netstate.ConvertToAddresses(addrs))
+	for _, address := range netstate.ConvertToAddresses(addrs) {
 		if !netstate.IsAccessibleIP(address) {
+			vlog.Infof("RETURN A %v", added)
 			return added
 		}
 		host := getHost(address)
+		vlog.Infof("LISTEN ST: %v", s.listenState)
 		for ls, _ := range s.listenState {
 			if ls != nil && ls.roaming {
 				niep := ls.protoIEP
@@ -755,6 +724,7 @@
 			}
 		}
 	}
+	vlog.Infof("RETURN B %v", added)
 	return added
 }
 
diff --git a/profiles/internal/rpc/server_test.go b/profiles/internal/rpc/server_test.go
index 4e4dbe5..4e30884 100644
--- a/profiles/internal/rpc/server_test.go
+++ b/profiles/internal/rpc/server_test.go
@@ -371,7 +371,7 @@
 	return &niep
 }
 
-func getIPAddrs(eps []naming.Endpoint) []rpc.Address {
+func getIPAddrs(eps []naming.Endpoint) []net.Addr {
 	hosts := map[string]struct{}{}
 	for _, ep := range eps {
 		iep := (ep).(*inaming.Endpoint)
@@ -380,10 +380,9 @@
 			hosts[h] = struct{}{}
 		}
 	}
-	addrs := []rpc.Address{}
+	addrs := []net.Addr{}
 	for h, _ := range hosts {
-		a := &netstate.AddrIfc{Addr: &net.IPAddr{IP: net.ParseIP(h)}}
-		addrs = append(addrs, a)
+		addrs = append(addrs, netstate.NewNetAddr("ip", h))
 	}
 	return addrs
 }
@@ -439,11 +438,11 @@
 	}
 	defer func() { publisher.Shutdown(); <-stop }()
 
-	ipv4And6 := func(network string, addrs []rpc.Address) ([]rpc.Address, error) {
-		accessible := netstate.AddrList(addrs)
+	ipv4And6 := func(network string, addrs []net.Addr) ([]net.Addr, error) {
+		accessible := netstate.ConvertToAddresses(addrs)
 		ipv4 := accessible.Filter(netstate.IsUnicastIPv4)
 		ipv6 := accessible.Filter(netstate.IsUnicastIPv6)
-		return append(ipv4, ipv6...), nil
+		return append(ipv4.AsNetAddrs(), ipv6.AsNetAddrs()...), nil
 	}
 	spec := rpc.ListenSpec{
 		Addrs: rpc.ListenAddrs{
@@ -481,14 +480,14 @@
 		t.Fatalf("got %d, want %d", got, want)
 	}
 
-	n1 := &netstate.AddrIfc{Addr: &net.IPAddr{IP: net.ParseIP("1.1.1.1")}}
-	n2 := &netstate.AddrIfc{Addr: &net.IPAddr{IP: net.ParseIP("2.2.2.2")}}
+	n1 := netstate.NewNetAddr("ip", "1.1.1.1")
+	n2 := netstate.NewNetAddr("ip", "2.2.2.2")
 
 	watcher := make(chan rpc.NetworkChange, 10)
 	server.WatchNetwork(watcher)
 	defer close(watcher)
 
-	roaming <- rpc.NewAddAddrsSetting([]rpc.Address{n1, n2})
+	roaming <- rpc.NewAddAddrsSetting([]net.Addr{n1, n2})
 
 	waitForChange := func() *rpc.NetworkChange {
 		vlog.Infof("Waiting on %p", watcher)
@@ -527,7 +526,7 @@
 		t.Fatalf("got %d, want %d", got, want)
 	}
 
-	roaming <- rpc.NewRmAddrsSetting([]rpc.Address{n1})
+	roaming <- rpc.NewRmAddrsSetting([]net.Addr{n1})
 
 	// We expect 2 changes, one for each usable listen spec addr.
 	change = waitForChange()
@@ -561,7 +560,7 @@
 		t.Fatalf("got %d, want %d: %v", got, want, status.Mounts)
 	}
 
-	roaming <- rpc.NewAddAddrsSetting([]rpc.Address{n1})
+	roaming <- rpc.NewAddAddrsSetting([]net.Addr{n1})
 	// We expect 2 changes, one for each usable listen spec addr.
 	change = waitForChange()
 	if got, want := len(change.Changed), 2; got != want {
@@ -618,9 +617,9 @@
 	roaming <- rpc.NewRmAddrsSetting(getIPAddrs(eps))
 
 	// Add in two new addresses
-	n1 := &netstate.AddrIfc{Addr: &net.IPAddr{IP: net.ParseIP("1.1.1.1")}}
-	n2 := &netstate.AddrIfc{Addr: &net.IPAddr{IP: net.ParseIP("2.2.2.2")}}
-	roaming <- rpc.NewAddAddrsSetting([]rpc.Address{n1, n2})
+	n1 := netstate.NewNetAddr("ip", "1.1.1.1")
+	n2 := netstate.NewNetAddr("ip", "2.2.2.2")
+	roaming <- rpc.NewAddAddrsSetting([]net.Addr{n1, n2})
 
 	neps := make([]naming.Endpoint, 0, len(eps))
 	for _, p := range getUniqPorts(eps) {
diff --git a/profiles/internal/rpc/sort_endpoints.go b/profiles/internal/rpc/sort_endpoints.go
index 8ea0bd9..57409e7 100644
--- a/profiles/internal/rpc/sort_endpoints.go
+++ b/profiles/internal/rpc/sort_endpoints.go
@@ -200,16 +200,16 @@
 
 // ipNetworks returns the IP networks on this machine.
 func ipNetworks() []*net.IPNet {
-	ifcs, err := netstate.GetAll()
+	ifcs, err := netstate.GetAllAddresses()
 	if err != nil {
-		vlog.VI(5).Infof("netstate.GetAll failed: %v", err)
+		vlog.VI(5).Infof("netstate.GetAllAddresses failed: %v", err)
 		return nil
 	}
 	ret := make([]*net.IPNet, 0, len(ifcs))
 	for _, a := range ifcs {
-		_, ipnet, err := net.ParseCIDR(a.Address().String())
+		_, ipnet, err := net.ParseCIDR(a.String())
 		if err != nil {
-			vlog.VI(5).Infof("net.ParseCIDR(%q) failed: %v", a.Address(), err)
+			vlog.VI(5).Infof("net.ParseCIDR(%q) failed: %v", a, err)
 			continue
 		}
 		ret = append(ret, ipnet)
diff --git a/profiles/internal/rpc/stream/manager/listener.go b/profiles/internal/rpc/stream/manager/listener.go
index 1122689..191ca22 100644
--- a/profiles/internal/rpc/stream/manager/listener.go
+++ b/profiles/internal/rpc/stream/manager/listener.go
@@ -9,6 +9,7 @@
 	"net"
 	"strings"
 	"sync"
+	"syscall"
 	"time"
 
 	"v.io/x/ref/profiles/internal/lib/upcqueue"
@@ -106,15 +107,29 @@
 	return false
 }
 
+func isTooManyOpenFiles(err error) bool {
+	if oErr, ok := err.(*net.OpError); ok && oErr.Err == syscall.EMFILE {
+		return true
+	}
+	return false
+}
+
 func (ln *netListener) netAcceptLoop(principal security.Principal, blessings security.Blessings, opts []stream.ListenerOpt) {
 	defer ln.netLoop.Done()
 	opts = append([]stream.ListenerOpt{vc.StartTimeout{defaultStartTimeout}}, opts...)
 	for {
 		conn, err := ln.netLn.Accept()
 		if isTemporaryError(err) {
-			// TODO(rthellend): Aggressively close other connections?
-			vlog.Errorf("net.Listener.Accept() failed on %v with %v", ln.netLn, err)
-			for isTemporaryError(err) {
+			// Use Info instead of Error to reduce the changes that
+			// the log library will cause the process to abort on
+			// failing to create a new file.
+			vlog.Infof("net.Listener.Accept() failed on %v with %v", ln.netLn, err)
+			for tokill := 1; isTemporaryError(err); tokill *= 2 {
+				if isTooManyOpenFiles(err) {
+					ln.manager.killConnections(tokill)
+				} else {
+					tokill = 1
+				}
 				time.Sleep(10 * time.Millisecond)
 				conn, err = ln.netLn.Accept()
 			}
diff --git a/profiles/internal/rpc/stream/manager/manager.go b/profiles/internal/rpc/stream/manager/manager.go
index ee05736..4b9c4b8 100644
--- a/profiles/internal/rpc/stream/manager/manager.go
+++ b/profiles/internal/rpc/stream/manager/manager.go
@@ -7,6 +7,7 @@
 
 import (
 	"fmt"
+	"math/rand"
 	"net"
 	"strings"
 	"sync"
@@ -19,6 +20,7 @@
 	"v.io/x/lib/vlog"
 
 	"v.io/x/ref/lib/stats"
+	"v.io/x/ref/lib/stats/counter"
 	inaming "v.io/x/ref/profiles/internal/naming"
 	"v.io/x/ref/profiles/internal/rpc/stream"
 	"v.io/x/ref/profiles/internal/rpc/stream/crypto"
@@ -55,14 +57,16 @@
 // placed inside v.io/x/ref/profiles/internal. Code outside the
 // v.io/x/ref/profiles/internal/* packages should never call this method.
 func InternalNew(rid naming.RoutingID) stream.Manager {
+	statsPrefix := naming.Join("rpc", "stream", "routing-id", rid.String())
 	m := &manager{
 		rid:          rid,
 		vifs:         vif.NewSet(),
 		sessionCache: crypto.NewTLSClientSessionCache(),
 		listeners:    make(map[listener]bool),
-		statsName:    naming.Join("rpc", "stream", "routing-id", rid.String(), "debug"),
+		statsPrefix:  statsPrefix,
+		killedConns:  stats.NewCounter(naming.Join(statsPrefix, "killed-connections")),
 	}
-	stats.NewStringFunc(m.statsName, m.DebugString)
+	stats.NewStringFunc(naming.Join(m.statsPrefix, "debug"), m.DebugString)
 	return m
 }
 
@@ -75,7 +79,8 @@
 	listeners   map[listener]bool // GUARDED_BY(muListeners)
 	shutdown    bool              // GUARDED_BY(muListeners)
 
-	statsName string
+	statsPrefix string
+	killedConns *counter.Counter
 }
 
 var _ stream.Manager = (*manager)(nil)
@@ -273,7 +278,7 @@
 }
 
 func (m *manager) Shutdown() {
-	stats.Delete(m.statsName)
+	stats.Delete(m.statsPrefix)
 	m.muListeners.Lock()
 	if m.shutdown {
 		m.muListeners.Unlock()
@@ -327,6 +332,29 @@
 	return strings.Join(l, "\n")
 }
 
+func (m *manager) killConnections(n int) {
+	vifs := m.vifs.List()
+	if n > len(vifs) {
+		n = len(vifs)
+	}
+	vlog.Infof("Killing %d VIFs", n)
+	var wg sync.WaitGroup
+	wg.Add(n)
+	for i := 0; i < n; i++ {
+		idx := rand.Intn(len(vifs))
+		vf := vifs[idx]
+		go func(vf *vif.VIF) {
+			vlog.Infof("Killing VIF %v", vf)
+			vf.Shutdown()
+			m.killedConns.Incr(1)
+			wg.Done()
+		}(vf)
+		vifs[idx], vifs[0] = vifs[0], vifs[idx]
+		vifs = vifs[1:]
+	}
+	wg.Wait()
+}
+
 func extractBlessingNames(p security.Principal, b security.Blessings) ([]string, error) {
 	if !b.IsZero() && p == nil {
 		return nil, verror.New(stream.ErrBadArg, nil, verror.New(errProvidedServerBlessingsWithoutPrincipal, nil))
diff --git a/profiles/internal/rpc/stream/manager/manager_test.go b/profiles/internal/rpc/stream/manager/manager_test.go
index 8b551e4..b8cc61c 100644
--- a/profiles/internal/rpc/stream/manager/manager_test.go
+++ b/profiles/internal/rpc/stream/manager/manager_test.go
@@ -13,7 +13,9 @@
 	"reflect"
 	"runtime"
 	"sort"
+	"strconv"
 	"strings"
+	"syscall"
 	"testing"
 	"time"
 
@@ -38,6 +40,7 @@
 
 func init() {
 	modules.RegisterChild("runServer", "", runServer)
+	modules.RegisterChild("runRLimitedServer", "", runRLimitedServer)
 }
 
 // We write our own TestMain here instead of relying on v23 test generate because
@@ -669,12 +672,10 @@
 	if err != nil {
 		t.Fatalf("unexpected error: %s", err)
 	}
-	s := expect.NewSession(t, h.Stdout(), time.Minute)
-	addr := s.ReadLine()
-
-	ep, err := inaming.NewEndpoint(naming.FormatEndpoint(protocol, addr))
+	epstr := expect.NewSession(t, h.Stdout(), time.Minute).ExpectVar("ENDPOINT")
+	ep, err := inaming.NewEndpoint(epstr)
 	if err != nil {
-		t.Fatalf("inaming.NewEndpoint(%q): %v", addr, err)
+		t.Fatalf("inaming.NewEndpoint(%q): %v", epstr, err)
 	}
 	if _, err := client.Dial(ep, pclient); err != nil {
 		t.Fatal(err)
@@ -686,15 +687,19 @@
 		t.Fatal("Expected client.Dial to fail since server is dead")
 	}
 
-	h, err = sh.Start("runServer", nil, protocol, addr)
+	h, err = sh.Start("runServer", nil, protocol, ep.Addr().String())
 	if err != nil {
 		t.Fatalf("unexpected error: %s", err)
 	}
 	// Restarting the server, listening on the same address as before
-	if addr2 := h.ReadLine(); addr2 != addr || err != nil {
-		t.Fatalf("Got (%q, %v) want (%q, nil)", addr2, err, addr)
+	ep2, err := inaming.NewEndpoint(expect.NewSession(t, h.Stdout(), time.Minute).ExpectVar("ENDPOINT"))
+	if err != nil {
+		t.Fatal(err)
 	}
-	if _, err := client.Dial(ep, pclient); err != nil {
+	if got, want := ep.Addr().String(), ep2.Addr().String(); got != want {
+		t.Fatalf("Got %q, want %q", got, want)
+	}
+	if _, err := client.Dial(ep2, pclient); err != nil {
 		t.Fatal(err)
 	}
 }
@@ -707,12 +712,27 @@
 		fmt.Fprintln(stderr, err)
 		return err
 	}
-	fmt.Fprintln(stdout, ep.Addr())
+	fmt.Fprintf(stdout, "ENDPOINT=%v\n", ep)
 	// Live forever (till the process is explicitly killed)
 	modules.WaitForEOF(stdin)
 	return nil
 }
 
+func runRLimitedServer(stdin io.Reader, stdout, stderr io.Writer, env map[string]string, args ...string) error {
+	var rlimit syscall.Rlimit
+	if err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &rlimit); err != nil {
+		fmt.Fprintln(stderr, err)
+		return err
+	}
+	rlimit.Cur = 9
+	if err := syscall.Setrlimit(syscall.RLIMIT_NOFILE, &rlimit); err != nil {
+		fmt.Fprintln(stderr, err)
+		return err
+	}
+	fmt.Fprintf(stdout, "RLIMIT_NOFILE=%d\n", rlimit.Cur)
+	return runServer(stdin, stdout, stderr, env, args...)
+}
+
 func readLine(f stream.Flow) (string, error) {
 	var result bytes.Buffer
 	var buf [5]byte
@@ -833,3 +853,71 @@
 		}
 	}
 }
+
+func TestVIFCleanupWhenFDLimitIsReached(t *testing.T) {
+	sh, err := modules.NewShell(nil, nil, testing.Verbose(), t)
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer sh.Cleanup(nil, nil)
+	h, err := sh.Start("runRLimitedServer", nil, "--logtostderr=true", "tcp", "127.0.0.1:0")
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer h.CloseStdin()
+	stdout := expect.NewSession(t, h.Stdout(), time.Minute)
+	nfiles, err := strconv.Atoi(stdout.ExpectVar("RLIMIT_NOFILE"))
+	if stdout.Error() != nil {
+		t.Fatal(stdout.Error())
+	}
+	if err != nil {
+		t.Fatal(err)
+	}
+	epstr := stdout.ExpectVar("ENDPOINT")
+	if stdout.Error() != nil {
+		t.Fatal(stdout.Error())
+	}
+	ep, err := inaming.NewEndpoint(epstr)
+	if err != nil {
+		t.Fatal(err)
+	}
+	// Different client processes (represented by different stream managers
+	// in this test) should be able to make progress, even if the server
+	// has reached its file descriptor limit.
+	nattempts := 0
+	for i := 0; i < 2*nfiles; i++ {
+		client := InternalNew(naming.FixedRoutingID(uint64(i)))
+		defer client.Shutdown()
+		principal := testutil.NewPrincipal(fmt.Sprintf("client%d", i))
+		connected := false
+		for !connected {
+			nattempts++
+			// If the client connection reached the server when it
+			// was at its limit, it might fail.  However, this
+			// failure will trigger the "kill connections" logic at
+			// the server and eventually the client should succeed.
+			vc, err := client.Dial(ep, principal)
+			if err != nil {
+				continue
+			}
+			// Establish a flow to prevent the VC (and thus the
+			// underlying VIF) from being garbage collected as an
+			// "inactive" connection.
+			flow, err := vc.Connect()
+			if err != nil {
+				continue
+			}
+			defer flow.Close()
+			connected = true
+		}
+	}
+	var stderr bytes.Buffer
+	if err := h.Shutdown(nil, &stderr); err != nil {
+		t.Fatal(err)
+	}
+	if log := expect.NewSession(t, bytes.NewReader(stderr.Bytes()), time.Minute).ExpectSetEventuallyRE("manager.go.*Killing [1-9][0-9]* VIFs"); len(log) == 0 {
+		t.Errorf("Failed to find log message talking about killing VIFs in:\n%v", stderr.String())
+	}
+	t.Logf("Server FD limit:%d", nfiles)
+	t.Logf("Client connection attempts: %d", nattempts)
+}
diff --git a/profiles/internal/rpc/stream/proxy/proxy.go b/profiles/internal/rpc/stream/proxy/proxy.go
index 7836963..f74428b 100644
--- a/profiles/internal/rpc/stream/proxy/proxy.go
+++ b/profiles/internal/rpc/stream/proxy/proxy.go
@@ -10,6 +10,8 @@
 	"sync"
 	"time"
 
+	"v.io/x/lib/netstate"
+
 	"v.io/v23"
 	"v.io/v23/context"
 	"v.io/v23/naming"
@@ -67,6 +69,9 @@
 	errFailedToFowardOpenFlow    = reg(".errFailedToFowardOpenFlow", "failed to forward open flow{:3}")
 	errServerNotBeingProxied     = reg(".errServerNotBeingProxied", "no server with routing id {3} is being proxied")
 	errServerVanished            = reg(".errServerVanished", "server with routing id {3} vanished")
+	errAccessibleAddresses       = reg(".errAccessibleAddresses", "failed to obtain a set of accessible addresses{:3}")
+	errNoAccessibleAddresses     = reg(".errNoAccessibleAddresses", "no accessible addresses were available for {3}")
+	errEmptyListenSpec           = reg(".errEmptyListenSpec", "no addresses supplied in the listen spec")
 )
 
 // Proxy routes virtual circuit (VC) traffic between multiple underlying
@@ -181,13 +186,13 @@
 // TODO(mattr): This should take a ListenSpec instead of network, address, and
 // pubAddress.  However using a ListenSpec requires a great deal of supporting
 // code that should be refactored out of v.io/x/ref/profiles/internal/rpc/server.go.
-func New(ctx *context.T, network, address, pubAddress string, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
+func New(ctx *context.T, spec rpc.ListenSpec, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
 	rid, err := naming.NewRoutingID()
 	if err != nil {
 		return nil, nil, err
 	}
 
-	proxy, err := internalNew(rid, v23.GetPrincipal(ctx), network, address, pubAddress)
+	proxy, err := internalNew(rid, v23.GetPrincipal(ctx), spec)
 	if err != nil {
 		return nil, nil, err
 	}
@@ -216,7 +221,13 @@
 	return shutdown, proxy.endpoint(), nil
 }
 
-func internalNew(rid naming.RoutingID, principal security.Principal, network, address, pubAddress string) (*Proxy, error) {
+func internalNew(rid naming.RoutingID, principal security.Principal, spec rpc.ListenSpec) (*Proxy, error) {
+	if len(spec.Addrs) == 0 {
+		return nil, verror.New(stream.ErrProxy, nil, verror.New(errEmptyListenSpec, nil))
+	}
+	laddr := spec.Addrs[0]
+	network := laddr.Protocol
+	address := laddr.Address
 	_, listenFn, _ := rpc.RegisteredProtocol(network)
 	if listenFn == nil {
 		return nil, verror.New(stream.ErrProxy, nil, verror.New(errUnknownNetwork, nil, network))
@@ -225,15 +236,23 @@
 	if err != nil {
 		return nil, verror.New(stream.ErrProxy, nil, verror.New(errListenFailed, nil, network, address, err))
 	}
-	if len(pubAddress) == 0 {
-		pubAddress = ln.Addr().String()
+	pub, _, err := netstate.PossibleAddresses(ln.Addr().Network(), ln.Addr().String(), spec.AddressChooser)
+	if err != nil {
+		ln.Close()
+		return nil, verror.New(stream.ErrProxy, nil, verror.New(errAccessibleAddresses, nil, err))
 	}
+	if len(pub) == 0 {
+		ln.Close()
+		return nil, verror.New(stream.ErrProxy, nil, verror.New(errNoAccessibleAddresses, nil, ln.Addr().String()))
+	}
+
 	proxy := &Proxy{
-		ln:         ln,
-		rid:        rid,
-		servers:    &servermap{m: make(map[naming.RoutingID]*server)},
-		processes:  make(map[*process]struct{}),
-		pubAddress: pubAddress,
+		ln:        ln,
+		rid:       rid,
+		servers:   &servermap{m: make(map[naming.RoutingID]*server)},
+		processes: make(map[*process]struct{}),
+		// TODO(cnicolaou): should use all of the available addresses
+		pubAddress: pub[0].String(),
 		principal:  principal,
 		statsName:  naming.Join("rpc", "proxy", "routing-id", rid.String(), "debug"),
 	}
diff --git a/profiles/internal/rpc/stream/proxy/proxy_test.go b/profiles/internal/rpc/stream/proxy/proxy_test.go
index 6a08df9..c84e7cf 100644
--- a/profiles/internal/rpc/stream/proxy/proxy_test.go
+++ b/profiles/internal/rpc/stream/proxy/proxy_test.go
@@ -13,7 +13,11 @@
 	"testing"
 	"time"
 
+	"v.io/x/lib/vlog"
+
+	"v.io/v23"
 	"v.io/v23/naming"
+	"v.io/v23/verror"
 
 	_ "v.io/x/ref/profiles"
 	inaming "v.io/x/ref/profiles/internal/naming"
@@ -22,14 +26,19 @@
 	"v.io/x/ref/profiles/internal/rpc/stream/proxy"
 	"v.io/x/ref/profiles/internal/rpc/stream/vc"
 	"v.io/x/ref/profiles/internal/rpc/stream/vif"
+	"v.io/x/ref/test"
 	"v.io/x/ref/test/testutil"
 )
 
 //go:generate v23 test generate
 
 func TestProxy(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	pproxy := testutil.NewPrincipal("proxy")
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -37,6 +46,8 @@
 	principal := testutil.NewPrincipal("test")
 	blessings := principal.BlessingStore().Default()
 
+	vlog.Infof("PROXYEP: %s", proxyEp)
+
 	// Create the stream.Manager for the server.
 	server1 := manager.InternalNew(naming.FixedRoutingID(0x1111111111111111))
 	defer server1.Shutdown()
@@ -44,6 +55,7 @@
 	// through the proxy.
 	ln1, ep1, err := server1.Listen(proxyEp.Network(), proxyEp.String(), principal, blessings)
 	if err != nil {
+		t.Logf(verror.DebugString(err))
 		t.Fatal(err)
 	}
 	defer ln1.Close()
@@ -91,8 +103,11 @@
 }
 
 func TestDuplicateRoutingID(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	pproxy := testutil.NewPrincipal("proxy")
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -122,8 +137,11 @@
 }
 
 func TestProxyAuthentication(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	pproxy := testutil.NewPrincipal("proxy")
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -150,13 +168,16 @@
 }
 
 func TestServerBlessings(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	var (
 		pproxy  = testutil.NewPrincipal("proxy")
 		pserver = testutil.NewPrincipal("server")
 		pclient = testutil.NewPrincipal("client")
 	)
 
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -200,8 +221,11 @@
 }
 
 func TestHostPort(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	pproxy := testutil.NewPrincipal("proxy")
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -220,8 +244,11 @@
 }
 
 func TestClientBecomesServer(t *testing.T) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	pproxy := testutil.NewPrincipal("proxy")
-	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	_, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -275,6 +302,9 @@
 }
 
 func testProxyIdleTimeout(t *testing.T, testServer bool) {
+	ctx, shutdown := test.InitForTest()
+	defer shutdown()
+
 	const (
 		idleTime = 10 * time.Millisecond
 		// We use a long wait time here since it takes some time to handle VC close
@@ -299,7 +329,7 @@
 	// Pause the idle timers.
 	triggerTimers := vif.SetFakeTimers()
 
-	Proxy, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, "tcp", "127.0.0.1:0", "")
+	Proxy, shutdown, proxyEp, err := proxy.InternalNew(naming.FixedRoutingID(0xbbbbbbbbbbbbbbbb), pproxy, v23.GetListenSpec(ctx))
 	if err != nil {
 		t.Fatal(err)
 	}
diff --git a/profiles/internal/rpc/stream/proxy/testutil_test.go b/profiles/internal/rpc/stream/proxy/testutil_test.go
index 1e469b2..8a1568c 100644
--- a/profiles/internal/rpc/stream/proxy/testutil_test.go
+++ b/profiles/internal/rpc/stream/proxy/testutil_test.go
@@ -6,13 +6,17 @@
 
 import (
 	"v.io/v23/naming"
+	"v.io/v23/rpc"
 	"v.io/v23/security"
 )
 
 // These are the internal functions only for use in the proxy_test package.
 
-func InternalNew(rid naming.RoutingID, p security.Principal, net, addr, pubAddr string) (*Proxy, func(), naming.Endpoint, error) {
-	proxy, err := internalNew(rid, p, net, addr, pubAddr)
+func InternalNew(rid naming.RoutingID, p security.Principal, spec rpc.ListenSpec) (*Proxy, func(), naming.Endpoint, error) {
+	proxy, err := internalNew(rid, p, spec)
+	if err != nil {
+		return nil, nil, nil, err
+	}
 	return proxy, proxy.shutdown, proxy.endpoint(), err
 }
 
diff --git a/profiles/internal/rpc/stream/vif/vif.go b/profiles/internal/rpc/stream/vif/vif.go
index f19a190..4298040 100644
--- a/profiles/internal/rpc/stream/vif/vif.go
+++ b/profiles/internal/rpc/stream/vif/vif.go
@@ -435,6 +435,12 @@
 	}
 }
 
+// Shutdown terminates the underlying network connection (any pending reads and
+// writes of flows/VCs over it will be discarded).
+func (vif *VIF) Shutdown() {
+	vif.conn.Close()
+}
+
 // StartAccepting begins accepting Flows (and VCs) initiated by the remote end
 // of a VIF. opts is used to setup the listener on newly established VCs.
 func (vif *VIF) StartAccepting(opts ...stream.ListenerOpt) error {
diff --git a/profiles/internal/rpc/test/proxy_test.go b/profiles/internal/rpc/test/proxy_test.go
index ab8c7e9..bd326f5 100644
--- a/profiles/internal/rpc/test/proxy_test.go
+++ b/profiles/internal/rpc/test/proxy_test.go
@@ -14,6 +14,8 @@
 	"testing"
 	"time"
 
+	"v.io/x/lib/vlog"
+
 	"v.io/v23"
 	"v.io/v23/context"
 	"v.io/v23/namespace"
@@ -52,15 +54,14 @@
 	defer shutdown()
 
 	expected := len(args)
-	listenSpec := v23.GetListenSpec(ctx)
-	protocol := listenSpec.Addrs[0].Protocol
-	addr := listenSpec.Addrs[0].Address
-	proxyShutdown, proxyEp, err := proxy.New(ctx, protocol, addr, "")
+
+	listenSpec := rpc.ListenSpec{Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}}}
+	proxyShutdown, proxyEp, err := proxy.New(ctx, listenSpec)
 	if err != nil {
+		fmt.Fprintf(stderr, "%s\n", verror.DebugString(err))
 		return err
 	}
 	defer proxyShutdown()
-
 	fmt.Fprintf(stdout, "PID=%d\n", os.Getpid())
 	if expected > 0 {
 		pub := publisher.New(ctx, v23.GetNamespace(ctx), time.Minute)
@@ -128,6 +129,7 @@
 	p.ReadLine()
 	h.name = p.ExpectVar("PROXY_NAME")
 	if len(h.name) == 0 {
+		h.proxy.Shutdown(os.Stderr, os.Stderr)
 		t.Fatalf("failed to get PROXY_NAME from proxyd")
 	}
 	return h.ns.Mount(ctx, "proxy", h.name, time.Hour)
@@ -150,14 +152,18 @@
 }
 
 func TestProxy(t *testing.T) {
-	proxyListenSpec := rpc.ListenSpec{Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}}}
-	proxyListenSpec.Proxy = "proxy"
+	proxyListenSpec := rpc.ListenSpec{
+		Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}},
+		Proxy: "proxy",
+	}
 	testProxy(t, proxyListenSpec)
 }
 
 func TestWSProxy(t *testing.T) {
-	proxyListenSpec := rpc.ListenSpec{Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}}}
-	proxyListenSpec.Proxy = "proxy"
+	proxyListenSpec := rpc.ListenSpec{
+		Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}},
+		Proxy: "proxy",
+	}
 	// The proxy uses websockets only, but the server is using tcp.
 	testProxy(t, proxyListenSpec, "--v23.tcp.protocol=ws")
 }
@@ -165,6 +171,7 @@
 func testProxy(t *testing.T, spec rpc.ListenSpec, args ...string) {
 	ctx, shutdown := testContext()
 	defer shutdown()
+
 	var (
 		pserver   = testutil.NewPrincipal("server")
 		pclient   = testutil.NewPrincipal("client")
@@ -237,12 +244,18 @@
 		then := time.Now().Add(time.Minute)
 		for {
 			me, err := ns.Resolve(ctx, name)
+			if err != nil {
+				continue
+			}
+			for i, s := range me.Servers {
+				vlog.Infof("%d: %s", i, s)
+			}
 			if err == nil && len(me.Servers) == expect {
 				ch <- 1
 				return
 			}
 			if time.Now().After(then) {
-				t.Fatalf("timed out")
+				t.Fatalf("timed out waiting for %d servers, found %d", expect, len(me.Servers))
 			}
 			time.Sleep(100 * time.Millisecond)
 		}
diff --git a/profiles/internal/rt/runtime.go b/profiles/internal/rt/runtime.go
index d57dfa9..f0ec69f 100644
--- a/profiles/internal/rt/runtime.go
+++ b/profiles/internal/rt/runtime.go
@@ -136,43 +136,23 @@
 		return nil, nil, nil, err
 	}
 
-	// Set the initial stream manager.
-	ctx, err = r.setNewStreamManager(ctx)
+	// Create and set the principal
+	principal, deps, err := r.initPrincipal(ctx, flags.Credentials)
+	if err != nil {
+		return nil, nil, nil, err
+	}
+	ctx, err = r.setPrincipal(ctx, principal, deps...)
 	if err != nil {
 		return nil, nil, nil, err
 	}
 
-	// The client we create here is incomplete (has a nil principal) and only works
-	// because the agent uses anonymous unix sockets and SecurityNone.
-	// After security is initialized we attach a real client.
-	// We do not capture the ctx here on purpose, to avoid anyone accidentally
-	// using this client anywhere.
-	_, client, err := r.WithNewClient(ctx)
+	// Setup authenticated "networking"
+	ctx, err = r.WithNewStreamManager(ctx)
 	if err != nil {
 		return nil, nil, nil, err
 	}
 
-	// Initialize security.
-	principal, err := initSecurity(ctx, flags.Credentials, client)
-	if err != nil {
-		return nil, nil, nil, err
-	}
-	// If the principal is an agent principal, it depends on the client created
-	// above.  If not, there's no harm in the dependency.
-	ctx, err = r.setPrincipal(ctx, principal, client)
-	if err != nil {
-		return nil, nil, nil, err
-	}
-
-	// Set up secure client.
-	ctx, _, err = r.WithNewClient(ctx)
-	if err != nil {
-		return nil, nil, nil, err
-	}
-
-	ctx = r.WithBackgroundContext(ctx)
-
-	return r, ctx, r.shutdown, nil
+	return r, r.WithBackgroundContext(ctx), r.shutdown, nil
 }
 
 func (r *Runtime) addChild(ctx *context.T, me interface{}, stop func(), dependsOn ...interface{}) error {
diff --git a/profiles/internal/rt/security.go b/profiles/internal/rt/security.go
index 715ea0b..1545001 100644
--- a/profiles/internal/rt/security.go
+++ b/profiles/internal/rt/security.go
@@ -13,7 +13,6 @@
 
 	"v.io/v23/context"
 	"v.io/v23/naming"
-	"v.io/v23/rpc"
 	"v.io/v23/security"
 	"v.io/v23/verror"
 	"v.io/x/ref/envvar"
@@ -24,23 +23,12 @@
 	"v.io/x/ref/services/agent/agentlib"
 )
 
-func initSecurity(ctx *context.T, credentials string, client rpc.Client) (security.Principal, error) {
-	principal, err := setupPrincipal(ctx, credentials, client)
-	if err != nil {
-		return nil, err
-	}
-
-	return principal, nil
-}
-
-func setupPrincipal(ctx *context.T, credentials string, client rpc.Client) (security.Principal, error) {
-	var err error
-	var principal security.Principal
+func (r *Runtime) initPrincipal(ctx *context.T, credentials string) (principal security.Principal, deps []interface{}, err error) {
 	if principal, _ = ctx.Value(principalKey).(security.Principal); principal != nil {
-		return principal, nil
+		return principal, nil, nil
 	}
 	if len(credentials) > 0 {
-		// We close the agentFD if that is also provided
+		// Explicitly specified credentials, ignore the agent.
 		if _, fd, _ := agentEP(); fd >= 0 {
 			syscall.Close(fd)
 		}
@@ -51,23 +39,44 @@
 		if principal, err = vsecurity.LoadPersistentPrincipal(credentials, nil); err != nil {
 			if os.IsNotExist(err) {
 				if principal, err = vsecurity.CreatePersistentPrincipal(credentials, nil); err != nil {
-					return principal, err
+					return principal, nil, err
 				}
-				return principal, vsecurity.InitDefaultBlessings(principal, defaultBlessingName())
+				return principal, nil, vsecurity.InitDefaultBlessings(principal, defaultBlessingName())
 			}
-			return nil, err
+			return nil, nil, err
 		}
-		return principal, nil
+		return principal, nil, nil
 	}
+	// Use credentials stored in the agent.
 	if ep, _, err := agentEP(); err != nil {
-		return nil, err
+		return nil, nil, err
 	} else if ep != nil {
-		return agentlib.NewAgentPrincipal(ctx, ep, client)
+		// Use a new stream manager and an "incomplete" client (the
+		// principal is nil) to talk to the agent.
+		//
+		// The lack of a principal works out for the rpc.Client
+		// only because the agent uses anonymous unix sockets and
+		// the SecurityNone option.
+		//
+		// Using a distinct stream manager to manage agent-related
+		// connections helps isolate these connections to the agent
+		// from management of any other connections created in the
+		// process (such as future RPCs to other services).
+		if ctx, err = r.WithNewStreamManager(ctx); err != nil {
+			return nil, nil, err
+		}
+		client := r.GetClient(ctx)
+		if principal, err = agentlib.NewAgentPrincipal(ctx, ep, client); err != nil {
+			client.Close()
+			return nil, nil, err
+		}
+		return principal, []interface{}{client}, nil
 	}
+	// No agent, no explicit credentials specified: - create a new principal and blessing in memory.
 	if principal, err = vsecurity.NewPrincipal(); err != nil {
-		return principal, err
+		return principal, nil, err
 	}
-	return principal, vsecurity.InitDefaultBlessings(principal, defaultBlessingName())
+	return principal, nil, vsecurity.InitDefaultBlessings(principal, defaultBlessingName())
 }
 
 func parseAgentFD(ep naming.Endpoint) (int, error) {
diff --git a/profiles/internal/util.go b/profiles/internal/util.go
index 600ff0a..a530087 100644
--- a/profiles/internal/util.go
+++ b/profiles/internal/util.go
@@ -6,10 +6,10 @@
 
 import (
 	"fmt"
+	"net"
 	"os"
 	"strings"
 
-	"v.io/v23/rpc"
 	"v.io/v23/verror"
 	"v.io/x/lib/vlog"
 
@@ -46,11 +46,11 @@
 // IPAddressChooser returns the preferred IP address, which is,
 // a public IPv4 address, then any non-loopback IPv4, then a public
 // IPv6 address and finally any non-loopback/link-local IPv6
-func IPAddressChooser(network string, addrs []rpc.Address) ([]rpc.Address, error) {
+func IPAddressChooser(network string, addrs []net.Addr) ([]net.Addr, error) {
 	if !netstate.IsIPProtocol(network) {
 		return nil, fmt.Errorf("can't support network protocol %q", network)
 	}
-	accessible := netstate.AddrList(addrs)
+	accessible := netstate.ConvertToAddresses(addrs)
 
 	// Try and find an address on a interface with a default route.
 	// We give preference to IPv4 over IPv6 for compatibility for now.
@@ -65,7 +65,7 @@
 		if addrs := accessible.Filter(predicate); len(addrs) > 0 {
 			onDefaultRoutes := addrs.Filter(netstate.IsOnDefaultRoute)
 			if len(onDefaultRoutes) > 0 {
-				return onDefaultRoutes, nil
+				return onDefaultRoutes.AsNetAddrs(), nil
 			}
 		}
 	}
@@ -74,11 +74,10 @@
 	// but without the default route requirement.
 	for _, predicate := range predicates {
 		if addrs := accessible.Filter(predicate); len(addrs) > 0 {
-			return addrs, nil
+			return addrs.AsNetAddrs(), nil
 		}
 	}
-
-	return nil, fmt.Errorf("failed to find any usable address for %q", network)
+	return []net.Addr{}, nil
 }
 
 // HasPublicIP returns true if the host has at least one public IP address.
diff --git a/profiles/proxy.go b/profiles/proxy.go
index 3392743..0a12b4e 100644
--- a/profiles/proxy.go
+++ b/profiles/proxy.go
@@ -7,12 +7,13 @@
 import (
 	"v.io/v23/context"
 	"v.io/v23/naming"
+	"v.io/v23/rpc"
 
 	"v.io/x/ref/profiles/internal/rpc/stream/proxy"
 )
 
 // NewProxy creates a new Proxy that listens for network connections on the provided
 // (network, address) pair and routes VC traffic between accepted connections.
-func NewProxy(ctx *context.T, network, address, pubAddress string, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
-	return proxy.New(ctx, network, address, pubAddress, names...)
+func NewProxy(ctx *context.T, spec rpc.ListenSpec, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
+	return proxy.New(ctx, spec, names...)
 }
diff --git a/profiles/roaming/net_watcher.go b/profiles/roaming/net_watcher.go
index 579fafb..f4880d4 100644
--- a/profiles/roaming/net_watcher.go
+++ b/profiles/roaming/net_watcher.go
@@ -8,12 +8,14 @@
 
 import (
 	"fmt"
+	"os"
 	"strings"
 
+	"v.io/x/lib/netstate"
+
 	"v.io/v23"
 	"v.io/v23/config"
 
-	"v.io/x/lib/netstate"
 	"v.io/x/ref/profiles/roaming"
 )
 
@@ -25,8 +27,26 @@
 	fmt.Println("Profile: ", profileName)
 
 	accessible, err := netstate.GetAccessibleIPs()
-	routes := netstate.GetRoutes()
-	fmt.Printf("Routes:\n%s\n", strings.Replace(routes.String(), ")", ")\n", -1))
+	interfaces, err := netstate.GetAllInterfaces()
+
+	fmt.Printf("Addresses\n")
+	for _, addr := range accessible {
+		fmt.Printf("%s\n", addr.DebugString())
+	}
+
+	fmt.Printf("\nInterfaces\n")
+	for _, ifc := range interfaces {
+		fmt.Printf("%s\n", ifc)
+	}
+
+	fmt.Printf("\nRoutes\n")
+	for _, ifc := range interfaces {
+		if ipifc, ok := ifc.(netstate.IPNetworkInterface); ok {
+			if routes := ipifc.IPRoutes(); len(routes) > 0 {
+				fmt.Printf("%s: %s\n", ifc.Name(), routes)
+			}
+		}
+	}
 
 	listenSpec := v23.GetListenSpec(ctx)
 	chooser := listenSpec.AddressChooser
@@ -36,17 +56,17 @@
 		}
 	}
 
-	if chosen, err := listenSpec.AddressChooser("tcp", accessible); err != nil {
+	if chosen, err := listenSpec.AddressChooser("tcp", accessible.AsNetAddrs()); err != nil {
 		fmt.Printf("Failed to chosen address %s\n", err)
 	} else {
-		al := netstate.AddrList(chosen)
+		al := netstate.ConvertToAddresses(chosen)
 		fmt.Printf("Chosen:\n%s\n", strings.Replace(al.String(), ") ", ")\n", -1))
 	}
 
 	ch := make(chan config.Setting, 10)
-	settings, err := v23.GetPublisher(ctx).ForkStream(roaming.SettingsStreamName, ch)
+	settings, err := listenSpec.StreamPublisher.ForkStream(roaming.SettingsStreamName, ch)
 	if err != nil {
-		r.Logger().Infof("failed to fork stream: %s", err)
+		fmt.Fprintf(os.Stderr, "failed to fork stream: %s\n", err)
 	}
 	for _, setting := range settings.Latest {
 		fmt.Println("Setting: ", setting)
diff --git a/profiles/roaming/proxy.go b/profiles/roaming/proxy.go
index ed9a6b6..845ad11 100644
--- a/profiles/roaming/proxy.go
+++ b/profiles/roaming/proxy.go
@@ -7,12 +7,13 @@
 import (
 	"v.io/v23/context"
 	"v.io/v23/naming"
+	"v.io/v23/rpc"
 
 	"v.io/x/ref/profiles/internal/rpc/stream/proxy"
 )
 
 // NewProxy creates a new Proxy that listens for network connections on the provided
 // (network, address) pair and routes VC traffic between accepted connections.
-func NewProxy(ctx *context.T, network, address, pubAddress string, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
-	return proxy.New(ctx, network, address, pubAddress, names...)
+func NewProxy(ctx *context.T, spec rpc.ListenSpec, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
+	return proxy.New(ctx, spec, names...)
 }
diff --git a/profiles/roaming/roaminginit.go b/profiles/roaming/roaminginit.go
index f09f1c6..4996320 100644
--- a/profiles/roaming/roaminginit.go
+++ b/profiles/roaming/roaminginit.go
@@ -15,14 +15,17 @@
 
 import (
 	"flag"
+	"net"
+
+	"v.io/x/lib/netconfig"
+	"v.io/x/lib/netstate"
+	"v.io/x/lib/vlog"
 
 	"v.io/v23"
 	"v.io/v23/config"
 	"v.io/v23/context"
 	"v.io/v23/rpc"
-	"v.io/x/lib/netconfig"
-	"v.io/x/lib/netstate"
-	"v.io/x/lib/vlog"
+
 	"v.io/x/ref/lib/flags"
 	"v.io/x/ref/lib/security/securityflag"
 	"v.io/x/ref/profiles/internal"
@@ -65,8 +68,11 @@
 	// 1:1 NAT configuration.
 	if !internal.HasPublicIP(vlog.Log) {
 		if addr := internal.GCEPublicAddress(vlog.Log); addr != nil {
-			listenSpec.AddressChooser = func(string, []rpc.Address) ([]rpc.Address, error) {
-				return []rpc.Address{&netstate.AddrIfc{addr, "nat", nil}}, nil
+			listenSpec.AddressChooser = func(string, []net.Addr) ([]net.Addr, error) {
+				// TODO(cnicolaou): the protocol at least should
+				// be configurable, or maybe there's a profile specific
+				// flag to configure both the protocol and address.
+				return []net.Addr{netstate.NewNetAddr("wsh", addr.String())}, nil
 			}
 			runtime, ctx, shutdown, err := rt.Init(ctx, ac, nil, &listenSpec, commonFlags.RuntimeFlags(), reservedDispatcher)
 			if err != nil {
@@ -146,6 +152,7 @@
 	for {
 		select {
 		case <-watcher.Channel():
+			netstate.InvalidateCache()
 			cur, err := netstate.GetAccessibleIPs()
 			if err != nil {
 				vlog.Errorf("failed to read network state: %s", err)
@@ -163,10 +170,10 @@
 			}
 			if len(removed) > 0 {
 				vlog.VI(2).Infof("Sending removed: %s", removed)
-				ch <- rpc.NewRmAddrsSetting(removed)
+				ch <- rpc.NewRmAddrsSetting(removed.AsNetAddrs())
 			}
 			// We will always send the best currently available address
-			if chosen, err := listenSpec.AddressChooser(listenSpec.Addrs[0].Protocol, cur); err == nil && chosen != nil {
+			if chosen, err := listenSpec.AddressChooser(listenSpec.Addrs[0].Protocol, cur.AsNetAddrs()); err == nil && chosen != nil {
 				vlog.VI(2).Infof("Sending added and chosen: %s", chosen)
 				ch <- rpc.NewAddAddrsSetting(chosen)
 			} else {
diff --git a/profiles/static/proxy.go b/profiles/static/proxy.go
index e2617e0..a398460 100644
--- a/profiles/static/proxy.go
+++ b/profiles/static/proxy.go
@@ -7,12 +7,13 @@
 import (
 	"v.io/v23/context"
 	"v.io/v23/naming"
+	"v.io/v23/rpc"
 
 	"v.io/x/ref/profiles/internal/rpc/stream/proxy"
 )
 
 // NewProxy creates a new Proxy that listens for network connections on the provided
 // (network, address) pair and routes VC traffic between accepted connections.
-func NewProxy(ctx *context.T, network, address, pubAddress string, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
-	return proxy.New(ctx, network, address, pubAddress, names...)
+func NewProxy(ctx *context.T, spec rpc.ListenSpec, names ...string) (shutdown func(), endpoint naming.Endpoint, err error) {
+	return proxy.New(ctx, spec, names...)
 }
diff --git a/profiles/static/staticinit.go b/profiles/static/staticinit.go
index 1908308..339e32a 100644
--- a/profiles/static/staticinit.go
+++ b/profiles/static/staticinit.go
@@ -7,12 +7,14 @@
 
 import (
 	"flag"
+	"net"
+
+	"v.io/x/lib/vlog"
 
 	"v.io/v23"
 	"v.io/v23/context"
 	"v.io/v23/rpc"
-	"v.io/x/lib/netstate"
-	"v.io/x/lib/vlog"
+
 	"v.io/x/ref/lib/flags"
 	"v.io/x/ref/lib/security/securityflag"
 	"v.io/x/ref/profiles/internal"
@@ -48,11 +50,12 @@
 	ac := appcycle.New()
 
 	// Our address is private, so we test for running on GCE and for its 1:1 NAT
-	// configuration. GCEPublicAddress returns a non-nil addr if we are running on GCE.
+	// configuration. GCEPublicAddress returns a non-nil addr if we are
+	// running on GCE.
 	if !internal.HasPublicIP(vlog.Log) {
 		if addr := internal.GCEPublicAddress(vlog.Log); addr != nil {
-			listenSpec.AddressChooser = func(string, []rpc.Address) ([]rpc.Address, error) {
-				return []rpc.Address{&netstate.AddrIfc{addr, "nat", nil}}, nil
+			listenSpec.AddressChooser = func(string, []net.Addr) ([]net.Addr, error) {
+				return []net.Addr{addr}, nil
 			}
 			runtime, ctx, shutdown, err := rt.Init(ctx, ac, nil, &listenSpec, commonFlags.RuntimeFlags(), reservedDispatcher)
 			if err != nil {
diff --git a/services/binary/binaryd/main.go b/services/binary/binaryd/main.go
index e5fdc80..16cd87c 100644
--- a/services/binary/binaryd/main.go
+++ b/services/binary/binaryd/main.go
@@ -32,6 +32,7 @@
 // toIPPort tries to swap in the 'best' accessible IP for the host part of the
 // address, if the provided address has an unspecified IP.
 func toIPPort(ctx *context.T, addr string) string {
+	// TODO(caprita): consider using netstate.PossibleAddresses()
 	host, port, err := net.SplitHostPort(addr)
 	if err != nil {
 		vlog.Errorf("SplitHostPort(%v) failed: %v", addr, err)
@@ -43,8 +44,8 @@
 		ips, err := netstate.GetAccessibleIPs()
 		if err == nil {
 			ls := v23.GetListenSpec(ctx)
-			if a, err := ls.AddressChooser("tcp", ips); err == nil && len(a) > 0 {
-				host = a[0].Address().String()
+			if a, err := ls.AddressChooser("tcp", ips.AsNetAddrs()); err == nil && len(a) > 0 {
+				host = a[0].String()
 			}
 		}
 	}
diff --git a/services/device/internal/starter/starter.go b/services/device/internal/starter/starter.go
index a28fe26..654d0df 100644
--- a/services/device/internal/starter/starter.go
+++ b/services/device/internal/starter/starter.go
@@ -14,7 +14,6 @@
 	"strconv"
 	"time"
 
-	"v.io/x/lib/netstate"
 	"v.io/x/ref/profiles/roaming"
 	"v.io/x/ref/services/debug/debuglib"
 	"v.io/x/ref/services/device/internal/config"
@@ -282,17 +281,9 @@
 	protocol, addr := "tcp", net.JoinHostPort("", port)
 	// Attempt to get a publicly accessible address for the proxy to publish
 	// under.
-	var publishAddr string
 	ls := v23.GetListenSpec(ctx)
-	if addrs, err := netstate.GetAccessibleIPs(); err == nil {
-		if ac := ls.AddressChooser; ac != nil {
-			if a, err := ac(protocol, addrs); err == nil && len(a) > 0 {
-				addrs = a
-			}
-		}
-		publishAddr = net.JoinHostPort(addrs[0].Address().String(), port)
-	}
-	shutdown, ep, err := roaming.NewProxy(ctx, protocol, addr, publishAddr)
+	ls.Addrs = rpc.ListenAddrs{{protocol, addr}}
+	shutdown, ep, err := roaming.NewProxy(ctx, ls)
 	if err != nil {
 		return nil, verror.New(errCantCreateProxy, ctx, err)
 	}
diff --git a/services/internal/servicetest/modules.go b/services/internal/servicetest/modules.go
index 13b14a0..6cc9400 100644
--- a/services/internal/servicetest/modules.go
+++ b/services/internal/servicetest/modules.go
@@ -101,12 +101,8 @@
 	// The shell, will, by default share credentials with its children.
 	sh.ClearVar(envvar.Credentials)
 
-	mtName, mtHandle := startRootMT(t, sh)
+	mtName, _ := startRootMT(t, sh)
 	vlog.VI(1).Infof("Started shell mounttable with name %v", mtName)
-	// Make sure the root mount table is the last process to be shutdown
-	// since the others will likely want to communicate with it during
-	// their shutdown process
-	sh.Forget(mtHandle)
 
 	// TODO(caprita): Define a GetNamespaceRootsCommand in modules/core and
 	// use that?
@@ -117,15 +113,9 @@
 		vlog.VI(1).Info("---------------------------------")
 		vlog.VI(1).Info("--(cleaning up shell)------------")
 		if err := sh.Cleanup(os.Stdout, os.Stderr); err != nil {
-			t.Fatalf(testutil.FormatLogLine(2, "sh.Cleanup failed with %v", err))
+			t.Errorf(testutil.FormatLogLine(2, "sh.Cleanup failed with %v", err))
 		}
 		vlog.VI(1).Info("--(done cleaning up shell)-------")
-		vlog.VI(1).Info("--(shutting down root mt)--------")
-		if err := mtHandle.Shutdown(os.Stdout, os.Stderr); err != nil {
-			t.Fatalf(testutil.FormatLogLine(2, "mtHandle.Shutdown failed with %v", err))
-		}
-		vlog.VI(1).Info("--(done shutting down root mt)---")
-		vlog.VI(1).Info("--------- DONE CLEANUP ----------")
 		setNSRoots(t, ctx, oldNamespaceRoots...)
 	}
 	setNSRoots(t, ctx, mtName)
diff --git a/services/proxy/proxyd/main.go b/services/proxy/proxyd/main.go
index c454b68..d9114eb 100644
--- a/services/proxy/proxyd/main.go
+++ b/services/proxy/proxyd/main.go
@@ -23,7 +23,7 @@
 )
 
 var (
-	pubAddress  = flag.String("published-address", "", "Network address the proxy publishes. If empty, the value of --address will be used")
+	pubAddress  = flag.String("published-address", "", "deprecated - the proxy now uses listenspecs and the address chooser mechanism")
 	healthzAddr = flag.String("healthz-address", "", "Network address on which the HTTP healthz server runs. It is intended to be used with a load balancer. The load balancer must be able to reach this address in order to verify that the proxy server is running")
 	name        = flag.String("name", "", "Name to mount the proxy as")
 )
@@ -39,7 +39,7 @@
 	if listenSpec.Proxy != "" {
 		vlog.Fatalf("proxyd cannot listen through another proxy")
 	}
-	proxyShutdown, proxyEndpoint, err := static.NewProxy(ctx, listenSpec.Addrs[0].Protocol, listenSpec.Addrs[0].Address, *pubAddress, *name)
+	proxyShutdown, proxyEndpoint, err := static.NewProxy(ctx, listenSpec, *name)
 	if err != nil {
 		vlog.Fatal(err)
 	}
@@ -49,6 +49,8 @@
 		// Print out a directly accessible name for the proxy table so
 		// that integration tests can reliably read it from stdout.
 		fmt.Printf("NAME=%s\n", proxyEndpoint.Name())
+	} else {
+		fmt.Printf("Proxy listening on %s\n", proxyEndpoint)
 	}
 
 	if len(*healthzAddr) != 0 {
diff --git a/services/wspr/internal/app/app_test.go b/services/wspr/internal/app/app_test.go
index aca197c..6e14735 100644
--- a/services/wspr/internal/app/app_test.go
+++ b/services/wspr/internal/app/app_test.go
@@ -315,8 +315,8 @@
 	if err != nil {
 		return nil, fmt.Errorf("unable to start mounttable: %v", err)
 	}
-
-	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, "tcp", "127.0.0.1:0", "")
+	proxySpec := rpc.ListenSpec{Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}}}
+	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, proxySpec)
 	if err != nil {
 		return nil, fmt.Errorf("unable to start proxy: %v", err)
 	}
diff --git a/services/wspr/internal/browspr/browspr_test.go b/services/wspr/internal/browspr/browspr_test.go
index 76b6185..bbb0e0a 100644
--- a/services/wspr/internal/browspr/browspr_test.go
+++ b/services/wspr/internal/browspr/browspr_test.go
@@ -82,7 +82,8 @@
 	ctx, shutdown := test.InitForTest()
 	defer shutdown()
 
-	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, "tcp", "127.0.0.1:0", "")
+	proxySpec := rpc.ListenSpec{Addrs: rpc.ListenAddrs{{"tcp", "127.0.0.1:0"}}}
+	proxyShutdown, proxyEndpoint, err := profiles.NewProxy(ctx, proxySpec)
 	if err != nil {
 		t.Fatalf("Failed to start proxy: %v", err)
 	}