Преглед изворни кода

integration: move watch tests to v3_watch_test.go

Xiang Li пре 9 година
родитељ
комит
5f1d30b76a
2 измењених фајлова са 715 додато и 690 уклоњено
  1. 0 690
      integration/v3_grpc_test.go
  2. 715 0
      integration/v3_watch_test.go

+ 0 - 690
integration/v3_grpc_test.go

@@ -14,11 +14,8 @@
 package integration
 
 import (
-	"bytes"
 	"fmt"
 	"reflect"
-	"sort"
-	"sync"
 	"testing"
 	"time"
 
@@ -27,7 +24,6 @@ import (
 	"github.com/coreos/etcd/etcdserver/api/v3rpc"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/pkg/testutil"
-	"github.com/coreos/etcd/storage/storagepb"
 )
 
 // TestV3PutOverwrite puts a key with the v3 api to a random cluster member,
@@ -453,692 +449,6 @@ func TestV3Hash(t *testing.T) {
 	}
 }
 
-// TestV3WatchFromCurrentRevision tests Watch APIs from current revision.
-func TestV3WatchFromCurrentRevision(t *testing.T) {
-	defer testutil.AfterTest(t)
-	tests := []struct {
-		putKeys      []string
-		watchRequest *pb.WatchRequest
-
-		wresps []*pb.WatchResponse
-	}{
-		// watch the key, matching
-		{
-			[]string{"foo"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Key: []byte("foo")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 2},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-						},
-					},
-				},
-			},
-		},
-		// watch the key, non-matching
-		{
-			[]string{"foo"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Key: []byte("helloworld")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-			},
-		},
-		// watch the prefix, matching
-		{
-			[]string{"fooLong"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Prefix: []byte("foo")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 2},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("fooLong"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-						},
-					},
-				},
-			},
-		},
-		// watch the prefix, non-matching
-		{
-			[]string{"foo"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Prefix: []byte("helloworld")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-			},
-		},
-		// multiple puts, one watcher with matching key
-		{
-			[]string{"foo", "foo", "foo"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Key: []byte("foo")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 2},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-						},
-					},
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 3},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 3, Version: 2},
-						},
-					},
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 4},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 3},
-						},
-					},
-				},
-			},
-		},
-		// multiple puts, one watcher with matching prefix
-		{
-			[]string{"foo", "foo", "foo"},
-			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Prefix: []byte("foo")}}},
-
-			[]*pb.WatchResponse{
-				{
-					Header:  &pb.ResponseHeader{Revision: 1},
-					Created: true,
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 2},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-						},
-					},
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 3},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 3, Version: 2},
-						},
-					},
-				},
-				{
-					Header:  &pb.ResponseHeader{Revision: 4},
-					Created: false,
-					Events: []*storagepb.Event{
-						{
-							Type: storagepb.PUT,
-							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 3},
-						},
-					},
-				},
-			},
-		},
-	}
-
-	for i, tt := range tests {
-		clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-
-		wAPI := clus.RandClient().Watch
-		ctx, cancel := context.WithCancel(context.Background())
-		defer cancel()
-		wStream, err := wAPI.Watch(ctx)
-		if err != nil {
-			t.Fatalf("#%d: wAPI.Watch error: %v", i, err)
-		}
-
-		if err := wStream.Send(tt.watchRequest); err != nil {
-			t.Fatalf("#%d: wStream.Send error: %v", i, err)
-		}
-
-		go func() {
-			for _, k := range tt.putKeys {
-				kvc := clus.RandClient().KV
-				req := &pb.PutRequest{Key: []byte(k), Value: []byte("bar")}
-				if _, err := kvc.Put(context.TODO(), req); err != nil {
-					t.Fatalf("#%d: couldn't put key (%v)", i, err)
-				}
-			}
-		}()
-
-		var createdWatchId int64
-		for j, wresp := range tt.wresps {
-			resp, err := wStream.Recv()
-			if err != nil {
-				t.Errorf("#%d.%d: wStream.Recv error: %v", i, j, err)
-			}
-
-			if resp.Header == nil {
-				t.Fatalf("#%d.%d: unexpected nil resp.Header", i, j)
-			}
-			if resp.Header.Revision != wresp.Header.Revision {
-				t.Errorf("#%d.%d: resp.Header.Revision got = %d, want = %d", i, j, resp.Header.Revision, wresp.Header.Revision)
-			}
-
-			if wresp.Created != resp.Created {
-				t.Errorf("#%d.%d: resp.Created got = %v, want = %v", i, j, resp.Created, wresp.Created)
-			}
-			if resp.Created {
-				createdWatchId = resp.WatchId
-			}
-			if resp.WatchId != createdWatchId {
-				t.Errorf("#%d.%d: resp.WatchId got = %d, want = %d", i, j, resp.WatchId, createdWatchId)
-			}
-
-			if !reflect.DeepEqual(resp.Events, wresp.Events) {
-				t.Errorf("#%d.%d: resp.Events got = %+v, want = %+v", i, j, resp.Events, wresp.Events)
-			}
-		}
-
-		rok, nr := WaitResponse(wStream, 1*time.Second)
-		if !rok {
-			t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-		}
-
-		// can't defer because tcp ports will be in use
-		clus.Terminate(t)
-	}
-}
-
-// TestV3WatchCancelSynced tests Watch APIs cancellation from synced map.
-func TestV3WatchCancelSynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchCancel(t, 0)
-}
-
-// TestV3WatchCancelUnsynced tests Watch APIs cancellation from unsynced map.
-func TestV3WatchCancelUnsynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchCancel(t, 1)
-}
-
-func testV3WatchCancel(t *testing.T, startRev int64) {
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-
-	ctx, cancel := context.WithCancel(context.Background())
-	defer cancel()
-	wStream, errW := clus.RandClient().Watch.Watch(ctx)
-	if errW != nil {
-		t.Fatalf("wAPI.Watch error: %v", errW)
-	}
-
-	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-		CreateRequest: &pb.WatchCreateRequest{
-			Key: []byte("foo"), StartRevision: startRev}}}
-	if err := wStream.Send(wreq); err != nil {
-		t.Fatalf("wStream.Send error: %v", err)
-	}
-
-	wresp, errR := wStream.Recv()
-	if errR != nil {
-		t.Errorf("wStream.Recv error: %v", errR)
-	}
-	if !wresp.Created {
-		t.Errorf("wresp.Created got = %v, want = true", wresp.Created)
-	}
-
-	creq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CancelRequest{
-		CancelRequest: &pb.WatchCancelRequest{
-			WatchId: wresp.WatchId}}}
-	if err := wStream.Send(creq); err != nil {
-		t.Fatalf("wStream.Send error: %v", err)
-	}
-
-	cresp, err := wStream.Recv()
-	if err != nil {
-		t.Errorf("wStream.Recv error: %v", err)
-	}
-	if !cresp.Canceled {
-		t.Errorf("cresp.Canceled got = %v, want = true", cresp.Canceled)
-	}
-
-	kvc := clus.RandClient().KV
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
-		t.Errorf("couldn't put key (%v)", err)
-	}
-
-	// watch got canceled, so this should block
-	rok, nr := WaitResponse(wStream, 1*time.Second)
-	if !rok {
-		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-	}
-
-	clus.Terminate(t)
-}
-
-func TestV3WatchMultipleWatchersSynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleWatchers(t, 0)
-}
-
-func TestV3WatchMultipleWatchersUnsynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleWatchers(t, 1)
-}
-
-// testV3WatchMultipleWatchers tests multiple watchers on the same key
-// and one watcher with matching prefix. It first puts the key
-// that matches all watchers, and another key that matches only
-// one watcher to test if it receives expected events.
-func testV3WatchMultipleWatchers(t *testing.T, startRev int64) {
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-	kvc := clus.RandClient().KV
-
-	ctx, cancel := context.WithCancel(context.Background())
-	defer cancel()
-	wStream, errW := clus.RandClient().Watch.Watch(ctx)
-	if errW != nil {
-		t.Fatalf("wAPI.Watch error: %v", errW)
-	}
-
-	watchKeyN := 4
-	for i := 0; i < watchKeyN+1; i++ {
-		var wreq *pb.WatchRequest
-		if i < watchKeyN {
-			wreq = &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Key: []byte("foo"), StartRevision: startRev}}}
-		} else {
-			wreq = &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-				CreateRequest: &pb.WatchCreateRequest{
-					Prefix: []byte("fo"), StartRevision: startRev}}}
-		}
-		if err := wStream.Send(wreq); err != nil {
-			t.Fatalf("wStream.Send error: %v", err)
-		}
-	}
-
-	ids := make(map[int64]struct{})
-	for i := 0; i < watchKeyN+1; i++ {
-		wresp, err := wStream.Recv()
-		if err != nil {
-			t.Fatalf("wStream.Recv error: %v", err)
-		}
-		if !wresp.Created {
-			t.Fatalf("wresp.Created got = %v, want = true", wresp.Created)
-		}
-		ids[wresp.WatchId] = struct{}{}
-	}
-
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-
-	for i := 0; i < watchKeyN+1; i++ {
-		wresp, err := wStream.Recv()
-		if err != nil {
-			t.Fatalf("wStream.Recv error: %v", err)
-		}
-		if _, ok := ids[wresp.WatchId]; !ok {
-			t.Errorf("watchId %d is not created!", wresp.WatchId)
-		} else {
-			delete(ids, wresp.WatchId)
-		}
-		if len(wresp.Events) == 0 {
-			t.Errorf("#%d: no events received", i)
-		}
-		for _, ev := range wresp.Events {
-			if string(ev.Kv.Key) != "foo" {
-				t.Errorf("ev.Kv.Key got = %s, want = foo", ev.Kv.Key)
-			}
-			if string(ev.Kv.Value) != "bar" {
-				t.Errorf("ev.Kv.Value got = %s, want = bar", ev.Kv.Value)
-			}
-		}
-	}
-
-	// now put one key that has only one matching watcher
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("fo"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-	wresp, err := wStream.Recv()
-	if err != nil {
-		t.Errorf("wStream.Recv error: %v", err)
-	}
-	if len(wresp.Events) != 1 {
-		t.Fatalf("len(wresp.Events) got = %d, want = 1", len(wresp.Events))
-	}
-	if string(wresp.Events[0].Kv.Key) != "fo" {
-		t.Errorf("wresp.Events[0].Kv.Key got = %s, want = fo", wresp.Events[0].Kv.Key)
-	}
-
-	// now Recv should block because there is no more events coming
-	rok, nr := WaitResponse(wStream, 1*time.Second)
-	if !rok {
-		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-	}
-
-	clus.Terminate(t)
-}
-
-func TestV3WatchMultipleEventsTxnSynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleEventsTxn(t, 0)
-}
-
-func TestV3WatchMultipleEventsTxnUnsynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleEventsTxn(t, 1)
-}
-
-// testV3WatchMultipleEventsTxn tests Watch APIs when it receives multiple events.
-func testV3WatchMultipleEventsTxn(t *testing.T, startRev int64) {
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-
-	ctx, cancel := context.WithCancel(context.Background())
-	defer cancel()
-	wStream, wErr := clus.RandClient().Watch.Watch(ctx)
-	if wErr != nil {
-		t.Fatalf("wAPI.Watch error: %v", wErr)
-	}
-
-	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-		CreateRequest: &pb.WatchCreateRequest{
-			Prefix: []byte("foo"), StartRevision: startRev}}}
-	if err := wStream.Send(wreq); err != nil {
-		t.Fatalf("wStream.Send error: %v", err)
-	}
-
-	kvc := clus.RandClient().KV
-	txn := pb.TxnRequest{}
-	for i := 0; i < 3; i++ {
-		ru := &pb.RequestUnion{}
-		ru.Request = &pb.RequestUnion_RequestPut{
-			RequestPut: &pb.PutRequest{
-				Key: []byte(fmt.Sprintf("foo%d", i)), Value: []byte("bar")}}
-		txn.Success = append(txn.Success, ru)
-	}
-
-	tresp, err := kvc.Txn(context.Background(), &txn)
-	if err != nil {
-		t.Fatalf("kvc.Txn error: %v", err)
-	}
-	if !tresp.Succeeded {
-		t.Fatalf("kvc.Txn failed: %+v", tresp)
-	}
-
-	events := []*storagepb.Event{}
-	for len(events) < 3 {
-		resp, err := wStream.Recv()
-		if err != nil {
-			t.Errorf("wStream.Recv error: %v", err)
-		}
-		if resp.Created {
-			continue
-		}
-		events = append(events, resp.Events...)
-	}
-	sort.Sort(eventsSortByKey(events))
-
-	wevents := []*storagepb.Event{
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-		},
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-		},
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo2"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-		},
-	}
-
-	if !reflect.DeepEqual(events, wevents) {
-		t.Errorf("events got = %+v, want = %+v", events, wevents)
-	}
-
-	rok, nr := WaitResponse(wStream, 1*time.Second)
-	if !rok {
-		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-	}
-
-	// can't defer because tcp ports will be in use
-	clus.Terminate(t)
-}
-
-type eventsSortByKey []*storagepb.Event
-
-func (evs eventsSortByKey) Len() int           { return len(evs) }
-func (evs eventsSortByKey) Swap(i, j int)      { evs[i], evs[j] = evs[j], evs[i] }
-func (evs eventsSortByKey) Less(i, j int) bool { return bytes.Compare(evs[i].Kv.Key, evs[j].Kv.Key) < 0 }
-
-func TestV3WatchMultipleEventsPutUnsynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-	defer clus.Terminate(t)
-
-	kvc := clus.RandClient().KV
-
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo0"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo1"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-
-	ctx, cancel := context.WithCancel(context.Background())
-	defer cancel()
-	wStream, wErr := clus.RandClient().Watch.Watch(ctx)
-	if wErr != nil {
-		t.Fatalf("wAPI.Watch error: %v", wErr)
-	}
-
-	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-		CreateRequest: &pb.WatchCreateRequest{
-			Prefix: []byte("foo"), StartRevision: 1}}}
-	if err := wStream.Send(wreq); err != nil {
-		t.Fatalf("wStream.Send error: %v", err)
-	}
-
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo0"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo1"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-
-	allWevents := []*storagepb.Event{
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-		},
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 3, ModRevision: 3, Version: 1},
-		},
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 2},
-		},
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 3, ModRevision: 5, Version: 2},
-		},
-	}
-
-	events := []*storagepb.Event{}
-	for len(events) < 4 {
-		resp, err := wStream.Recv()
-		if err != nil {
-			t.Errorf("wStream.Recv error: %v", err)
-		}
-		if resp.Created {
-			continue
-		}
-		events = append(events, resp.Events...)
-		// if PUT requests are committed by now, first receive would return
-		// multiple events, but if not, it returns a single event. In SSD,
-		// it should return 4 events at once.
-	}
-
-	if !reflect.DeepEqual(events, allWevents) {
-		t.Errorf("events got = %+v, want = %+v", events, allWevents)
-	}
-
-	rok, nr := WaitResponse(wStream, 1*time.Second)
-	if !rok {
-		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-	}
-}
-
-func TestV3WatchMultipleStreamsSynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleStreams(t, 0)
-}
-
-func TestV3WatchMultipleStreamsUnsynced(t *testing.T) {
-	defer testutil.AfterTest(t)
-	testV3WatchMultipleStreams(t, 1)
-}
-
-// testV3WatchMultipleStreams tests multiple watchers on the same key on multiple streams.
-func testV3WatchMultipleStreams(t *testing.T, startRev int64) {
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-	wAPI := clus.RandClient().Watch
-	kvc := clus.RandClient().KV
-
-	streams := make([]pb.Watch_WatchClient, 5)
-	for i := range streams {
-		ctx, cancel := context.WithCancel(context.Background())
-		defer cancel()
-		wStream, errW := wAPI.Watch(ctx)
-		if errW != nil {
-			t.Fatalf("wAPI.Watch error: %v", errW)
-		}
-		wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
-			CreateRequest: &pb.WatchCreateRequest{
-				Key: []byte("foo"), StartRevision: startRev}}}
-		if err := wStream.Send(wreq); err != nil {
-			t.Fatalf("wStream.Send error: %v", err)
-		}
-		streams[i] = wStream
-	}
-
-	for _, wStream := range streams {
-		wresp, err := wStream.Recv()
-		if err != nil {
-			t.Fatalf("wStream.Recv error: %v", err)
-		}
-		if !wresp.Created {
-			t.Fatalf("wresp.Created got = %v, want = true", wresp.Created)
-		}
-	}
-
-	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
-		t.Fatalf("couldn't put key (%v)", err)
-	}
-
-	var wg sync.WaitGroup
-	wg.Add(len(streams))
-	wevents := []*storagepb.Event{
-		{
-			Type: storagepb.PUT,
-			Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
-		},
-	}
-	for i := range streams {
-		go func(i int) {
-			defer wg.Done()
-			wStream := streams[i]
-			wresp, err := wStream.Recv()
-			if err != nil {
-				t.Fatalf("wStream.Recv error: %v", err)
-			}
-			if wresp.WatchId != 0 {
-				t.Errorf("watchId got = %d, want = 0", wresp.WatchId)
-			}
-			if !reflect.DeepEqual(wresp.Events, wevents) {
-				t.Errorf("wresp.Events got = %+v, want = %+v", wresp.Events, wevents)
-			}
-			// now Recv should block because there is no more events coming
-			rok, nr := WaitResponse(wStream, 1*time.Second)
-			if !rok {
-				t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
-			}
-		}(i)
-	}
-	wg.Wait()
-
-	clus.Terminate(t)
-}
-
-// WaitResponse waits on the given stream for given duration.
-// If there is no more events, true and a nil response will be
-// returned closing the WatchClient stream. Or the response will
-// be returned.
-func WaitResponse(wc pb.Watch_WatchClient, timeout time.Duration) (bool, *pb.WatchResponse) {
-	rCh := make(chan *pb.WatchResponse)
-	go func() {
-		resp, _ := wc.Recv()
-		rCh <- resp
-	}()
-	select {
-	case nr := <-rCh:
-		return false, nr
-	case <-time.After(timeout):
-	}
-	wc.CloseSend()
-	rv, ok := <-rCh
-	if rv != nil || !ok {
-		return false, rv
-	}
-	return true, nil
-}
-
 func TestV3RangeRequest(t *testing.T) {
 	defer testutil.AfterTest(t)
 	tests := []struct {

+ 715 - 0
integration/v3_watch_test.go

@@ -0,0 +1,715 @@
+// Copyright 2016 CoreOS, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.package recipe
+package integration
+
+import (
+	"bytes"
+	"fmt"
+	"reflect"
+	"sort"
+	"sync"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/pkg/testutil"
+	"github.com/coreos/etcd/storage/storagepb"
+)
+
+// TestV3WatchFromCurrentRevision tests Watch APIs from current revision.
+func TestV3WatchFromCurrentRevision(t *testing.T) {
+	defer testutil.AfterTest(t)
+	tests := []struct {
+		putKeys      []string
+		watchRequest *pb.WatchRequest
+
+		wresps []*pb.WatchResponse
+	}{
+		// watch the key, matching
+		{
+			[]string{"foo"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Key: []byte("foo")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 2},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+						},
+					},
+				},
+			},
+		},
+		// watch the key, non-matching
+		{
+			[]string{"foo"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Key: []byte("helloworld")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+			},
+		},
+		// watch the prefix, matching
+		{
+			[]string{"fooLong"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Prefix: []byte("foo")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 2},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("fooLong"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+						},
+					},
+				},
+			},
+		},
+		// watch the prefix, non-matching
+		{
+			[]string{"foo"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Prefix: []byte("helloworld")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+			},
+		},
+		// multiple puts, one watcher with matching key
+		{
+			[]string{"foo", "foo", "foo"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Key: []byte("foo")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 2},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+						},
+					},
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 3},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 3, Version: 2},
+						},
+					},
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 4},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 3},
+						},
+					},
+				},
+			},
+		},
+		// multiple puts, one watcher with matching prefix
+		{
+			[]string{"foo", "foo", "foo"},
+			&pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Prefix: []byte("foo")}}},
+
+			[]*pb.WatchResponse{
+				{
+					Header:  &pb.ResponseHeader{Revision: 1},
+					Created: true,
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 2},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+						},
+					},
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 3},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 3, Version: 2},
+						},
+					},
+				},
+				{
+					Header:  &pb.ResponseHeader{Revision: 4},
+					Created: false,
+					Events: []*storagepb.Event{
+						{
+							Type: storagepb.PUT,
+							Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 3},
+						},
+					},
+				},
+			},
+		},
+	}
+
+	for i, tt := range tests {
+		clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+
+		wAPI := clus.RandClient().Watch
+		ctx, cancel := context.WithCancel(context.Background())
+		defer cancel()
+		wStream, err := wAPI.Watch(ctx)
+		if err != nil {
+			t.Fatalf("#%d: wAPI.Watch error: %v", i, err)
+		}
+
+		if err := wStream.Send(tt.watchRequest); err != nil {
+			t.Fatalf("#%d: wStream.Send error: %v", i, err)
+		}
+
+		go func() {
+			for _, k := range tt.putKeys {
+				kvc := clus.RandClient().KV
+				req := &pb.PutRequest{Key: []byte(k), Value: []byte("bar")}
+				if _, err := kvc.Put(context.TODO(), req); err != nil {
+					t.Fatalf("#%d: couldn't put key (%v)", i, err)
+				}
+			}
+		}()
+
+		var createdWatchId int64
+		for j, wresp := range tt.wresps {
+			resp, err := wStream.Recv()
+			if err != nil {
+				t.Errorf("#%d.%d: wStream.Recv error: %v", i, j, err)
+			}
+
+			if resp.Header == nil {
+				t.Fatalf("#%d.%d: unexpected nil resp.Header", i, j)
+			}
+			if resp.Header.Revision != wresp.Header.Revision {
+				t.Errorf("#%d.%d: resp.Header.Revision got = %d, want = %d", i, j, resp.Header.Revision, wresp.Header.Revision)
+			}
+
+			if wresp.Created != resp.Created {
+				t.Errorf("#%d.%d: resp.Created got = %v, want = %v", i, j, resp.Created, wresp.Created)
+			}
+			if resp.Created {
+				createdWatchId = resp.WatchId
+			}
+			if resp.WatchId != createdWatchId {
+				t.Errorf("#%d.%d: resp.WatchId got = %d, want = %d", i, j, resp.WatchId, createdWatchId)
+			}
+
+			if !reflect.DeepEqual(resp.Events, wresp.Events) {
+				t.Errorf("#%d.%d: resp.Events got = %+v, want = %+v", i, j, resp.Events, wresp.Events)
+			}
+		}
+
+		rok, nr := WaitResponse(wStream, 1*time.Second)
+		if !rok {
+			t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+		}
+
+		// can't defer because tcp ports will be in use
+		clus.Terminate(t)
+	}
+}
+
+// TestV3WatchCancelSynced tests Watch APIs cancellation from synced map.
+func TestV3WatchCancelSynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchCancel(t, 0)
+}
+
+// TestV3WatchCancelUnsynced tests Watch APIs cancellation from unsynced map.
+func TestV3WatchCancelUnsynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchCancel(t, 1)
+}
+
+func testV3WatchCancel(t *testing.T, startRev int64) {
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+	wStream, errW := clus.RandClient().Watch.Watch(ctx)
+	if errW != nil {
+		t.Fatalf("wAPI.Watch error: %v", errW)
+	}
+
+	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+		CreateRequest: &pb.WatchCreateRequest{
+			Key: []byte("foo"), StartRevision: startRev}}}
+	if err := wStream.Send(wreq); err != nil {
+		t.Fatalf("wStream.Send error: %v", err)
+	}
+
+	wresp, errR := wStream.Recv()
+	if errR != nil {
+		t.Errorf("wStream.Recv error: %v", errR)
+	}
+	if !wresp.Created {
+		t.Errorf("wresp.Created got = %v, want = true", wresp.Created)
+	}
+
+	creq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CancelRequest{
+		CancelRequest: &pb.WatchCancelRequest{
+			WatchId: wresp.WatchId}}}
+	if err := wStream.Send(creq); err != nil {
+		t.Fatalf("wStream.Send error: %v", err)
+	}
+
+	cresp, err := wStream.Recv()
+	if err != nil {
+		t.Errorf("wStream.Recv error: %v", err)
+	}
+	if !cresp.Canceled {
+		t.Errorf("cresp.Canceled got = %v, want = true", cresp.Canceled)
+	}
+
+	kvc := clus.RandClient().KV
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
+		t.Errorf("couldn't put key (%v)", err)
+	}
+
+	// watch got canceled, so this should block
+	rok, nr := WaitResponse(wStream, 1*time.Second)
+	if !rok {
+		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+	}
+
+	clus.Terminate(t)
+}
+
+func TestV3WatchMultipleWatchersSynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleWatchers(t, 0)
+}
+
+func TestV3WatchMultipleWatchersUnsynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleWatchers(t, 1)
+}
+
+// testV3WatchMultipleWatchers tests multiple watchers on the same key
+// and one watcher with matching prefix. It first puts the key
+// that matches all watchers, and another key that matches only
+// one watcher to test if it receives expected events.
+func testV3WatchMultipleWatchers(t *testing.T, startRev int64) {
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+	kvc := clus.RandClient().KV
+
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+	wStream, errW := clus.RandClient().Watch.Watch(ctx)
+	if errW != nil {
+		t.Fatalf("wAPI.Watch error: %v", errW)
+	}
+
+	watchKeyN := 4
+	for i := 0; i < watchKeyN+1; i++ {
+		var wreq *pb.WatchRequest
+		if i < watchKeyN {
+			wreq = &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Key: []byte("foo"), StartRevision: startRev}}}
+		} else {
+			wreq = &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+				CreateRequest: &pb.WatchCreateRequest{
+					Prefix: []byte("fo"), StartRevision: startRev}}}
+		}
+		if err := wStream.Send(wreq); err != nil {
+			t.Fatalf("wStream.Send error: %v", err)
+		}
+	}
+
+	ids := make(map[int64]struct{})
+	for i := 0; i < watchKeyN+1; i++ {
+		wresp, err := wStream.Recv()
+		if err != nil {
+			t.Fatalf("wStream.Recv error: %v", err)
+		}
+		if !wresp.Created {
+			t.Fatalf("wresp.Created got = %v, want = true", wresp.Created)
+		}
+		ids[wresp.WatchId] = struct{}{}
+	}
+
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+
+	for i := 0; i < watchKeyN+1; i++ {
+		wresp, err := wStream.Recv()
+		if err != nil {
+			t.Fatalf("wStream.Recv error: %v", err)
+		}
+		if _, ok := ids[wresp.WatchId]; !ok {
+			t.Errorf("watchId %d is not created!", wresp.WatchId)
+		} else {
+			delete(ids, wresp.WatchId)
+		}
+		if len(wresp.Events) == 0 {
+			t.Errorf("#%d: no events received", i)
+		}
+		for _, ev := range wresp.Events {
+			if string(ev.Kv.Key) != "foo" {
+				t.Errorf("ev.Kv.Key got = %s, want = foo", ev.Kv.Key)
+			}
+			if string(ev.Kv.Value) != "bar" {
+				t.Errorf("ev.Kv.Value got = %s, want = bar", ev.Kv.Value)
+			}
+		}
+	}
+
+	// now put one key that has only one matching watcher
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("fo"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+	wresp, err := wStream.Recv()
+	if err != nil {
+		t.Errorf("wStream.Recv error: %v", err)
+	}
+	if len(wresp.Events) != 1 {
+		t.Fatalf("len(wresp.Events) got = %d, want = 1", len(wresp.Events))
+	}
+	if string(wresp.Events[0].Kv.Key) != "fo" {
+		t.Errorf("wresp.Events[0].Kv.Key got = %s, want = fo", wresp.Events[0].Kv.Key)
+	}
+
+	// now Recv should block because there is no more events coming
+	rok, nr := WaitResponse(wStream, 1*time.Second)
+	if !rok {
+		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+	}
+
+	clus.Terminate(t)
+}
+
+func TestV3WatchMultipleEventsTxnSynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleEventsTxn(t, 0)
+}
+
+func TestV3WatchMultipleEventsTxnUnsynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleEventsTxn(t, 1)
+}
+
+// testV3WatchMultipleEventsTxn tests Watch APIs when it receives multiple events.
+func testV3WatchMultipleEventsTxn(t *testing.T, startRev int64) {
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+	wStream, wErr := clus.RandClient().Watch.Watch(ctx)
+	if wErr != nil {
+		t.Fatalf("wAPI.Watch error: %v", wErr)
+	}
+
+	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+		CreateRequest: &pb.WatchCreateRequest{
+			Prefix: []byte("foo"), StartRevision: startRev}}}
+	if err := wStream.Send(wreq); err != nil {
+		t.Fatalf("wStream.Send error: %v", err)
+	}
+
+	kvc := clus.RandClient().KV
+	txn := pb.TxnRequest{}
+	for i := 0; i < 3; i++ {
+		ru := &pb.RequestUnion{}
+		ru.Request = &pb.RequestUnion_RequestPut{
+			RequestPut: &pb.PutRequest{
+				Key: []byte(fmt.Sprintf("foo%d", i)), Value: []byte("bar")}}
+		txn.Success = append(txn.Success, ru)
+	}
+
+	tresp, err := kvc.Txn(context.Background(), &txn)
+	if err != nil {
+		t.Fatalf("kvc.Txn error: %v", err)
+	}
+	if !tresp.Succeeded {
+		t.Fatalf("kvc.Txn failed: %+v", tresp)
+	}
+
+	events := []*storagepb.Event{}
+	for len(events) < 3 {
+		resp, err := wStream.Recv()
+		if err != nil {
+			t.Errorf("wStream.Recv error: %v", err)
+		}
+		if resp.Created {
+			continue
+		}
+		events = append(events, resp.Events...)
+	}
+	sort.Sort(eventsSortByKey(events))
+
+	wevents := []*storagepb.Event{
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+		},
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+		},
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo2"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+		},
+	}
+
+	if !reflect.DeepEqual(events, wevents) {
+		t.Errorf("events got = %+v, want = %+v", events, wevents)
+	}
+
+	rok, nr := WaitResponse(wStream, 1*time.Second)
+	if !rok {
+		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+	}
+
+	// can't defer because tcp ports will be in use
+	clus.Terminate(t)
+}
+
+type eventsSortByKey []*storagepb.Event
+
+func (evs eventsSortByKey) Len() int           { return len(evs) }
+func (evs eventsSortByKey) Swap(i, j int)      { evs[i], evs[j] = evs[j], evs[i] }
+func (evs eventsSortByKey) Less(i, j int) bool { return bytes.Compare(evs[i].Kv.Key, evs[j].Kv.Key) < 0 }
+
+func TestV3WatchMultipleEventsPutUnsynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+	defer clus.Terminate(t)
+
+	kvc := clus.RandClient().KV
+
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo0"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo1"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+
+	ctx, cancel := context.WithCancel(context.Background())
+	defer cancel()
+	wStream, wErr := clus.RandClient().Watch.Watch(ctx)
+	if wErr != nil {
+		t.Fatalf("wAPI.Watch error: %v", wErr)
+	}
+
+	wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+		CreateRequest: &pb.WatchCreateRequest{
+			Prefix: []byte("foo"), StartRevision: 1}}}
+	if err := wStream.Send(wreq); err != nil {
+		t.Fatalf("wStream.Send error: %v", err)
+	}
+
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo0"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo1"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+
+	allWevents := []*storagepb.Event{
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+		},
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 3, ModRevision: 3, Version: 1},
+		},
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo0"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 4, Version: 2},
+		},
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo1"), Value: []byte("bar"), CreateRevision: 3, ModRevision: 5, Version: 2},
+		},
+	}
+
+	events := []*storagepb.Event{}
+	for len(events) < 4 {
+		resp, err := wStream.Recv()
+		if err != nil {
+			t.Errorf("wStream.Recv error: %v", err)
+		}
+		if resp.Created {
+			continue
+		}
+		events = append(events, resp.Events...)
+		// if PUT requests are committed by now, first receive would return
+		// multiple events, but if not, it returns a single event. In SSD,
+		// it should return 4 events at once.
+	}
+
+	if !reflect.DeepEqual(events, allWevents) {
+		t.Errorf("events got = %+v, want = %+v", events, allWevents)
+	}
+
+	rok, nr := WaitResponse(wStream, 1*time.Second)
+	if !rok {
+		t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+	}
+}
+
+func TestV3WatchMultipleStreamsSynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleStreams(t, 0)
+}
+
+func TestV3WatchMultipleStreamsUnsynced(t *testing.T) {
+	defer testutil.AfterTest(t)
+	testV3WatchMultipleStreams(t, 1)
+}
+
+// testV3WatchMultipleStreams tests multiple watchers on the same key on multiple streams.
+func testV3WatchMultipleStreams(t *testing.T, startRev int64) {
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+	wAPI := clus.RandClient().Watch
+	kvc := clus.RandClient().KV
+
+	streams := make([]pb.Watch_WatchClient, 5)
+	for i := range streams {
+		ctx, cancel := context.WithCancel(context.Background())
+		defer cancel()
+		wStream, errW := wAPI.Watch(ctx)
+		if errW != nil {
+			t.Fatalf("wAPI.Watch error: %v", errW)
+		}
+		wreq := &pb.WatchRequest{RequestUnion: &pb.WatchRequest_CreateRequest{
+			CreateRequest: &pb.WatchCreateRequest{
+				Key: []byte("foo"), StartRevision: startRev}}}
+		if err := wStream.Send(wreq); err != nil {
+			t.Fatalf("wStream.Send error: %v", err)
+		}
+		streams[i] = wStream
+	}
+
+	for _, wStream := range streams {
+		wresp, err := wStream.Recv()
+		if err != nil {
+			t.Fatalf("wStream.Recv error: %v", err)
+		}
+		if !wresp.Created {
+			t.Fatalf("wresp.Created got = %v, want = true", wresp.Created)
+		}
+	}
+
+	if _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}); err != nil {
+		t.Fatalf("couldn't put key (%v)", err)
+	}
+
+	var wg sync.WaitGroup
+	wg.Add(len(streams))
+	wevents := []*storagepb.Event{
+		{
+			Type: storagepb.PUT,
+			Kv:   &storagepb.KeyValue{Key: []byte("foo"), Value: []byte("bar"), CreateRevision: 2, ModRevision: 2, Version: 1},
+		},
+	}
+	for i := range streams {
+		go func(i int) {
+			defer wg.Done()
+			wStream := streams[i]
+			wresp, err := wStream.Recv()
+			if err != nil {
+				t.Fatalf("wStream.Recv error: %v", err)
+			}
+			if wresp.WatchId != 0 {
+				t.Errorf("watchId got = %d, want = 0", wresp.WatchId)
+			}
+			if !reflect.DeepEqual(wresp.Events, wevents) {
+				t.Errorf("wresp.Events got = %+v, want = %+v", wresp.Events, wevents)
+			}
+			// now Recv should block because there is no more events coming
+			rok, nr := WaitResponse(wStream, 1*time.Second)
+			if !rok {
+				t.Errorf("unexpected pb.WatchResponse is received %+v", nr)
+			}
+		}(i)
+	}
+	wg.Wait()
+
+	clus.Terminate(t)
+}
+
+// WaitResponse waits on the given stream for given duration.
+// If there is no more events, true and a nil response will be
+// returned closing the WatchClient stream. Or the response will
+// be returned.
+func WaitResponse(wc pb.Watch_WatchClient, timeout time.Duration) (bool, *pb.WatchResponse) {
+	rCh := make(chan *pb.WatchResponse)
+	go func() {
+		resp, _ := wc.Recv()
+		rCh <- resp
+	}()
+	select {
+	case nr := <-rCh:
+		return false, nr
+	case <-time.After(timeout):
+	}
+	wc.CloseSend()
+	rv, ok := <-rCh
+	if rv != nil || !ok {
+		return false, rv
+	}
+	return true, nil
+}