discovery: introduce session to discovery

  Currently we have only one discovery instance per runtime and all
  discovery clients share the same instance. Although this makes sense
  and required, it may cause inconvenience in some cases like when a
  user wants to excludes services that are advertised by the same
  discovery client.

  This change introduces 'session' to discovery so that we can
  distinguish discovery clients including

    o simplify discovery.T interface
      - remove unnecessary/not-used interfaces.
    o make discovery.T as a sessioned discovery and change v23 API for
      getting discovery instance.
    o change scan() to filter out advertisements from the same discovery.T
      instance.
    o remove discovery service
      - the current interface is not sufficient. E.g., there is no way
        to remove obsolete advertisement.
      - discovery as a service doesn't make sense for now.
      - if needed, we can add it again with more complete
        api/implementation later.

MultiPart: 2/4
Change-Id: I3cd7f60150b0b3265c598a968564bd2d2185e514
diff --git a/lib/discovery/advertise.go b/lib/discovery/advertise.go
index e893a87..c0e6130 100644
--- a/lib/discovery/advertise.go
+++ b/lib/discovery/advertise.go
@@ -16,8 +16,7 @@
 	errInvalidService         = verror.Register(pkgPath+"errInvalidService", verror.NoRetry, "{1:}{2:} service not valid{:_}")
 )
 
-// Advertise implements discovery.Advertiser.
-func (ds *ds) Advertise(ctx *context.T, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
+func (d *idiscovery) advertise(ctx *context.T, session sessionId, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
 	if len(service.InstanceId) == 0 {
 		var err error
 		if service.InstanceId, err = newInstanceId(); err != nil {
@@ -34,24 +33,24 @@
 	}
 	hashAdvertisement(&ad)
 
-	ctx, cancel, err := ds.addTask(ctx)
+	ctx, cancel, err := d.addTask(ctx)
 	if err != nil {
 		return nil, err
 	}
 
-	if !ds.addAd(ad.Service.InstanceId) {
+	if !d.addAd(session, ad.Service.InstanceId) {
 		cancel()
-		ds.removeTask(ctx)
+		d.removeTask(ctx)
 		return nil, verror.New(errAlreadyBeingAdvertised, ctx)
 	}
 
 	done := make(chan struct{})
 	barrier := NewBarrier(func() {
-		ds.removeAd(ad.Service.InstanceId)
-		ds.removeTask(ctx)
+		d.removeAd(ad.Service.InstanceId)
+		d.removeTask(ctx)
 		close(done)
 	})
-	for _, plugin := range ds.plugins {
+	for _, plugin := range d.plugins {
 		if err := plugin.Advertise(ctx, ad, barrier.Add()); err != nil {
 			cancel()
 			return nil, err
@@ -60,19 +59,19 @@
 	return done, nil
 }
 
-func (ds *ds) addAd(id string) bool {
-	ds.mu.Lock()
-	if _, exist := ds.ads[id]; exist {
-		ds.mu.Unlock()
+func (d *idiscovery) addAd(session sessionId, id string) bool {
+	d.mu.Lock()
+	if _, exist := d.ads[id]; exist {
+		d.mu.Unlock()
 		return false
 	}
-	ds.ads[id] = struct{}{}
-	ds.mu.Unlock()
+	d.ads[id] = session
+	d.mu.Unlock()
 	return true
 }
 
-func (ds *ds) removeAd(id string) {
-	ds.mu.Lock()
-	delete(ds.ads, id)
-	ds.mu.Unlock()
+func (d *idiscovery) removeAd(id string) {
+	d.mu.Lock()
+	delete(d.ads, id)
+	d.mu.Unlock()
 }
diff --git a/lib/discovery/advertise_server.go b/lib/discovery/advertise_server.go
index a65b907..cf7e6fb 100644
--- a/lib/discovery/advertise_server.go
+++ b/lib/discovery/advertise_server.go
@@ -18,12 +18,21 @@
 // addresses will be updated automatically when the underlying network are
 // changed. Advertising will continue until the context is canceled or exceeds
 // its deadline and the returned channel will be closed when it stops.
-func AdvertiseServer(ctx *context.T, server rpc.Server, suffix string, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
+//
+// If a discovery instance is not provided, this will create a new one.
+func AdvertiseServer(ctx *context.T, d discovery.T, server rpc.Server, suffix string, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
+	if d == nil {
+		var err error
+		d, err = v23.NewDiscovery(ctx)
+		if err != nil {
+			return nil, err
+		}
+	}
 	// Take a copy of the service to avoid any interference from changes in user side.
 	copiedService := copyService(service)
 
 	eps, valid := getEndpoints(server)
-	stop, err := advertiseServer(ctx, &copiedService, eps, suffix, visibility)
+	stop, err := advertiseServer(ctx, d, &copiedService, eps, suffix, visibility)
 	if err != nil {
 		return nil, err
 	}
@@ -40,7 +49,7 @@
 					stop() // Stop the previous advertisement.
 				}
 				eps, valid = getEndpoints(server)
-				stop, err = advertiseServer(ctx, &copiedService, eps, suffix, visibility)
+				stop, err = advertiseServer(ctx, d, &copiedService, eps, suffix, visibility)
 				if err != nil {
 					ctx.Error(err)
 				}
@@ -54,14 +63,13 @@
 	return done, nil
 }
 
-func advertiseServer(ctx *context.T, service *discovery.Service, eps []naming.Endpoint, suffix string, visibility []security.BlessingPattern) (func(), error) {
+func advertiseServer(ctx *context.T, d discovery.T, service *discovery.Service, eps []naming.Endpoint, suffix string, visibility []security.BlessingPattern) (func(), error) {
 	service.Addrs = make([]string, len(eps))
 	for i, ep := range eps {
 		service.Addrs[i] = naming.JoinAddressName(ep.Name(), suffix)
 	}
-	ds := v23.GetDiscovery(ctx)
 	ctx, cancel := context.WithCancel(ctx)
-	done, err := ds.Advertise(ctx, service, visibility)
+	done, err := d.Advertise(ctx, service, visibility)
 	if err != nil {
 		cancel()
 		return nil, err
diff --git a/lib/discovery/advertise_server_test.go b/lib/discovery/advertise_server_test.go
index 1155de1..da7976a 100644
--- a/lib/discovery/advertise_server_test.go
+++ b/lib/discovery/advertise_server_test.go
@@ -70,11 +70,12 @@
 }
 
 func TestAdvertiseServer(t *testing.T) {
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()})
-	fdiscovery.InjectDiscovery(ds)
 	ctx, shutdown := test.V23Init()
 	defer shutdown()
 
+	df, _ := idiscovery.NewFactory(ctx, mock.New())
+	fdiscovery.InjectFactory(df)
+
 	const suffix = "test"
 
 	eps := newEndpoints("addr1:123")
@@ -85,13 +86,14 @@
 		Attrs:         discovery.Attributes{"a1": "v1"},
 	}
 
-	_, err := idiscovery.AdvertiseServer(ctx, mock, suffix, &service, nil)
+	_, err := idiscovery.AdvertiseServer(ctx, nil, mock, suffix, &service, nil)
 	if err != nil {
 		t.Fatal(err)
 	}
 
+	d, _ := v23.NewDiscovery(ctx)
 	setServiceAddrs(&service, eps, suffix)
-	if err := scanAndMatch(ctx, ds, "", service); err != nil {
+	if err := scanAndMatch(ctx, d, "", service); err != nil {
 		t.Error(err)
 	}
 
@@ -104,7 +106,7 @@
 		mock.updateNetwork(eps)
 
 		setServiceAddrs(&service, eps, suffix)
-		if err := scanAndMatch(ctx, ds, "", service); err != nil {
+		if err := scanAndMatch(ctx, d, "", service); err != nil {
 			t.Error(err)
 		}
 	}
diff --git a/lib/discovery/discovery.go b/lib/discovery/discovery.go
index 471087b..c316899 100644
--- a/lib/discovery/discovery.go
+++ b/lib/discovery/discovery.go
@@ -8,18 +8,17 @@
 	"sync"
 
 	"v.io/v23/context"
-	"v.io/v23/discovery"
 	"v.io/v23/verror"
 )
 
-const pkgPath = "v.io/x/ref/runtime/internal/discovery"
+const pkgPath = "v.io/x/ref/lib/discovery"
 
 var (
-	errDiscoveryClosed = verror.Register(pkgPath+".errDiscoveryClosed", verror.NoRetry, "{1:}{2:} discovery closed")
+	errNoDiscoveryPlugin = verror.Register(pkgPath+".errNoDiscoveryPlugin", verror.NoRetry, "{1:}{2:} no discovery plugin")
+	errDiscoveryClosed   = verror.Register(pkgPath+".errDiscoveryClosed", verror.NoRetry, "{1:}{2:} discovery closed")
 )
 
-// ds is an implementation of discovery.T.
-type ds struct {
+type idiscovery struct {
 	plugins []Plugin
 
 	mu     sync.Mutex
@@ -27,57 +26,54 @@
 	tasks  map[*context.T]func() // GUARDED_BY(mu)
 	wg     sync.WaitGroup
 
-	ads map[string]struct{} // GUARDED_BY(mu)
+	ads map[string]sessionId // GUARDED_BY(mu)
 }
 
-func (ds *ds) Close() {
-	ds.mu.Lock()
-	if ds.closed {
-		ds.mu.Unlock()
+func (d *idiscovery) shutdown() {
+	d.mu.Lock()
+	if d.closed {
+		d.mu.Unlock()
 		return
 	}
-	for _, cancel := range ds.tasks {
+	for _, cancel := range d.tasks {
 		cancel()
 	}
-	ds.closed = true
-	ds.mu.Unlock()
-	ds.wg.Wait()
+	d.closed = true
+	d.mu.Unlock()
+	d.wg.Wait()
 }
 
-func (ds *ds) addTask(ctx *context.T) (*context.T, func(), error) {
-	ds.mu.Lock()
-	if ds.closed {
-		ds.mu.Unlock()
+func (d *idiscovery) addTask(ctx *context.T) (*context.T, func(), error) {
+	d.mu.Lock()
+	if d.closed {
+		d.mu.Unlock()
 		return nil, nil, verror.New(errDiscoveryClosed, ctx)
 	}
 	ctx, cancel := context.WithCancel(ctx)
-	ds.tasks[ctx] = cancel
-	ds.wg.Add(1)
-	ds.mu.Unlock()
+	d.tasks[ctx] = cancel
+	d.wg.Add(1)
+	d.mu.Unlock()
 	return ctx, cancel, nil
 }
 
-func (ds *ds) removeTask(ctx *context.T) {
-	ds.mu.Lock()
-	if _, exist := ds.tasks[ctx]; exist {
-		delete(ds.tasks, ctx)
-		ds.wg.Done()
+func (d *idiscovery) removeTask(ctx *context.T) {
+	d.mu.Lock()
+	if _, exist := d.tasks[ctx]; exist {
+		delete(d.tasks, ctx)
+		d.wg.Done()
 	}
-	ds.mu.Unlock()
+	d.mu.Unlock()
 }
 
-// New returns a new Discovery instance initialized with the given plugins.
-//
-// Mostly for internal use. Consider to use factory.New.
-func NewWithPlugins(plugins []Plugin) discovery.T {
+func newDiscovery(ctx *context.T, plugins []Plugin) (*idiscovery, error) {
 	if len(plugins) == 0 {
-		panic("no plugins")
+		return nil, verror.New(errNoDiscoveryPlugin, ctx)
 	}
-	ds := &ds{
+	d := &idiscovery{
 		plugins: make([]Plugin, len(plugins)),
 		tasks:   make(map[*context.T]func()),
-		ads:     make(map[string]struct{}),
+		ads:     make(map[string]sessionId),
 	}
-	copy(ds.plugins, plugins)
-	return ds
+	copy(d.plugins, plugins)
+	return d, nil
 }
diff --git a/lib/discovery/discovery_test.go b/lib/discovery/discovery_test.go
index fd586c9..0717a70 100644
--- a/lib/discovery/discovery_test.go
+++ b/lib/discovery/discovery_test.go
@@ -23,8 +23,11 @@
 	ctx, shutdown := test.V23Init()
 	defer shutdown()
 
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()})
-	defer ds.Close()
+	df, err := idiscovery.NewFactory(ctx, mock.New())
+	if err != nil {
+		t.Fatal(err)
+	}
+	defer df.Shutdown()
 
 	services := []discovery.Service{
 		{
@@ -39,31 +42,47 @@
 			Addrs:         []string{"/h1:123/x", "/h2:123/z"},
 		},
 	}
+
+	d1, err := df.New()
+	if err != nil {
+		t.Fatal(err)
+	}
+
 	var stops []func()
 	for i, _ := range services {
-		stop, err := advertise(ctx, ds, nil, &services[i])
+		stop, err := advertise(ctx, d1, nil, &services[i])
 		if err != nil {
 			t.Fatal(err)
 		}
 		stops = append(stops, stop)
 	}
 
-	// Make sure all advertisements are discovered.
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a", services[0]); err != nil {
+	// Make sure none of advertisements are discoverable by the same discovery instance.
+	if err := scanAndMatch(ctx, d1, ""); err != nil {
 		t.Error(err)
 	}
-	if err := scanAndMatch(ctx, ds, "v.io/v23/b", services[1]); err != nil {
+
+	// Create a new discovery instance. All advertisements should be discovered with that.
+	d2, err := df.New()
+	if err != nil {
+		t.Fatal(err)
+	}
+
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a", services[0]); err != nil {
 		t.Error(err)
 	}
-	if err := scanAndMatch(ctx, ds, "", services...); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/b", services[1]); err != nil {
 		t.Error(err)
 	}
-	if err := scanAndMatch(ctx, ds, "v.io/v23/c"); err != nil {
+	if err := scanAndMatch(ctx, d2, "", services...); err != nil {
+		t.Error(err)
+	}
+	if err := scanAndMatch(ctx, d2, "v.io/v23/c"); err != nil {
 		t.Error(err)
 	}
 
 	// Open a new scan channel and consume expected advertisements first.
-	scan, scanStop, err := startScan(ctx, ds, "v.io/v23/a")
+	scan, scanStop, err := startScan(ctx, d2, "v.io/v23/a")
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -82,13 +101,13 @@
 	}
 
 	// Also it shouldn't affect the other.
-	if err := scanAndMatch(ctx, ds, "v.io/v23/b", services[1]); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/b", services[1]); err != nil {
 		t.Error(err)
 	}
 
 	// Stop advertising the remaining one; Shouldn't discover any service.
 	stops[1]()
-	if err := scanAndMatch(ctx, ds, ""); err != nil {
+	if err := scanAndMatch(ctx, d2, ""); err != nil {
 		t.Error(err)
 	}
 }
@@ -99,8 +118,8 @@
 	ctx, shutdown := test.V23Init()
 	defer shutdown()
 
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()})
-	defer ds.Close()
+	df, _ := idiscovery.NewFactory(ctx, mock.New())
+	defer df.Shutdown()
 
 	service := discovery.Service{
 		InterfaceName: "v.io/v23/a",
@@ -111,35 +130,39 @@
 		security.BlessingPattern("v.io:bob"),
 		security.BlessingPattern("v.io:alice").MakeNonExtendable(),
 	}
-	stop, err := advertise(ctx, ds, visibility, &service)
-	defer stop()
+
+	d1, _ := df.New()
+	stop, err := advertise(ctx, d1, visibility, &service)
 	if err != nil {
 		t.Fatal(err)
 	}
+	defer stop()
+
+	d2, _ := df.New()
 
 	// Bob and his friend should discover the advertisement.
 	ctx, _ = v23.WithPrincipal(ctx, testutil.NewPrincipal("v.io:bob"))
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a", service); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a", service); err != nil {
 		t.Error(err)
 	}
 	ctx, _ = v23.WithPrincipal(ctx, testutil.NewPrincipal("v.io:bob:friend"))
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a", service); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a", service); err != nil {
 		t.Error(err)
 	}
 
 	// Alice should discover the advertisement, but her friend shouldn't.
 	ctx, _ = v23.WithPrincipal(ctx, testutil.NewPrincipal("v.io:alice"))
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a", service); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a", service); err != nil {
 		t.Error(err)
 	}
 	ctx, _ = v23.WithPrincipal(ctx, testutil.NewPrincipal("v.io:alice:friend"))
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a"); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a"); err != nil {
 		t.Error(err)
 	}
 
 	// Other people shouldn't discover the advertisement.
 	ctx, _ = v23.WithPrincipal(ctx, testutil.NewPrincipal("v.io:carol"))
-	if err := scanAndMatch(ctx, ds, "v.io/v23/a"); err != nil {
+	if err := scanAndMatch(ctx, d2, "v.io/v23/a"); err != nil {
 		t.Error(err)
 	}
 }
@@ -148,8 +171,8 @@
 	ctx, shutdown := test.V23Init()
 	defer shutdown()
 
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()})
-	defer ds.Close()
+	df, _ := idiscovery.NewFactory(ctx, mock.New())
+	defer df.Shutdown()
 
 	service := discovery.Service{
 		InstanceId:    "123",
@@ -157,10 +180,11 @@
 		Addrs:         []string{"/h1:123/x"},
 	}
 
-	if _, err := advertise(ctx, ds, nil, &service); err != nil {
+	d, _ := df.New()
+	if _, err := advertise(ctx, d, nil, &service); err != nil {
 		t.Fatal(err)
 	}
-	if _, err := advertise(ctx, ds, nil, &service); err == nil {
+	if _, err := advertise(ctx, d, nil, &service); err == nil {
 		t.Error("expect an error; but got none")
 	}
 }
@@ -170,8 +194,8 @@
 	defer shutdown()
 
 	p1, p2 := mock.New(), mock.New()
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{p1, p2})
-	defer ds.Close()
+	df, _ := idiscovery.NewFactory(ctx, p1, p2)
+	defer df.Shutdown()
 
 	ad := idiscovery.Advertisement{
 		Service: discovery.Service{
@@ -182,7 +206,8 @@
 		Hash: []byte{1, 2, 3},
 	}
 
-	scan, scanStop, err := startScan(ctx, ds, "v.io/v23/a")
+	d, _ := df.New()
+	scan, scanStop, err := startScan(ctx, d, "v.io/v23/a")
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -227,34 +252,36 @@
 	}
 }
 
-func TestClose(t *testing.T) {
+func TestShutdown(t *testing.T) {
 	ctx, shutdown := test.V23Init()
 	defer shutdown()
 
-	ds := idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()})
+	df, _ := idiscovery.NewFactory(ctx, mock.New())
 
 	service := discovery.Service{
 		InterfaceName: "v.io/v23/a",
 		Addrs:         []string{"/h1:123/x"},
 	}
 
-	if _, err := advertise(ctx, ds, nil, &service); err != nil {
+	d1, _ := df.New()
+	if _, err := advertise(ctx, d1, nil, &service); err != nil {
 		t.Error(err)
 	}
-	if err := scanAndMatch(ctx, ds, "", service); err != nil {
+	d2, _ := df.New()
+	if err := scanAndMatch(ctx, d2, "", service); err != nil {
 		t.Error(err)
 	}
 
 	// Verify Close can be called multiple times.
-	ds.Close()
-	ds.Close()
+	df.Shutdown()
+	df.Shutdown()
 
 	// Make sure advertise and scan do not work after closed.
 	service.InstanceId = "" // To avoid dup error.
-	if _, err := advertise(ctx, ds, nil, &service); err == nil {
+	if _, err := advertise(ctx, d1, nil, &service); err == nil {
 		t.Error("expect an error; but got none")
 	}
-	if err := scanAndMatch(ctx, ds, "", service); err == nil {
+	if err := scanAndMatch(ctx, d2, "", service); err == nil {
 		t.Error("expect an error; but got none")
 	}
 }
diff --git a/lib/discovery/factory.go b/lib/discovery/factory.go
new file mode 100644
index 0000000..bfbb1f1
--- /dev/null
+++ b/lib/discovery/factory.go
@@ -0,0 +1,30 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package discovery
+
+import (
+	"v.io/v23/context"
+	"v.io/v23/discovery"
+)
+
+// Factory is the interface for creating a new discovery.T instance.
+type Factory interface {
+	// New creates a new Discovery.T instance.
+	New() (discovery.T, error)
+
+	// Shutdown closes all Discovery.T instances and shutdowns the factory.
+	Shutdown()
+}
+
+// NewFactory returns a new discovery factory with the given plugins.
+//
+// For internal use only.
+func NewFactory(ctx *context.T, plugins ...Plugin) (Factory, error) {
+	d, err := newDiscovery(ctx, plugins)
+	if err != nil {
+		return nil, err
+	}
+	return newSessionedDiscoveryFactory(d)
+}
diff --git a/lib/discovery/factory/factory.go b/lib/discovery/factory/factory.go
index 4a90c96..f029a89 100644
--- a/lib/discovery/factory/factory.go
+++ b/lib/discovery/factory/factory.go
@@ -5,23 +5,47 @@
 package factory
 
 import (
+	"errors"
 	"fmt"
 	"os"
+	"sync"
 
+	"v.io/v23/context"
 	"v.io/v23/discovery"
 
 	idiscovery "v.io/x/ref/lib/discovery"
 )
 
-// New returns a new Discovery instance with the given protocols.
-//
-// We instantiate a discovery instance lazily so that we do not turn it on
-// until it is actually used.
-func New(protocols ...string) (discovery.T, error) {
-	if injectedInstance != nil {
-		return injectedInstance, nil
-	}
+type lazyFactory struct {
+	ctx       *context.T
+	host      string
+	protocols []string
 
+	once sync.Once
+	d    idiscovery.Factory
+	err  error
+}
+
+func (f *lazyFactory) New() (discovery.T, error) {
+	f.once.Do(func() { f.d, f.err = newFactory(f.ctx, f.host, f.protocols) })
+	if f.err != nil {
+		return nil, f.err
+	}
+	return f.d.New()
+}
+
+func (f *lazyFactory) Shutdown() {
+	f.once.Do(func() { f.err = errors.New("factory closed") })
+	if f.d != nil {
+		f.d.Shutdown()
+	}
+}
+
+// New returns a new discovery factory with the given protocols.
+//
+// We instantiate a factory lazily so that we do not turn it on until
+// it is actually used.
+func New(ctx *context.T, protocols ...string) (idiscovery.Factory, error) {
 	host, _ := os.Hostname()
 	if len(host) == 0 {
 		// TODO(jhahn): Should we handle error here?
@@ -40,10 +64,14 @@
 		}
 	}
 
-	return newLazyFactory(func() (discovery.T, error) { return newInstance(host, protocols) }), nil
+	return &lazyFactory{ctx: ctx, host: host, protocols: protocols}, nil
 }
 
-func newInstance(host string, protocols []string) (discovery.T, error) {
+func newFactory(ctx *context.T, host string, protocols []string) (idiscovery.Factory, error) {
+	if injectedFactory != nil {
+		return injectedFactory, nil
+	}
+
 	plugins := make([]idiscovery.Plugin, 0, len(protocols))
 	for _, p := range protocols {
 		plugin, err := pluginFactories[p](host)
@@ -52,13 +80,13 @@
 		}
 		plugins = append(plugins, plugin)
 	}
-	return idiscovery.NewWithPlugins(plugins), nil
+	return idiscovery.NewFactory(ctx, plugins...)
 }
 
-var injectedInstance discovery.T
+var injectedFactory idiscovery.Factory
 
-// InjectDiscovery allows a runtime to use the given discovery instance. This
-// should be called before the runtime is initialized. Mostly used for testing.
-func InjectDiscovery(d discovery.T) {
-	injectedInstance = d
+// InjectFactory allows a runtime to use the given discovery factory. This
+// should be called before v23.NewDiscovery() is called. Mostly used for testing.
+func InjectFactory(factory idiscovery.Factory) {
+	injectedFactory = factory
 }
diff --git a/lib/discovery/factory/factory_test.go b/lib/discovery/factory/factory_test.go
new file mode 100644
index 0000000..fcc9838
--- /dev/null
+++ b/lib/discovery/factory/factory_test.go
@@ -0,0 +1,74 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package factory_test
+
+import (
+	"errors"
+	"testing"
+
+	"v.io/v23/discovery"
+
+	"v.io/x/ref/lib/discovery/factory"
+)
+
+type mock struct {
+	newErr                error
+	numNews, numShutdowns int
+}
+
+func (m *mock) New() (discovery.T, error) {
+	m.numNews++
+	return nil, m.newErr
+}
+
+func (m *mock) Shutdown() {
+	m.numShutdowns++
+}
+
+func TestFactoryBasic(t *testing.T) {
+	m := &mock{}
+	factory.InjectFactory(m)
+
+	f, _ := factory.New(nil)
+
+	for i := 0; i < 3; i++ {
+		_, err := f.New()
+		if err != nil {
+			t.Error(err)
+		}
+
+		if want := i + 1; m.numNews != want {
+			t.Errorf("expected %d New calls, but got %d times", want, m.numNews)
+		}
+	}
+
+	m.newErr = errors.New("new error")
+	if _, err := f.New(); err != m.newErr {
+		t.Error("expected an error %v, but got %v", m.newErr, err)
+	}
+
+	f.Shutdown()
+	if m.numShutdowns != 1 {
+		t.Errorf("expected one Shutdown call, but got %d times", m.numShutdowns)
+	}
+}
+
+func TestFactoryShutdownBeforeNew(t *testing.T) {
+	m := &mock{}
+	factory.InjectFactory(m)
+
+	f, _ := factory.New(nil)
+
+	f.Shutdown()
+	if _, err := f.New(); err == nil {
+		t.Error("expected an error, but got none")
+	}
+	if m.numNews != 0 {
+		t.Errorf("expected no New call, but got %d times", m.numNews)
+	}
+	if m.numShutdowns != 0 {
+		t.Errorf("expected no Shutdown call, but got %d times", m.numShutdowns)
+	}
+}
diff --git a/lib/discovery/factory/lazy.go b/lib/discovery/factory/lazy.go
deleted file mode 100644
index d7a874e..0000000
--- a/lib/discovery/factory/lazy.go
+++ /dev/null
@@ -1,63 +0,0 @@
-// Copyright 2015 The Vanadium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style
-// license that can be found in the LICENSE file.
-
-package factory
-
-import (
-	"errors"
-	"sync"
-
-	"v.io/v23/context"
-	"v.io/v23/discovery"
-	"v.io/v23/security"
-)
-
-var (
-	errClosed = errors.New("factory closed")
-)
-
-// lazyFactory is a simple wrapper that creates a new discovery instance lazily
-// when Advertise() or Scan() is called for the first time.
-type lazyFactory struct {
-	newInstance func() (discovery.T, error)
-
-	once sync.Once
-	d    discovery.T
-	derr error
-}
-
-func (l *lazyFactory) Advertise(ctx *context.T, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
-	l.once.Do(l.init)
-	if l.derr != nil {
-		return nil, l.derr
-	}
-	return l.d.Advertise(ctx, service, visibility)
-}
-
-func (l *lazyFactory) Scan(ctx *context.T, query string) (<-chan discovery.Update, error) {
-	l.once.Do(l.init)
-	if l.derr != nil {
-		return nil, l.derr
-	}
-	return l.d.Scan(ctx, query)
-}
-
-func (l *lazyFactory) Close() {
-	l.once.Do(l.noinit)
-	if l.d != nil {
-		l.d.Close()
-	}
-}
-
-func (l *lazyFactory) init() {
-	l.d, l.derr = l.newInstance()
-}
-
-func (l *lazyFactory) noinit() {
-	l.derr = errClosed
-}
-
-func newLazyFactory(newInstance func() (discovery.T, error)) discovery.T {
-	return &lazyFactory{newInstance: newInstance}
-}
diff --git a/lib/discovery/factory/lazy_test.go b/lib/discovery/factory/lazy_test.go
deleted file mode 100644
index 4d35f92..0000000
--- a/lib/discovery/factory/lazy_test.go
+++ /dev/null
@@ -1,132 +0,0 @@
-// Copyright 2015 The Vanadium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style
-// license that can be found in the LICENSE file.
-
-package factory
-
-import (
-	"errors"
-	"fmt"
-	"testing"
-
-	"v.io/v23/context"
-	"v.io/v23/discovery"
-	"v.io/v23/security"
-)
-
-type mock struct {
-	numInits, numAdvertises, numScans, numCloses int
-	initErr                                      error
-}
-
-func (m *mock) init() (discovery.T, error) {
-	m.numInits++
-	if m.initErr != nil {
-		return nil, m.initErr
-	}
-	return m, nil
-}
-
-func (m *mock) Advertise(_ *context.T, _ *discovery.Service, _ []security.BlessingPattern) (<-chan struct{}, error) {
-	m.numAdvertises++
-	return nil, nil
-}
-
-func (m *mock) Scan(_ *context.T, _ string) (<-chan discovery.Update, error) {
-	m.numScans++
-	return nil, nil
-}
-
-func (m *mock) Close() {
-	m.numCloses++
-}
-
-func (m *mock) check(numInits, numAdvertises, numScans, numCloses int) error {
-	switch {
-	case m.numInits != numInits:
-		return fmt.Errorf("expect %d init calls, but got %d times", numInits, m.numInits)
-	case m.numAdvertises != numAdvertises:
-		return fmt.Errorf("expect %d advertise calls, but got %d times", numAdvertises, m.numAdvertises)
-	case m.numScans != numScans:
-		return fmt.Errorf("expect %d scan calls, but got %d times", numScans, m.numScans)
-	case m.numCloses != numCloses:
-		return fmt.Errorf("expect %d close calls, but got %d times", numCloses, m.numCloses)
-	}
-	return nil
-}
-
-func TestLazyFactory(t *testing.T) {
-	m := mock{}
-	d := newLazyFactory(m.init)
-
-	if err := m.check(0, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-
-	for i := 0; i < 3; i++ {
-		d.Advertise(nil, nil, nil)
-		if err := m.check(1, i+1, i, i); err != nil {
-			t.Error(err)
-		}
-
-		d.Scan(nil, "")
-		if err := m.check(1, i+1, i+1, i); err != nil {
-			t.Error(err)
-		}
-
-		d.Close()
-		if err := m.check(1, i+1, i+1, i+1); err != nil {
-			t.Error(err)
-		}
-	}
-}
-
-func TestLazyFactoryClosed(t *testing.T) {
-	m := mock{}
-	d := newLazyFactory(m.init)
-
-	d.Close()
-	if err := m.check(0, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-
-	// Closed already; Shouldn't initialize it again.
-	if _, err := d.Advertise(nil, nil, nil); err != errClosed {
-		t.Errorf("expected an error %v, but got %v", errClosed, err)
-	}
-	if err := m.check(0, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-
-	if _, err := d.Scan(nil, ""); err != errClosed {
-		t.Errorf("expected an error %v, but got %v", errClosed, err)
-	}
-	if err := m.check(0, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-}
-
-func TestLazyFactoryInitError(t *testing.T) {
-	errInit := errors.New("test error")
-	m := mock{initErr: errInit}
-	d := newLazyFactory(m.init)
-
-	if _, err := d.Advertise(nil, nil, nil); err != errInit {
-		t.Errorf("expected an error %v, but got %v", errInit, err)
-	}
-	if err := m.check(1, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-
-	if _, err := d.Scan(nil, ""); err != errInit {
-		t.Errorf("expected an error %v, but got %v", errInit, err)
-	}
-	if err := m.check(1, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-
-	d.Close()
-	if err := m.check(1, 0, 0, 0); err != nil {
-		t.Error(err)
-	}
-}
diff --git a/lib/discovery/scan.go b/lib/discovery/scan.go
index 0527043..eb5f3f9 100644
--- a/lib/discovery/scan.go
+++ b/lib/discovery/scan.go
@@ -13,8 +13,7 @@
 	"v.io/v23/security"
 )
 
-// Scan implements discovery.Scanner.
-func (ds *ds) Scan(ctx *context.T, query string) (<-chan discovery.Update, error) {
+func (d *idiscovery) scan(ctx *context.T, session sessionId, query string) (<-chan discovery.Update, error) {
 	// TODO(jhahn): Consider to use multiple target services so that the plugins
 	// can filter advertisements more efficiently if possible.
 	matcher, targetInterfaceName, err := newMatcher(ctx, query)
@@ -22,7 +21,7 @@
 		return nil, err
 	}
 
-	ctx, cancel, err := ds.addTask(ctx)
+	ctx, cancel, err := d.addTask(ctx)
 	if err != nil {
 		return nil, err
 	}
@@ -31,9 +30,9 @@
 	scanCh := make(chan Advertisement, 10)
 	barrier := NewBarrier(func() {
 		close(scanCh)
-		ds.removeTask(ctx)
+		d.removeTask(ctx)
 	})
-	for _, plugin := range ds.plugins {
+	for _, plugin := range d.plugins {
 		if err := plugin.Scan(ctx, targetInterfaceName, scanCh, barrier.Add()); err != nil {
 			cancel()
 			return nil, err
@@ -41,11 +40,11 @@
 	}
 	// TODO(jhahn): Revisit the buffer size.
 	updateCh := make(chan discovery.Update, 10)
-	go doScan(ctx, matcher, scanCh, updateCh)
+	go d.doScan(ctx, session, matcher, scanCh, updateCh)
 	return updateCh, nil
 }
 
-func doScan(ctx *context.T, matcher matcher, scanCh <-chan Advertisement, updateCh chan<- discovery.Update) {
+func (d *idiscovery) doScan(ctx *context.T, session sessionId, matcher matcher, scanCh <-chan Advertisement, updateCh chan<- discovery.Update) {
 	defer close(updateCh)
 
 	// Get the blessing names belong to the principal.
@@ -70,6 +69,10 @@
 				}
 				continue
 			}
+			// Filter out advertisements from the same session.
+			if d.getAdSession(ad.Service.InstanceId) == session {
+				continue
+			}
 			// Note that 'Lost' advertisement may not have full service information.
 			// Thus we do not match the query against it. mergeAdvertisement() will
 			// ignore it if it has not been scanned.
@@ -91,6 +94,13 @@
 	}
 }
 
+func (d *idiscovery) getAdSession(id string) sessionId {
+	d.mu.Lock()
+	session := d.ads[id]
+	d.mu.Unlock()
+	return session
+}
+
 func mergeAdvertisement(found map[string]*Advertisement, ad *Advertisement) (updates []discovery.Update) {
 	// The multiple plugins may return the same advertisements. We ignores the update
 	// if it has been already sent through the update channel.
diff --git a/lib/discovery/session.go b/lib/discovery/session.go
new file mode 100644
index 0000000..0514431
--- /dev/null
+++ b/lib/discovery/session.go
@@ -0,0 +1,66 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package discovery
+
+import (
+	"errors"
+	"sync"
+
+	"v.io/v23/context"
+	"v.io/v23/discovery"
+	"v.io/v23/security"
+)
+
+type sessionId uint64
+
+// sdiscovery is an implementation of discovery.T.
+type sdiscovery struct {
+	d       *idiscovery
+	session sessionId
+}
+
+func (sd *sdiscovery) Advertise(ctx *context.T, service *discovery.Service, visibility []security.BlessingPattern) (<-chan struct{}, error) {
+	return sd.d.advertise(ctx, sd.session, service, visibility)
+}
+
+func (sd *sdiscovery) Scan(ctx *context.T, query string) (<-chan discovery.Update, error) {
+	return sd.d.scan(ctx, sd.session, query)
+}
+
+// sdFactory is an implementation of Factory.
+type sdFactory struct {
+	d *idiscovery
+
+	mu          sync.Mutex
+	lastSession sessionId // GUARDED_BY(mu)
+}
+
+func (f *sdFactory) New() (discovery.T, error) {
+	session, err := f.newSession()
+	if err != nil {
+		return nil, err
+	}
+	return &sdiscovery{d: f.d, session: session}, nil
+}
+
+func (f *sdFactory) Shutdown() {
+	f.d.shutdown()
+}
+
+func (f *sdFactory) newSession() (sessionId, error) {
+	f.mu.Lock()
+	session := f.lastSession + 1
+	if session == 0 {
+		f.mu.Unlock()
+		return 0, errors.New("session overflow")
+	}
+	f.lastSession = session
+	f.mu.Unlock()
+	return session, nil
+}
+
+func newSessionedDiscoveryFactory(d *idiscovery) (Factory, error) {
+	return &sdFactory{d: d}, nil
+}
diff --git a/lib/discovery/testutil_test.go b/lib/discovery/testutil_test.go
index fbd48f9..de60e20 100644
--- a/lib/discovery/testutil_test.go
+++ b/lib/discovery/testutil_test.go
@@ -19,13 +19,13 @@
 	_ "v.io/x/ref/runtime/factories/generic"
 )
 
-func advertise(ctx *context.T, ds discovery.Advertiser, visibility []security.BlessingPattern, services ...*discovery.Service) (func(), error) {
+func advertise(ctx *context.T, d discovery.T, visibility []security.BlessingPattern, services ...*discovery.Service) (func(), error) {
 	var wg sync.WaitGroup
 	tr := idiscovery.NewTrigger()
 	ctx, cancel := context.WithCancel(ctx)
 	for _, service := range services {
 		wg.Add(1)
-		done, err := ds.Advertise(ctx, service, visibility)
+		done, err := d.Advertise(ctx, service, visibility)
 		if err != nil {
 			cancel()
 			return nil, fmt.Errorf("Advertise failed: %v", err)
@@ -39,22 +39,22 @@
 	return stop, nil
 }
 
-func startScan(ctx *context.T, ds discovery.Scanner, interfaceName string) (<-chan discovery.Update, func(), error) {
+func startScan(ctx *context.T, d discovery.T, interfaceName string) (<-chan discovery.Update, func(), error) {
 	var query string
 	if len(interfaceName) > 0 {
 		query = `v.InterfaceName="` + interfaceName + `"`
 	}
 
 	ctx, stop := context.WithCancel(ctx)
-	scan, err := ds.Scan(ctx, query)
+	scan, err := d.Scan(ctx, query)
 	if err != nil {
 		return nil, nil, fmt.Errorf("Scan failed: %v", err)
 	}
 	return scan, stop, err
 }
 
-func scan(ctx *context.T, ds discovery.Scanner, interfaceName string) ([]discovery.Update, error) {
-	scan, stop, err := startScan(ctx, ds, interfaceName)
+func scan(ctx *context.T, d discovery.T, interfaceName string) ([]discovery.Update, error) {
+	scan, stop, err := startScan(ctx, d, interfaceName)
 	if err != nil {
 		return nil, err
 	}
@@ -71,7 +71,7 @@
 	}
 }
 
-func scanAndMatch(ctx *context.T, ds discovery.Scanner, interfaceName string, wants ...discovery.Service) error {
+func scanAndMatch(ctx *context.T, d discovery.T, interfaceName string, wants ...discovery.Service) error {
 	const timeout = 3 * time.Second
 
 	var updates []discovery.Update
@@ -79,7 +79,7 @@
 		runtime.Gosched()
 
 		var err error
-		updates, err = scan(ctx, ds, interfaceName)
+		updates, err = scan(ctx, d, interfaceName)
 		if err != nil {
 			return err
 		}
diff --git a/runtime/factories/android/android.go b/runtime/factories/android/android.go
index 7540987..5ef2684 100644
--- a/runtime/factories/android/android.go
+++ b/runtime/factories/android/android.go
@@ -50,7 +50,7 @@
 	}
 
 	ac := appcycle.New()
-	discovery, err := dfactory.New("mdns", "ble")
+	discoveryFactory, err := dfactory.New(ctx, "mdns", "ble")
 	if err != nil {
 		ac.Shutdown()
 		return nil, nil, nil, err
@@ -66,12 +66,12 @@
 
 	ishutdown := func() {
 		ac.Shutdown()
-		discovery.Close()
+		discoveryFactory.Shutdown()
 	}
 
 	publisher := pubsub.NewPublisher()
 
-	runtime, ctx, shutdown, err := rt.Init(ctx, ac, discovery, nil, &listenSpec, publisher, commonFlags.RuntimeFlags(), reservedDispatcher)
+	runtime, ctx, shutdown, err := rt.Init(ctx, ac, discoveryFactory, nil, &listenSpec, publisher, commonFlags.RuntimeFlags(), reservedDispatcher)
 	if err != nil {
 		ishutdown()
 		return nil, nil, nil, err
diff --git a/runtime/factories/fake/runtime.go b/runtime/factories/fake/runtime.go
index 3b12b47..86f87a6 100644
--- a/runtime/factories/fake/runtime.go
+++ b/runtime/factories/fake/runtime.go
@@ -60,11 +60,6 @@
 	panic("unimplemented")
 }
 
-func (r *Runtime) GetDiscovery(ctx *context.T) discovery.T {
-	// nologcall
-	panic("unimplemented")
-}
-
 func (r *Runtime) WithBackgroundContext(ctx *context.T) *context.T {
 	defer apilog.LogCall(ctx)(ctx) // gologcop: DO NOT EDIT, MUST BE FIRST STATEMENT
 	// Note we add an extra context with a nil value here.
@@ -87,6 +82,11 @@
 	return bctx
 }
 
+func (r *Runtime) NewDiscovery(ctx *context.T) (discovery.T, error) {
+	// nologcall
+	panic("unimplemented")
+}
+
 func (*Runtime) WithReservedNameDispatcher(ctx *context.T, d rpc.Dispatcher) *context.T {
 	defer apilog.LogCall(ctx)(ctx) // gologcop: DO NOT EDIT, MUST BE FIRST STATEMENT
 	panic("unimplemented")
diff --git a/runtime/factories/generic/generic.go b/runtime/factories/generic/generic.go
index 799e96d..c07704d 100644
--- a/runtime/factories/generic/generic.go
+++ b/runtime/factories/generic/generic.go
@@ -40,7 +40,7 @@
 	}
 
 	ac := appcycle.New()
-	discovery, err := dfactory.New()
+	discoveryFactory, err := dfactory.New(ctx)
 	if err != nil {
 		ac.Shutdown()
 		return nil, nil, nil, err
@@ -55,17 +55,10 @@
 
 	ishutdown := func() {
 		ac.Shutdown()
-		discovery.Close()
+		discoveryFactory.Shutdown()
 	}
 
-	runtime, ctx, shutdown, err := grt.Init(ctx,
-		ac,
-		discovery,
-		nil,
-		&listenSpec,
-		nil,
-		commonFlags.RuntimeFlags(),
-		nil)
+	runtime, ctx, shutdown, err := grt.Init(ctx, ac, discoveryFactory, nil, &listenSpec, nil, commonFlags.RuntimeFlags(), nil)
 	if err != nil {
 		ishutdown()
 		return nil, nil, nil, err
diff --git a/runtime/factories/roaming/roaming.go b/runtime/factories/roaming/roaming.go
index 873a074..9272b03 100644
--- a/runtime/factories/roaming/roaming.go
+++ b/runtime/factories/roaming/roaming.go
@@ -51,7 +51,7 @@
 	}
 
 	ac := appcycle.New()
-	discovery, err := dfactory.New()
+	discoveryFactory, err := dfactory.New(ctx)
 	if err != nil {
 		ac.Shutdown()
 		return nil, nil, nil, err
@@ -67,12 +67,12 @@
 
 	ishutdown := func() {
 		ac.Shutdown()
-		discovery.Close()
+		discoveryFactory.Shutdown()
 	}
 
 	publisher := pubsub.NewPublisher()
 
-	runtime, ctx, shutdown, err := rt.Init(ctx, ac, discovery, nil, &listenSpec, publisher, commonFlags.RuntimeFlags(), reservedDispatcher)
+	runtime, ctx, shutdown, err := rt.Init(ctx, ac, discoveryFactory, nil, &listenSpec, publisher, commonFlags.RuntimeFlags(), reservedDispatcher)
 	if err != nil {
 		ishutdown()
 		return nil, nil, nil, err
diff --git a/runtime/internal/rt/runtime.go b/runtime/internal/rt/runtime.go
index f302cae..781c7e0 100644
--- a/runtime/internal/rt/runtime.go
+++ b/runtime/internal/rt/runtime.go
@@ -30,6 +30,7 @@
 
 	"v.io/x/ref/internal/logger"
 	"v.io/x/ref/lib/apilog"
+	idiscovery "v.io/x/ref/lib/discovery"
 	"v.io/x/ref/lib/flags"
 	"v.io/x/ref/lib/pubsub"
 	"v.io/x/ref/lib/stats"
@@ -56,11 +57,15 @@
 	initKey
 )
 
+var (
+	errDiscoveryNotInitialized = verror.Register(pkgPath+".errDiscoveryNotInitialized", verror.NoRetry, "{1:}{2:} discovery not initialized")
+)
+
 var setPrincipalCounter int32 = -1
 
 type initData struct {
 	appCycle          v23.AppCycle
-	discovery         discovery.T
+	discoveryFactory  idiscovery.Factory
 	protocols         []string
 	settingsPublisher *pubsub.Publisher
 }
@@ -78,7 +83,7 @@
 func Init(
 	ctx *context.T,
 	appCycle v23.AppCycle,
-	discovery discovery.T,
+	discoveryFactory idiscovery.Factory,
 	protocols []string,
 	listenSpec *rpc.ListenSpec,
 	settingsPublisher *pubsub.Publisher,
@@ -88,7 +93,7 @@
 
 	ctx = context.WithValue(ctx, initKey, &initData{
 		appCycle:          appCycle,
-		discovery:         discovery,
+		discoveryFactory:  discoveryFactory,
 		protocols:         protocols,
 		settingsPublisher: settingsPublisher,
 	})
@@ -366,12 +371,6 @@
 	return context.WithValue(ctx, listenKey, ls.Copy())
 }
 
-func (*Runtime) GetDiscovery(ctx *context.T) discovery.T {
-	// nologcall
-	id, _ := ctx.Value(initKey).(*initData)
-	return id.discovery
-}
-
 func (*Runtime) WithBackgroundContext(ctx *context.T) *context.T {
 	defer apilog.LogCall(ctx)(ctx) // gologcop: DO NOT EDIT, MUST BE FIRST STATEMENT
 	// Note we add an extra context with a nil value here.
@@ -394,6 +393,15 @@
 	return bctx
 }
 
+func (*Runtime) NewDiscovery(ctx *context.T) (discovery.T, error) {
+	// nologcall
+	id, _ := ctx.Value(initKey).(*initData)
+	if id.discoveryFactory != nil {
+		return id.discoveryFactory.New()
+	}
+	return nil, verror.New(errDiscoveryNotInitialized, ctx)
+}
+
 func (*Runtime) WithReservedNameDispatcher(ctx *context.T, d rpc.Dispatcher) *context.T {
 	defer apilog.LogCall(ctx)(ctx) // gologcop: DO NOT EDIT, MUST BE FIRST STATEMENT
 	return context.WithValue(ctx, reservedNameKey, d)
diff --git a/services/discovery/service.go b/services/discovery/service.go
deleted file mode 100644
index 7eeeb3b..0000000
--- a/services/discovery/service.go
+++ /dev/null
@@ -1,103 +0,0 @@
-// Copyright 2015 The Vanadium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style
-// license that can be found in the LICENSE file.
-
-package discovery
-
-import (
-	"sync"
-
-	"v.io/v23"
-	"v.io/v23/context"
-	"v.io/v23/discovery"
-	"v.io/v23/rpc"
-	"v.io/v23/security"
-	sdiscovery "v.io/v23/services/discovery"
-	"v.io/v23/verror"
-)
-
-const pkgPath = "v.io/x/ref/services/discovery"
-
-const (
-	maxActiveHandles = int(^uint16(0)) // 65535.
-)
-
-var (
-	errTooManyServices = verror.Register(pkgPath+".errTooManyServices", verror.NoRetry, "{1:}{2:} too many registered services")
-)
-
-type impl struct {
-	ctx *context.T
-	d   discovery.T
-
-	mu         sync.Mutex
-	handles    map[sdiscovery.ServiceHandle]func() // GUARDED_BY(mu)
-	lastHandle sdiscovery.ServiceHandle            // GUARDED_BY(mu)
-}
-
-func (s *impl) RegisterService(ctx *context.T, call rpc.ServerCall, service discovery.Service, visibility []security.BlessingPattern) (sdiscovery.ServiceHandle, string, error) {
-	ctx, cancel := context.WithCancel(s.ctx)
-	done, err := s.d.Advertise(ctx, &service, visibility)
-	if err != nil {
-		cancel()
-		return 0, "", err
-	}
-
-	s.mu.Lock()
-	if len(s.handles) >= maxActiveHandles {
-		s.mu.Unlock()
-		cancel()
-		return 0, "", verror.New(errTooManyServices, ctx)
-	}
-	handle := s.lastHandle + 1
-	for {
-		if handle == 0 { // Avoid zero handle.
-			handle++
-		}
-		if _, exist := s.handles[handle]; !exist {
-			break
-		}
-	}
-	s.handles[handle] = func() {
-		cancel()
-		<-done
-	}
-	s.lastHandle = handle
-	s.mu.Unlock()
-	return handle, service.InstanceId, nil
-}
-
-func (s *impl) UnregisterService(ctx *context.T, call rpc.ServerCall, handle sdiscovery.ServiceHandle) error {
-	s.mu.Lock()
-	stop := s.handles[handle]
-	delete(s.handles, handle)
-	s.mu.Unlock()
-	if stop != nil {
-		stop()
-	}
-	return nil
-}
-
-func (s *impl) Scan(ctx *context.T, call sdiscovery.ScannerScanServerCall, query string) error {
-	updateCh, err := s.d.Scan(ctx, query)
-	if err != nil {
-		return err
-	}
-
-	stream := call.SendStream()
-	for update := range updateCh {
-		if err = stream.Send(update); err != nil {
-			return err
-		}
-	}
-	return nil
-}
-
-// NewDiscoveryService returns a new Discovery service implementation.
-func NewDiscoveryService(ctx *context.T) sdiscovery.DiscoveryServerMethods {
-	return &impl{
-		ctx:     ctx,
-		d:       v23.GetDiscovery(ctx),
-		handles: make(map[sdiscovery.ServiceHandle]func()),
-	}
-}
diff --git a/services/discovery/service_test.go b/services/discovery/service_test.go
deleted file mode 100644
index f0f6123..0000000
--- a/services/discovery/service_test.go
+++ /dev/null
@@ -1,120 +0,0 @@
-// Copyright 2015 The Vanadium Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style
-// license that can be found in the LICENSE file.
-
-package discovery
-
-import (
-	"fmt"
-	"reflect"
-	"testing"
-	"time"
-
-	"v.io/v23"
-	"v.io/v23/context"
-	"v.io/v23/discovery"
-	sdiscovery "v.io/v23/services/discovery"
-
-	idiscovery "v.io/x/ref/lib/discovery"
-	fdiscovery "v.io/x/ref/lib/discovery/factory"
-	"v.io/x/ref/lib/discovery/plugins/mock"
-	_ "v.io/x/ref/runtime/factories/generic"
-	"v.io/x/ref/test"
-)
-
-func TestBasic(t *testing.T) {
-	fdiscovery.InjectDiscovery(idiscovery.NewWithPlugins([]idiscovery.Plugin{mock.New()}))
-	ctx, shutdown := test.V23Init()
-	defer shutdown()
-
-	ds := NewDiscoveryService(ctx)
-	ctx, server, err := v23.WithNewServer(ctx, "", sdiscovery.DiscoveryServer(ds), nil)
-	if err != nil {
-		t.Fatalf("NewServer() failed: %v", err)
-	}
-	defer server.Stop()
-	addr := server.Status().Endpoints[0].Name()
-
-	services := []discovery.Service{
-		{
-			InstanceId:    "123",
-			InterfaceName: "v.io/v23/a",
-			Attrs:         discovery.Attributes{"a1": "v1"},
-			Addrs:         []string{"/h1:123/x"},
-		},
-		{
-			InterfaceName: "v.io/v23/b",
-			Attrs:         discovery.Attributes{"b1": "v1"},
-			Addrs:         []string{"/h1:123/y"},
-		},
-	}
-
-	var handles []sdiscovery.ServiceHandle
-	advertiser := sdiscovery.AdvertiserClient(addr)
-	for i, service := range services {
-		handle, instanceId, err := advertiser.RegisterService(ctx, service, nil)
-		if err != nil {
-			t.Fatalf("RegisterService() failed: %v", err)
-		}
-		switch {
-		case len(service.InstanceId) == 0:
-			if len(instanceId) == 0 {
-				t.Errorf("test[%d]: got empty instance id", i)
-			}
-			services[i].InstanceId = instanceId
-		default:
-			if instanceId != service.InstanceId {
-				t.Errorf("test[%d]: got instance id %v, but wanted %v", i, instanceId, service.InstanceId)
-			}
-		}
-		handles = append(handles, handle)
-	}
-
-	scanner := sdiscovery.ScannerClient(addr)
-	if err := scanAndMatch(ctx, scanner, "", services...); err != nil {
-		t.Error(err)
-	}
-
-	if err := advertiser.UnregisterService(ctx, handles[0]); err != nil {
-		t.Fatalf("UnregisterService() failed: %v", err)
-	}
-	if err := scanAndMatch(ctx, scanner, "", services[1]); err != nil {
-		t.Error(err)
-	}
-}
-
-func scanAndMatch(ctx *context.T, scanner sdiscovery.ScannerClientStub, query string, wants ...discovery.Service) error {
-	ctx, cancel := context.WithCancel(ctx)
-	defer cancel()
-
-	stream, err := scanner.Scan(ctx, query)
-	if err != nil {
-		return err
-	}
-
-	recv := stream.RecvStream()
-	for len(wants) > 0 {
-		if !recv.Advance() {
-			return recv.Err()
-		}
-		found := recv.Value().Interface().(discovery.Found)
-		matched := false
-		for i, want := range wants {
-			if reflect.DeepEqual(found.Service, want) {
-				wants = append(wants[:i], wants[i+1:]...)
-				matched = true
-				break
-			}
-		}
-		if !matched {
-			return fmt.Errorf("unexpected service found: %v", found.Service)
-		}
-	}
-
-	// Make sure there is no more update.
-	time.AfterFunc(5*time.Millisecond, cancel)
-	if recv.Advance() {
-		return fmt.Errorf("unexpected update: %v", recv.Value())
-	}
-	return nil
-}
diff --git a/services/syncbase/vsync/sync.go b/services/syncbase/vsync/sync.go
index bb5b519..3d65e9b 100644
--- a/services/syncbase/vsync/sync.go
+++ b/services/syncbase/vsync/sync.go
@@ -89,6 +89,7 @@
 	// access the info using the Syncbase names as keys.  The syncgroups
 	// map is a secondary index to access the info using the syncgroup names
 	// as keys of the outer-map, and instance IDs as keys of the inner-map.
+	discovery           discovery.T
 	discoveryIds        map[string]*discovery.Service
 	discoveryPeers      map[string]*discovery.Service
 	discoverySyncgroups map[string]map[string]*discovery.Service
@@ -163,12 +164,17 @@
 // that the syncer can pick from. In addition, the sync module responds to
 // incoming RPCs from remote sync modules and local clients.
 func New(ctx *context.T, sv interfaces.Service, blobStEngine, blobRootDir string, cl *vclock.VClock, publishInNH bool) (*syncService, error) {
+	discovery, err := v23.NewDiscovery(ctx)
+	if err != nil {
+		return nil, err
+	}
 	s := &syncService{
 		sv:                  sv,
 		batches:             make(batchSet),
 		sgPublishQueue:      list.New(),
 		vclock:              cl,
 		ctx:                 ctx,
+		discovery:           discovery,
 		publishInNH:         publishInNH,
 		cancelAdvSyncgroups: make(map[string]context.CancelFunc),
 	}
@@ -201,7 +207,6 @@
 	}
 
 	// Open a blob store.
-	var err error
 	s.bst, err = fsblob.Create(ctx, blobStEngine, path.Join(blobRootDir, "blobs"))
 	if err != nil {
 		return nil, err
@@ -262,13 +267,8 @@
 func (s *syncService) discoverNeighborhood(ctx *context.T) {
 	defer s.pending.Done()
 
-	scanner := v23.GetDiscovery(ctx)
-	if scanner == nil {
-		vlog.Fatal("sync: discoverNeighborhood: discovery service not initialized")
-	}
-
 	query := `v.InterfaceName="` + ifName + `"`
-	ch, err := scanner.Scan(ctx, query)
+	ch, err := s.discovery.Scan(ctx, query)
 	if err != nil {
 		vlog.Errorf("sync: discoverNeighborhood: cannot start discovery service: %v", err)
 		return
@@ -503,7 +503,7 @@
 	ctx, stop := context.WithCancel(s.ctx)
 
 	// Note that duplicate calls to advertise will return an error.
-	_, err := idiscovery.AdvertiseServer(ctx, s.svr, "", &sbService, nil)
+	_, err := idiscovery.AdvertiseServer(ctx, s.discovery, s.svr, "", &sbService, nil)
 
 	if err == nil {
 		vlog.VI(4).Infof("sync: advertiseSyncbaseInNeighborhood: successful")
@@ -550,8 +550,7 @@
 	vlog.VI(4).Infof("sync: advertiseSyncgroupInNeighborhood: advertising %v", sbService)
 
 	// Note that duplicate calls to advertise will return an error.
-	_, err := idiscovery.AdvertiseServer(ctx, s.svr, "", &sbService, nil)
-
+	_, err := idiscovery.AdvertiseServer(ctx, s.discovery, s.svr, "", &sbService, nil)
 	if err == nil {
 		vlog.VI(4).Infof("sync: advertiseSyncgroupInNeighborhood: successful")
 		s.cancelAdvSyncgroups[sg.Name] = stop