Browse Source

Merge pull request #1011 from jonboulle/clean_http

Validate HTTP API user input
Jonathan Boulle 11 years ago
parent
commit
c03798f99b
3 changed files with 447 additions and 133 deletions
  1. 6 0
      error/error.go
  2. 145 66
      etcdserver/etcdhttp/http.go
  3. 296 67
      etcdserver/etcdhttp/http_test.go

+ 6 - 0
error/error.go

@@ -46,6 +46,7 @@ var errors = map[int]string{
 	EcodeIndexOrValueRequired: "Index or value is required",
 	EcodeIndexValueMutex:      "Index and value cannot both be specified",
 	EcodeInvalidField:         "Invalid field",
+	EcodeInvalidForm:          "Invalid POST form",
 
 	// raft related errors
 	EcodeRaftInternal: "Raft Internal Error",
@@ -84,6 +85,7 @@ const (
 	EcodeIndexOrValueRequired = 207
 	EcodeIndexValueMutex      = 208
 	EcodeInvalidField         = 209
+	EcodeInvalidForm          = 210
 
 	EcodeRaftInternal = 300
 	EcodeLeaderElect  = 301
@@ -104,6 +106,10 @@ type Error struct {
 	Index     uint64 `json:"index"`
 }
 
+func NewRequestError(errorCode int, cause string) *Error {
+	return NewError(errorCode, cause, 0)
+}
+
 func NewError(errorCode int, cause string, index uint64) *Error {
 	return &Error{
 		ErrorCode: errorCode,

+ 145 - 66
etcdserver/etcdhttp/http.go

@@ -20,7 +20,7 @@ import (
 	"math/rand"
 
 	"github.com/coreos/etcd/elog"
-	etcderrors "github.com/coreos/etcd/error"
+	etcdErr "github.com/coreos/etcd/error"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/raft/raftpb"
@@ -192,27 +192,33 @@ func (h Handler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 func (h Handler) serveKeys(ctx context.Context, w http.ResponseWriter, r *http.Request) {
 	rr, err := parseRequest(r, genID())
 	if err != nil {
-		log.Println(err) // reading of body failed
+		writeError(w, err)
 		return
 	}
 
 	resp, err := h.Server.Do(ctx, rr)
-	switch e := err.(type) {
-	case nil:
-	case *etcderrors.Error:
-		// TODO: gross. this should be handled in encodeResponse
-		log.Println(err)
-		e.Write(w)
+	if err != nil {
+		writeError(w, err)
 		return
+	}
+
+	var ev *store.Event
+	switch {
+	case resp.Event != nil:
+		ev = resp.Event
+	case resp.Watcher != nil:
+		if ev, err = waitForEvent(ctx, w, resp.Watcher); err != nil {
+			http.Error(w, err.Error(), http.StatusGatewayTimeout)
+			return
+		}
 	default:
-		log.Println(err)
-		http.Error(w, "Internal Server Error", http.StatusInternalServerError)
+		writeError(w, errors.New("received response with no Event/Watcher!"))
 		return
 	}
 
-	if err := encodeResponse(ctx, w, resp); err != nil {
-		http.Error(w, "Timeout while waiting for response", http.StatusGatewayTimeout)
-		return
+	if err = writeEvent(w, ev); err != nil {
+		// Should never be reached
+		log.Println("error writing event: %v", err)
 	}
 }
 
@@ -256,39 +262,99 @@ func genID() int64 {
 	}
 }
 
+// parseRequest converts a received http.Request to a server Request,
+// performing validation of supplied fields as appropriate.
+// If any validation fails, an empty Request and non-nil error is returned.
 func parseRequest(r *http.Request, id int64) (etcdserverpb.Request, error) {
-	if err := r.ParseForm(); err != nil {
-		return etcdserverpb.Request{}, err
+	emptyReq := etcdserverpb.Request{}
+
+	err := r.ParseForm()
+	if err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeInvalidForm,
+			err.Error(),
+		)
 	}
+
 	if !strings.HasPrefix(r.URL.Path, keysPrefix) {
-		return etcdserverpb.Request{}, errors.New("unexpected key prefix!")
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeInvalidForm,
+			"incorrect key prefix",
+		)
+	}
+	p := r.URL.Path[len(keysPrefix):]
+
+	var pIdx, wIdx, ttl uint64
+	if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeIndexNaN,
+			`invalid value for "prevIndex"`,
+		)
+	}
+	if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeIndexNaN,
+			`invalid value for "waitIndex"`,
+		)
+	}
+	if ttl, err = getUint64(r.Form, "ttl"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeTTLNaN,
+			`invalid value for "ttl"`,
+		)
+	}
+
+	var rec, sort, wait bool
+	if rec, err = getBool(r.Form, "recursive"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeInvalidField,
+			`invalid value for "recursive"`,
+		)
+	}
+	if sort, err = getBool(r.Form, "sorted"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeInvalidField,
+			`invalid value for "sorted"`,
+		)
+	}
+	if wait, err = getBool(r.Form, "wait"); err != nil {
+		return emptyReq, etcdErr.NewRequestError(
+			etcdErr.EcodeInvalidField,
+			`invalid value for "wait"`,
+		)
+	}
+
+	// prevExists is nullable, so leave it null if not specified
+	var pe *bool
+	if _, ok := r.Form["prevExists"]; ok {
+		bv, err := getBool(r.Form, "prevExists")
+		if err != nil {
+			return emptyReq, etcdErr.NewRequestError(
+				etcdErr.EcodeInvalidField,
+				"invalid value for prevExists",
+			)
+		}
+		pe = &bv
 	}
 
-	q := r.URL.Query()
-	// TODO(jonboulle): perform strict validation of all parameters
-	// https://github.com/coreos/etcd/issues/1011
 	rr := etcdserverpb.Request{
-		Id:        id,
-		Method:    r.Method,
-		Val:       r.FormValue("value"),
-		Path:      r.URL.Path[len(keysPrefix):],
-		PrevValue: q.Get("prevValue"),
-		PrevIndex: parseUint64(q.Get("prevIndex")),
-		Recursive: parseBool(q.Get("recursive")),
-		Since:     parseUint64(q.Get("waitIndex")),
-		Sorted:    parseBool(q.Get("sorted")),
-		Wait:      parseBool(q.Get("wait")),
-	}
-
-	// PrevExists is nullable, so we leave it null if prevExist wasn't
-	// specified.
-	_, ok := q["prevExists"]
-	if ok {
-		bv := parseBool(q.Get("prevExists"))
-		rr.PrevExists = &bv
-	}
-
-	ttl := parseUint64(q.Get("ttl"))
+		Id:         id,
+		Method:     r.Method,
+		Path:       p,
+		Val:        r.FormValue("value"),
+		PrevValue:  r.FormValue("prevValue"),
+		PrevIndex:  pIdx,
+		PrevExists: pe,
+		Recursive:  rec,
+		Since:      wIdx,
+		Sorted:     sort,
+		Wait:       wait,
+	}
+
+	if pe != nil {
+		rr.PrevExists = pe
+	}
+
 	if ttl > 0 {
 		expr := time.Duration(ttl) * time.Second
 		// TODO(jonboulle): use fake clock instead of time module
@@ -299,32 +365,49 @@ func parseRequest(r *http.Request, id int64) (etcdserverpb.Request, error) {
 	return rr, nil
 }
 
-func parseBool(s string) bool {
-	v, _ := strconv.ParseBool(s)
-	return v
+// getUint64 extracts a uint64 by the given key from a Form. If the key does
+// not exist in the form, 0 is returned. If the key exists but the value is
+// badly formed, an error is returned. If multiple values are present only the
+// first is considered.
+func getUint64(form url.Values, key string) (i uint64, err error) {
+	if vals, ok := form[key]; ok {
+		i, err = strconv.ParseUint(vals[0], 10, 64)
+	}
+	return
 }
 
-func parseUint64(s string) uint64 {
-	v, _ := strconv.ParseUint(s, 10, 64)
-	return v
+// getBool extracts a bool by the given key from a Form. If the key does not
+// exist in the form, false is returned. If the key exists but the value is
+// badly formed, an error is returned. If multiple values are present only the
+// first is considered.
+func getBool(form url.Values, key string) (b bool, err error) {
+	if vals, ok := form[key]; ok {
+		b, err = strconv.ParseBool(vals[0])
+	}
+	return
 }
 
-// encodeResponse serializes the given etcdserver Response and writes the
-// resulting JSON to the given ResponseWriter, utilizing the provided context
-func encodeResponse(ctx context.Context, w http.ResponseWriter, resp etcdserver.Response) (err error) {
-	var ev *store.Event
-	switch {
-	case resp.Event != nil:
-		ev = resp.Event
-	case resp.Watcher != nil:
-		ev, err = waitForEvent(ctx, w, resp.Watcher)
-		if err != nil {
-			return err
-		}
-	default:
-		panic("should not be reachable")
+// writeError logs and writes the given Error to the ResponseWriter
+// If Error is an etcdErr, it is rendered to the ResponseWriter
+// Otherwise, it is assumed to be an InternalServerError
+func writeError(w http.ResponseWriter, err error) {
+	if err == nil {
+		return
+	}
+	log.Println(err)
+	if e, ok := err.(*etcdErr.Error); ok {
+		e.Write(w)
+	} else {
+		http.Error(w, "Internal Server Error", http.StatusInternalServerError)
 	}
+}
 
+// writeEvent serializes the given Event and writes the resulting JSON to the
+// given ResponseWriter
+func writeEvent(w http.ResponseWriter, ev *store.Event) error {
+	if ev == nil {
+		return errors.New("cannot write empty Event!")
+	}
 	w.Header().Set("Content-Type", "application/json")
 	w.Header().Add("X-Etcd-Index", fmt.Sprint(ev.Index()))
 
@@ -332,13 +415,10 @@ func encodeResponse(ctx context.Context, w http.ResponseWriter, resp etcdserver.
 		w.WriteHeader(http.StatusCreated)
 	}
 
-	if err := json.NewEncoder(w).Encode(ev); err != nil {
-		panic(err) // should never be reached
-	}
-	return nil
+	return json.NewEncoder(w).Encode(ev)
 }
 
-// waitForEvent waits for a given watcher to return its associated
+// waitForEvent waits for a given Watcher to return its associated
 // event. It returns a non-nil error if the given Context times out
 // or the given ResponseWriter triggers a CloseNotify.
 func waitForEvent(ctx context.Context, w http.ResponseWriter, wa store.Watcher) (*store.Event, error) {
@@ -348,7 +428,6 @@ func waitForEvent(ctx context.Context, w http.ResponseWriter, wa store.Watcher)
 	if x, ok := w.(http.CloseNotifier); ok {
 		nch = x.CloseNotify()
 	}
-
 	select {
 	case ev := <-wa.EventChan():
 		return ev, nil

+ 296 - 67
etcdserver/etcdhttp/http_test.go

@@ -1,15 +1,17 @@
 package etcdhttp
 
 import (
+	"errors"
 	"net/http"
 	"net/http/httptest"
 	"net/url"
 	"path"
 	"reflect"
+	"strings"
 	"sync"
 	"testing"
 
-	"github.com/coreos/etcd/etcdserver"
+	etcdErr "github.com/coreos/etcd/error"
 	"github.com/coreos/etcd/etcdserver/etcdserverpb"
 	"github.com/coreos/etcd/store"
 	"github.com/coreos/etcd/third_party/code.google.com/p/go.net/context"
@@ -25,9 +27,30 @@ func mustNewURL(t *testing.T, s string) *url.URL {
 	return u
 }
 
+// mustNewRequest takes a path, appends it to the standard keysPrefix, and constructs
+// an *http.Request referencing the resulting URL
+func mustNewRequest(t *testing.T, p string) *http.Request {
+	return &http.Request{
+		URL: mustNewURL(t, path.Join(keysPrefix, p)),
+	}
+}
+
+// mustNewForm takes a set of Values and constructs a PUT *http.Request,
+// with a URL constructed from appending the given path to the standard keysPrefix
+func mustNewForm(t *testing.T, p string, vals url.Values) *http.Request {
+	u := mustNewURL(t, path.Join(keysPrefix, p))
+	req, err := http.NewRequest("PUT", u.String(), strings.NewReader(vals.Encode()))
+	req.Header.Set("Content-Type", "application/x-www-form-urlencoded")
+	if err != nil {
+		t.Fatalf("error creating new request: %v", err)
+	}
+	return req
+}
+
 func TestBadParseRequest(t *testing.T) {
 	tests := []struct {
-		in *http.Request
+		in    *http.Request
+		wcode int
 	}{
 		{
 			// parseForm failure
@@ -35,21 +58,121 @@ func TestBadParseRequest(t *testing.T) {
 				Body:   nil,
 				Method: "PUT",
 			},
+			etcdErr.EcodeInvalidForm,
 		},
 		{
 			// bad key prefix
 			&http.Request{
 				URL: mustNewURL(t, "/badprefix/"),
 			},
+			etcdErr.EcodeInvalidForm,
+		},
+		// bad values for prevIndex, waitIndex, ttl
+		{
+			mustNewForm(t, "foo", url.Values{"prevIndex": []string{"garbage"}}),
+			etcdErr.EcodeIndexNaN,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"prevIndex": []string{"1.5"}}),
+			etcdErr.EcodeIndexNaN,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"prevIndex": []string{"-1"}}),
+			etcdErr.EcodeIndexNaN,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"waitIndex": []string{"garbage"}}),
+			etcdErr.EcodeIndexNaN,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"waitIndex": []string{"??"}}),
+			etcdErr.EcodeIndexNaN,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"ttl": []string{"-1"}}),
+			etcdErr.EcodeTTLNaN,
+		},
+		// bad values for recursive, sorted, wait, prevExists
+		{
+			mustNewForm(t, "foo", url.Values{"recursive": []string{"hahaha"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"recursive": []string{"1234"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"recursive": []string{"?"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"sorted": []string{"?"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"sorted": []string{"x"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"wait": []string{"?!"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"wait": []string{"yes"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"prevExists": []string{"yes"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewForm(t, "foo", url.Values{"prevExists": []string{"#2"}}),
+			etcdErr.EcodeInvalidField,
+		},
+		// query values are considered
+		{
+			mustNewRequest(t, "foo?prevExists=wrong"),
+			etcdErr.EcodeInvalidField,
+		},
+		{
+			mustNewRequest(t, "foo?ttl=wrong"),
+			etcdErr.EcodeTTLNaN,
+		},
+		// but body takes precedence if both are specified
+		{
+			mustNewForm(
+				t,
+				"foo?ttl=12",
+				url.Values{"ttl": []string{"garbage"}},
+			),
+			etcdErr.EcodeTTLNaN,
+		},
+		{
+			mustNewForm(
+				t,
+				"foo?prevExists=false",
+				url.Values{"prevExists": []string{"yes"}},
+			),
+			etcdErr.EcodeInvalidField,
 		},
 	}
 	for i, tt := range tests {
 		got, err := parseRequest(tt.in, 1234)
 		if err == nil {
-			t.Errorf("case %d: unexpected nil error!", i)
+			t.Errorf("#%d: unexpected nil error!", i)
+			continue
+		}
+		ee, ok := err.(*etcdErr.Error)
+		if !ok {
+			t.Errorf("#%d: err is not etcd.Error!", i)
+			continue
+		}
+		if ee.ErrorCode != tt.wcode {
+			t.Errorf("#%d: code=%d, want %v", i, ee.ErrorCode, tt.wcode)
+			t.Logf("cause: %#v", ee.Cause)
 		}
 		if !reflect.DeepEqual(got, etcdserverpb.Request{}) {
-			t.Errorf("case %d: unexpected non-empty Request: %#v", i, got)
+			t.Errorf("#%d: unexpected non-empty Request: %#v", i, got)
 		}
 	}
 }
@@ -61,9 +184,7 @@ func TestGoodParseRequest(t *testing.T) {
 	}{
 		{
 			// good prefix, all other values default
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo")),
-			},
+			mustNewRequest(t, "foo"),
 			etcdserverpb.Request{
 				Id:   1234,
 				Path: "/foo",
@@ -71,81 +192,152 @@ func TestGoodParseRequest(t *testing.T) {
 		},
 		{
 			// value specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?value=some_value")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"value": []string{"some_value"}},
+			),
 			etcdserverpb.Request{
-				Id:   1234,
-				Val:  "some_value",
-				Path: "/foo",
+				Id:     1234,
+				Method: "PUT",
+				Val:    "some_value",
+				Path:   "/foo",
 			},
 		},
 		{
 			// prevIndex specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?prevIndex=98765")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"prevIndex": []string{"98765"}},
+			),
 			etcdserverpb.Request{
 				Id:        1234,
+				Method:    "PUT",
 				PrevIndex: 98765,
 				Path:      "/foo",
 			},
 		},
 		{
 			// recursive specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?recursive=true")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"recursive": []string{"true"}},
+			),
 			etcdserverpb.Request{
 				Id:        1234,
+				Method:    "PUT",
 				Recursive: true,
 				Path:      "/foo",
 			},
 		},
 		{
 			// sorted specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?sorted=true")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"sorted": []string{"true"}},
+			),
 			etcdserverpb.Request{
 				Id:     1234,
+				Method: "PUT",
 				Sorted: true,
 				Path:   "/foo",
 			},
 		},
 		{
 			// wait specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?wait=true")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"wait": []string{"true"}},
+			),
 			etcdserverpb.Request{
-				Id:   1234,
-				Wait: true,
-				Path: "/foo",
+				Id:     1234,
+				Method: "PUT",
+				Wait:   true,
+				Path:   "/foo",
 			},
 		},
 		{
 			// prevExists should be non-null if specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?prevExists=true")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"prevExists": []string{"true"}},
+			),
 			etcdserverpb.Request{
 				Id:         1234,
+				Method:     "PUT",
 				PrevExists: boolp(true),
 				Path:       "/foo",
 			},
 		},
 		{
 			// prevExists should be non-null if specified
-			&http.Request{
-				URL: mustNewURL(t, path.Join(keysPrefix, "foo?prevExists=false")),
-			},
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{"prevExists": []string{"false"}},
+			),
 			etcdserverpb.Request{
 				Id:         1234,
+				Method:     "PUT",
 				PrevExists: boolp(false),
 				Path:       "/foo",
 			},
 		},
+		// mix various fields
+		{
+			mustNewForm(
+				t,
+				"foo",
+				url.Values{
+					"value":      []string{"some value"},
+					"prevExists": []string{"true"},
+					"prevValue":  []string{"previous value"},
+				},
+			),
+			etcdserverpb.Request{
+				Id:         1234,
+				Method:     "PUT",
+				PrevExists: boolp(true),
+				PrevValue:  "previous value",
+				Val:        "some value",
+				Path:       "/foo",
+			},
+		},
+		// query parameters should be used if given
+		{
+			mustNewForm(
+				t,
+				"foo?prevValue=woof",
+				url.Values{},
+			),
+			etcdserverpb.Request{
+				Id:        1234,
+				Method:    "PUT",
+				PrevValue: "woof",
+				Path:      "/foo",
+			},
+		},
+		// but form values should take precedence over query parameters
+		{
+			mustNewForm(
+				t,
+				"foo?prevValue=woof",
+				url.Values{
+					"prevValue": []string{"miaow"},
+				},
+			),
+			etcdserverpb.Request{
+				Id:        1234,
+				Method:    "PUT",
+				PrevValue: "miaow",
+				Path:      "/foo",
+			},
+		},
 	}
 
 	for i, tt := range tests {
@@ -154,7 +346,7 @@ func TestGoodParseRequest(t *testing.T) {
 			t.Errorf("#%d: err = %v, want %v", i, err, nil)
 		}
 		if !reflect.DeepEqual(got, tt.w) {
-			t.Errorf("#%d: bad request: got %#v, want %#v", i, got, tt.w)
+			t.Errorf("#%d: request=%#v, want %#v", i, got, tt.w)
 		}
 	}
 }
@@ -177,22 +369,77 @@ func (w *eventingWatcher) EventChan() chan *store.Event {
 
 func (w *eventingWatcher) Remove() {}
 
-func TestEncodeResponse(t *testing.T) {
+func TestWriteError(t *testing.T) {
+	// nil error should not panic
+	rw := httptest.NewRecorder()
+	writeError(rw, nil)
+	h := rw.Header()
+	if len(h) > 0 {
+		t.Fatalf("unexpected non-empty headers: %#v", h)
+	}
+	b := rw.Body.String()
+	if len(b) > 0 {
+		t.Fatalf("unexpected non-empty body: %q", b)
+	}
+
 	tests := []struct {
-		resp etcdserver.Response
+		err   error
+		wcode int
+		wi    string
+	}{
+		{
+			etcdErr.NewError(etcdErr.EcodeKeyNotFound, "/foo/bar", 123),
+			http.StatusNotFound,
+			"123",
+		},
+		{
+			etcdErr.NewError(etcdErr.EcodeTestFailed, "/foo/bar", 456),
+			http.StatusPreconditionFailed,
+			"456",
+		},
+		{
+			err:   errors.New("something went wrong"),
+			wcode: http.StatusInternalServerError,
+		},
+	}
+
+	for i, tt := range tests {
+		rw := httptest.NewRecorder()
+		writeError(rw, tt.err)
+		if code := rw.Code; code != tt.wcode {
+			t.Errorf("#%d: code=%d, want %d", i, code, tt.wcode)
+		}
+		if idx := rw.Header().Get("X-Etcd-Index"); idx != tt.wi {
+			t.Errorf("#%d: X-Etcd-Index=%q, want %q", i, idx, tt.wi)
+		}
+	}
+}
+
+func TestWriteEvent(t *testing.T) {
+	// nil event should not panic
+	rw := httptest.NewRecorder()
+	writeEvent(rw, nil)
+	h := rw.Header()
+	if len(h) > 0 {
+		t.Fatalf("unexpected non-empty headers: %#v", h)
+	}
+	b := rw.Body.String()
+	if len(b) > 0 {
+		t.Fatalf("unexpected non-empty body: %q", b)
+	}
+
+	tests := []struct {
+		ev   *store.Event
 		idx  string
 		code int
 		err  error
 	}{
 		// standard case, standard 200 response
 		{
-			etcdserver.Response{
-				Event: &store.Event{
-					Action:   store.Get,
-					Node:     &store.NodeExtern{},
-					PrevNode: &store.NodeExtern{},
-				},
-				Watcher: nil,
+			&store.Event{
+				Action:   store.Get,
+				Node:     &store.NodeExtern{},
+				PrevNode: &store.NodeExtern{},
 			},
 			"0",
 			http.StatusOK,
@@ -200,21 +447,10 @@ func TestEncodeResponse(t *testing.T) {
 		},
 		// check new nodes return StatusCreated
 		{
-			etcdserver.Response{
-				Event: &store.Event{
-					Action:   store.Create,
-					Node:     &store.NodeExtern{},
-					PrevNode: &store.NodeExtern{},
-				},
-				Watcher: nil,
-			},
-			"0",
-			http.StatusCreated,
-			nil,
-		},
-		{
-			etcdserver.Response{
-				Watcher: &eventingWatcher{store.Create},
+			&store.Event{
+				Action:   store.Create,
+				Node:     &store.NodeExtern{},
+				PrevNode: &store.NodeExtern{},
 			},
 			"0",
 			http.StatusCreated,
@@ -224,20 +460,13 @@ func TestEncodeResponse(t *testing.T) {
 
 	for i, tt := range tests {
 		rw := httptest.NewRecorder()
-		err := encodeResponse(context.Background(), rw, tt.resp)
-		if err != tt.err {
-			t.Errorf("case %d: unexpected err: got %v, want %v", i, err, tt.err)
-			continue
-		}
-
+		writeEvent(rw, tt.ev)
 		if gct := rw.Header().Get("Content-Type"); gct != "application/json" {
 			t.Errorf("case %d: bad Content-Type: got %q, want application/json", i, gct)
 		}
-
 		if gei := rw.Header().Get("X-Etcd-Index"); gei != tt.idx {
 			t.Errorf("case %d: bad X-Etcd-Index header: got %s, want %s", i, gei, tt.idx)
 		}
-
 		if rw.Code != tt.code {
 			t.Errorf("case %d: bad response code: got %d, want %v", i, rw.Code, tt.code)
 		}