Browse Source

refactor move http status logic to server.go

Xiang Li 12 years ago
parent
commit
b852a746ce
30 changed files with 2744 additions and 50 deletions
  1. 4 2
      etcd.go
  2. 61 6
      server/peer_server.go
  3. 65 4
      server/server.go
  4. 14 9
      server/v2/put_handler.go
  5. 152 0
      server/v2/tests/get_handler_test.go
  6. 38 0
      server/v2/tests/post_handler_test.go
  7. 280 0
      server/v2/tests/put_handler_test.go
  8. 4 5
      store/event.go
  9. 3 0
      test.sh
  10. 1 2
      tests/functional/internal_version_test.go
  11. 67 0
      tests/http_utils.go
  12. 66 0
      tests/mock/mock_store.go
  13. 55 0
      tests/mock/server_v2.go
  14. 53 0
      tests/server_utils.go
  15. 2 0
      third_party/deps
  16. 4 4
      third_party/github.com/coreos/go-etcd/etcd/client_test.go
  17. 1 0
      third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go
  18. 15 13
      third_party/github.com/coreos/go-log/log/commands.go
  19. 0 1
      third_party/github.com/coreos/go-log/log/fields.go
  20. 0 1
      third_party/github.com/coreos/go-log/log/logger.go
  21. 0 1
      third_party/github.com/coreos/go-log/log/priority.go
  22. 0 1
      third_party/github.com/coreos/go-log/log/sinks.go
  23. 1 1
      third_party/github.com/coreos/go-systemd/activation/files.go
  24. 465 0
      third_party/github.com/stretchr/testify/assert/assertions.go
  25. 383 0
      third_party/github.com/stretchr/testify/assert/assertions_test.go
  26. 74 0
      third_party/github.com/stretchr/testify/assert/doc.go
  27. 10 0
      third_party/github.com/stretchr/testify/assert/errors.go
  28. 43 0
      third_party/github.com/stretchr/testify/mock/doc.go
  29. 465 0
      third_party/github.com/stretchr/testify/mock/mock.go
  30. 418 0
      third_party/github.com/stretchr/testify/mock/mock_test.go

+ 4 - 2
etcd.go

@@ -194,6 +194,8 @@ func main() {
 
 	ps.SetServer(s)
 
-	ps.ListenAndServe(snapshot, cluster)
-	s.ListenAndServe()
+	go func() {
+		log.Fatal(ps.ListenAndServe(snapshot, cluster))
+	}()
+	log.Fatal(s.ListenAndServe())
 }

+ 61 - 6
server/peer_server.go

@@ -7,6 +7,7 @@ import (
 	"encoding/json"
 	"fmt"
 	"io/ioutil"
+	"net"
 	"net/http"
 	"net/url"
 	"time"
@@ -20,6 +21,8 @@ import (
 type PeerServer struct {
 	raftServer     raft.Server
 	server         *Server
+	httpServer     *http.Server
+	listener       net.Listener
 	joinIndex      uint64
 	name           string
 	url            string
@@ -88,7 +91,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC
 }
 
 // Start the raft server
-func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
+func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) error {
 	// LoadSnapshot
 	if snapshot {
 		err := s.raftServer.LoadSnapshot()
@@ -137,8 +140,60 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) {
 	}
 
 	// start to response to raft requests
-	go s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server)
+	return s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server)
+}
+
+// Overridden version of net/http added so we can manage the listener.
+func (s *PeerServer) listenAndServe() error {
+	addr := s.httpServer.Addr
+	if addr == "" {
+		addr = ":http"
+	}
+	l, e := net.Listen("tcp", addr)
+	if e != nil {
+		return e
+	}
+	s.listener = l
+	return s.httpServer.Serve(l)
+}
+
+// Overridden version of net/http added so we can manage the listener.
+func (s *PeerServer) listenAndServeTLS(certFile, keyFile string) error {
+	addr := s.httpServer.Addr
+	if addr == "" {
+		addr = ":https"
+	}
+	config := &tls.Config{}
+	if s.httpServer.TLSConfig != nil {
+		*config = *s.httpServer.TLSConfig
+	}
+	if config.NextProtos == nil {
+		config.NextProtos = []string{"http/1.1"}
+	}
 
+	var err error
+	config.Certificates = make([]tls.Certificate, 1)
+	config.Certificates[0], err = tls.LoadX509KeyPair(certFile, keyFile)
+	if err != nil {
+		return err
+	}
+
+	conn, err := net.Listen("tcp", addr)
+	if err != nil {
+		return err
+	}
+
+	tlsListener := tls.NewListener(conn, config)
+	s.listener = tlsListener
+	return s.httpServer.Serve(tlsListener)
+}
+
+// Stops the server.
+func (s *PeerServer) Close() {
+	if s.listener != nil {
+		s.listener.Close()
+		s.listener = nil
+	}
 }
 
 // Retrieves the underlying Raft server.
@@ -177,12 +232,12 @@ func (s *PeerServer) startAsFollower(cluster []string) {
 }
 
 // Start to listen and response raft command
-func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) {
+func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error {
 	log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url)
 
 	raftMux := http.NewServeMux()
 
-	server := &http.Server{
+	s.httpServer = &http.Server{
 		Handler:   raftMux,
 		TLSConfig: &tlsConf,
 		Addr:      s.listenHost,
@@ -201,9 +256,9 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) {
 	raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler)
 
 	if scheme == "http" {
-		log.Fatal(server.ListenAndServe())
+		return s.listenAndServe()
 	} else {
-		log.Fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile))
+		return s.listenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)
 	}
 
 }

+ 65 - 4
server/server.go

@@ -1,8 +1,10 @@
 package server
 
 import (
+	"crypto/tls"
 	"encoding/json"
 	"fmt"
+	"net"
 	"net/http"
 	"net/url"
 	"strings"
@@ -22,6 +24,7 @@ type Server struct {
 	http.Server
 	peerServer  *PeerServer
 	registry    *Registry
+	listener    net.Listener
 	store       store.Store
 	name        string
 	url         string
@@ -157,13 +160,66 @@ func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Reque
 }
 
 // Start to listen and response etcd client command
-func (s *Server) ListenAndServe() {
+func (s *Server) ListenAndServe() error {
 	log.Infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url)
 
 	if s.tlsConf.Scheme == "http" {
-		log.Fatal(s.Server.ListenAndServe())
+		return s.listenAndServe()
 	} else {
-		log.Fatal(s.Server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile))
+		return s.listenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)
+	}
+}
+
+// Overridden version of net/http added so we can manage the listener.
+func (s *Server) listenAndServe() error {
+	addr := s.Server.Addr
+	if addr == "" {
+		addr = ":http"
+	}
+	l, e := net.Listen("tcp", addr)
+	if e != nil {
+		return e
+	}
+	s.listener = l
+	return s.Server.Serve(l)
+}
+
+// Overridden version of net/http added so we can manage the listener.
+func (s *Server) listenAndServeTLS(certFile, keyFile string) error {
+	addr := s.Server.Addr
+	if addr == "" {
+		addr = ":https"
+	}
+	config := &tls.Config{}
+	if s.Server.TLSConfig != nil {
+		*config = *s.Server.TLSConfig
+	}
+	if config.NextProtos == nil {
+		config.NextProtos = []string{"http/1.1"}
+	}
+
+	var err error
+	config.Certificates = make([]tls.Certificate, 1)
+	config.Certificates[0], err = tls.LoadX509KeyPair(certFile, keyFile)
+	if err != nil {
+		return err
+	}
+
+	conn, err := net.Listen("tcp", addr)
+	if err != nil {
+		return err
+	}
+
+	tlsListener := tls.NewListener(conn, config)
+	s.listener = tlsListener
+	return s.Server.Serve(tlsListener)
+}
+
+// Stops the server.
+func (s *Server) Close() {
+	if s.listener != nil {
+		s.listener.Close()
+		s.listener = nil
 	}
 }
 
@@ -193,7 +249,12 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque
 		} else {
 			e, _ := result.(*store.Event)
 			b, _ = json.Marshal(e)
-			w.WriteHeader(e.HttpStatusCode())
+
+			if e.IsCreated() {
+				w.WriteHeader(http.StatusCreated)
+			} else {
+				w.WriteHeader(http.StatusOK)
+			}
 		}
 
 		w.Write(b)

+ 14 - 9
server/v2/put_handler.go

@@ -12,6 +12,8 @@ import (
 )
 
 func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
+	var c raft.Command
+
 	vars := mux.Vars(req)
 	key := "/" + vars["key"]
 
@@ -23,11 +25,14 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 		return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm)
 	}
 
-	prevValue, valueOk := req.Form["prevValue"]
-	prevIndexStr, indexOk := req.Form["prevIndex"]
-	prevExist, existOk := req.Form["prevExist"]
+	_, valueOk := req.Form["prevValue"]
+	prevValue := req.Form.Get("prevValue")
 
-	var c raft.Command
+	_, indexOk := req.Form["prevIndex"]
+	prevIndexStr := req.Form.Get("prevIndex")
+
+	_, existOk := req.Form["prevExist"]
+	prevExist := req.Form.Get("prevExist")
 
 	// Set handler: create a new node or replace the old one.
 	if !valueOk && !indexOk && !existOk {
@@ -36,13 +41,13 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 
 	// update with test
 	if existOk {
-		if prevExist[0] == "false" {
+		if prevExist == "false" {
 			// Create command: create a new node. Fail, if a node already exists
 			// Ignore prevIndex and prevValue
 			return CreateHandler(w, req, s, key, value, expireTime)
 		}
 
-		if prevExist[0] == "true" && !indexOk && !valueOk {
+		if prevExist == "true" && !indexOk && !valueOk {
 			return UpdateHandler(w, req, s, key, value, expireTime)
 		}
 	}
@@ -50,7 +55,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	var prevIndex uint64
 
 	if indexOk {
-		prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64)
+		prevIndex, err = strconv.ParseUint(prevIndexStr, 10, 64)
 
 		// bad previous index
 		if err != nil {
@@ -61,7 +66,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	}
 
 	if valueOk {
-		if prevValue[0] == "" {
+		if prevValue == "" {
 			return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", store.UndefIndex, store.UndefTerm)
 		}
 	}
@@ -69,7 +74,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error {
 	c = &store.CompareAndSwapCommand{
 		Key:       key,
 		Value:     value,
-		PrevValue: prevValue[0],
+		PrevValue: prevValue,
 		PrevIndex: prevIndex,
 	}
 

+ 152 - 0
server/v2/tests/get_handler_test.go

@@ -0,0 +1,152 @@
+package v2
+
+import (
+	"fmt"
+	"net/url"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/server"
+	"github.com/coreos/etcd/tests"
+	"github.com/stretchr/testify/assert"
+)
+
+// Ensures that a value can be retrieve for a given key.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl localhost:4001/v2/keys/foo/bar
+//
+func TestV2GetKey(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"))
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "get", "")
+		assert.Equal(t, body["key"], "/foo/bar", "")
+		assert.Equal(t, body["value"], "XXX", "")
+		assert.Equal(t, body["index"], 3, "")
+		assert.Equal(t, body["term"], 0, "")
+	})
+}
+
+// Ensures that a directory of values can be recursively retrieved for a given key.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/x -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/y/z -d value=YYY
+//   $ curl localhost:4001/v2/keys/foo -d recursive=true
+//
+func TestV2GetKeyRecursively(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/x"), v)
+		tests.ReadBody(resp)
+
+		v.Set("value", "YYY")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/y/z"), v)
+		tests.ReadBody(resp)
+
+		resp, _ = tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo?recursive=true"))
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "get", "")
+		assert.Equal(t, body["key"], "/foo", "")
+		assert.Equal(t, body["dir"], true, "")
+		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, len(body["kvs"].([]interface{})), 2, "")
+
+		kv0 := body["kvs"].([]interface{})[0].(map[string]interface{})
+		assert.Equal(t, kv0["key"], "/foo/x", "")
+		assert.Equal(t, kv0["value"], "XXX", "")
+
+		kv1 := body["kvs"].([]interface{})[1].(map[string]interface{})
+		assert.Equal(t, kv1["key"], "/foo/y", "")
+		assert.Equal(t, kv1["dir"], true, "")
+
+		kvs2 := kv1["kvs"].([]interface{})[0].(map[string]interface{})
+		assert.Equal(t, kvs2["key"], "/foo/y/z", "")
+		assert.Equal(t, kvs2["value"], "YYY", "")
+	})
+}
+
+// Ensures that a watcher can wait for a value to be set and return it to the client.
+//
+//   $ curl localhost:4001/v2/keys/foo/bar?wait=true
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//
+func TestV2WatchKey(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		var body map[string]interface{}
+		go func() {
+			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true"))
+			body = tests.ReadBodyJSON(resp)
+		}()
+
+		// Make sure response didn't fire early.
+		time.Sleep(1 * time.Millisecond)
+		assert.Nil(t, body, "")
+
+		// Set a value.
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+
+		// A response should follow from the GET above.
+		time.Sleep(1 * time.Millisecond)
+		assert.NotNil(t, body, "")
+		assert.Equal(t, body["action"], "set", "")
+		assert.Equal(t, body["key"], "/foo/bar", "")
+		assert.Equal(t, body["value"], "XXX", "")
+		assert.Equal(t, body["index"], 3, "")
+		assert.Equal(t, body["term"], 0, "")
+	})
+}
+
+
+// Ensures that a watcher can wait for a value to be set after a given index.
+//
+//   $ curl localhost:4001/v2/keys/foo/bar?wait=true&waitIndex=4
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY
+//
+func TestV2WatchKeyWithIndex(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		var body map[string]interface{}
+		go func() {
+			resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=5"))
+			body = tests.ReadBodyJSON(resp)
+		}()
+
+		// Make sure response didn't fire early.
+		time.Sleep(1 * time.Millisecond)
+		assert.Nil(t, body, "")
+
+		// Set a value (before given index).
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+
+		// Make sure response didn't fire early.
+		time.Sleep(1 * time.Millisecond)
+		assert.Nil(t, body, "")
+
+		// Set a value (before given index).
+		v.Set("value", "YYY")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+
+		// A response should follow from the GET above.
+		time.Sleep(1 * time.Millisecond)
+		assert.NotNil(t, body, "")
+		assert.Equal(t, body["action"], "set", "")
+		assert.Equal(t, body["key"], "/foo/bar", "")
+		assert.Equal(t, body["value"], "YYY", "")
+		assert.Equal(t, body["index"], 4, "")
+		assert.Equal(t, body["term"], 0, "")
+	})
+}
+

+ 38 - 0
server/v2/tests/post_handler_test.go

@@ -0,0 +1,38 @@
+package v2
+
+import (
+	"fmt"
+	"testing"
+
+	"github.com/coreos/etcd/server"
+	"github.com/coreos/etcd/tests"
+	"github.com/stretchr/testify/assert"
+)
+
+// Ensures a unique value is added to the key's children.
+//
+//   $ curl -X POST localhost:4001/v2/keys/foo/bar
+//   $ curl -X POST localhost:4001/v2/keys/foo/bar
+//   $ curl -X POST localhost:4001/v2/keys/foo/baz
+//
+func TestV2CreateUnique(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		// POST should add index to list.
+		resp, _ := tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "create", "")
+		assert.Equal(t, body["key"], "/foo/bar/3", "")
+		assert.Equal(t, body["dir"], true, "")
+		assert.Equal(t, body["index"], 3, "")
+
+		// Second POST should add next index to list.
+		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil)
+		body = tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["key"], "/foo/bar/4", "")
+
+		// POST to a different key should add index to that list.
+		resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/baz"), nil)
+		body = tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["key"], "/foo/baz/5", "")
+	})
+}

+ 280 - 0
server/v2/tests/put_handler_test.go

@@ -0,0 +1,280 @@
+package v2
+
+import (
+	"fmt"
+	"net/url"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/server"
+	"github.com/coreos/etcd/tests"
+	"github.com/stretchr/testify/assert"
+)
+
+// Ensures that a key is set to a given value.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//
+func TestV2SetKey(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBody(resp)
+		assert.Nil(t, err, "")
+		assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":3,"term":0}`, "")
+	})
+}
+
+// Ensures that a time-to-live is added to a key.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d ttl=20
+//
+func TestV2SetKeyWithTTL(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		t0 := time.Now()
+		v := url.Values{}
+		v.Set("value", "XXX")
+		v.Set("ttl", "20")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["ttl"], 20, "")
+
+		// Make sure the expiration date is correct.
+		expiration, _ := time.Parse(time.RFC3339Nano, body["expiration"].(string))
+		assert.Equal(t, expiration.Sub(t0) / time.Second, 20, "")
+	})
+}
+
+// Ensures that an invalid time-to-live is returned as an error.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d ttl=bad_ttl
+//
+func TestV2SetKeyWithBadTTL(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		v.Set("ttl", "bad_ttl")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 202, "")
+		assert.Equal(t, body["message"], "The given TTL in POST form is not a number", "")
+		assert.Equal(t, body["cause"], "Update", "")
+	})
+}
+
+// Ensures that a key is conditionally set only if it previously did not exist.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=false
+//
+func TestV2CreateKeySuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		v.Set("prevExist", "false")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["value"], "XXX", "")
+	})
+}
+
+// Ensures that a key is not conditionally because it previously existed.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=false
+//
+func TestV2CreateKeyFail(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		v.Set("prevExist", "false")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 105, "")
+		assert.Equal(t, body["message"], "Already exists", "")
+		assert.Equal(t, body["cause"], "/foo/bar", "")
+	})
+}
+
+// Ensures that a key is conditionally set only if it previously did exist.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevExist=true
+//
+func TestV2UpdateKeySuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		
+		v.Set("value", "YYY")
+		v.Set("prevExist", "true")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "update", "")
+		assert.Equal(t, body["prevValue"], "XXX", "")
+	})
+}
+
+// Ensures that a key is not conditionally set if it previously did not exist.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=true
+//
+func TestV2UpdateKeyFailOnValue(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo"), v)
+
+		v.Set("value", "YYY")
+		v.Set("prevExist", "true")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 100, "")
+		assert.Equal(t, body["message"], "Key Not Found", "")
+		assert.Equal(t, body["cause"], "/foo/bar", "")
+	})
+}
+
+// Ensures that a key is not conditionally set if it previously did not exist.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo -d value=XXX -d prevExist=true
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=true
+//
+func TestV2UpdateKeyFailOnMissingDirectory(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "YYY")
+		v.Set("prevExist", "true")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 100, "")
+		assert.Equal(t, body["message"], "Key Not Found", "")
+		assert.Equal(t, body["cause"], "/foo", "")
+	})
+}
+
+// Ensures that a key is set only if the previous index matches.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=3
+//
+func TestV2SetKeyCASOnIndexSuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		v.Set("value", "YYY")
+		v.Set("prevIndex", "3")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "compareAndSwap", "")
+		assert.Equal(t, body["prevValue"], "XXX", "")
+		assert.Equal(t, body["value"], "YYY", "")
+		assert.Equal(t, body["index"], 4, "")
+	})
+}
+
+// Ensures that a key is not set if the previous index does not match.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=10
+//
+func TestV2SetKeyCASOnIndexFail(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		v.Set("value", "YYY")
+		v.Set("prevIndex", "10")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 101, "")
+		assert.Equal(t, body["message"], "Test Failed", "")
+		assert.Equal(t, body["cause"], "[ != XXX] [10 != 3]", "")
+		assert.Equal(t, body["index"], 4, "")
+	})
+}
+
+// Ensures that an error is thrown if an invalid previous index is provided.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=bad_index
+//
+func TestV2SetKeyCASWithInvalidIndex(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "YYY")
+		v.Set("prevIndex", "bad_index")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 203, "")
+		assert.Equal(t, body["message"], "The given index in POST form is not a number", "")
+		assert.Equal(t, body["cause"], "CompareAndSwap", "")
+	})
+}
+
+// Ensures that a key is set only if the previous value matches.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevValue=XXX
+//
+func TestV2SetKeyCASOnValueSuccess(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		v.Set("value", "YYY")
+		v.Set("prevValue", "XXX")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["action"], "compareAndSwap", "")
+		assert.Equal(t, body["prevValue"], "XXX", "")
+		assert.Equal(t, body["value"], "YYY", "")
+		assert.Equal(t, body["index"], 4, "")
+	})
+}
+
+// Ensures that a key is not set if the previous value does not match.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevValue=AAA
+//
+func TestV2SetKeyCASOnValueFail(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		tests.ReadBody(resp)
+		v.Set("value", "YYY")
+		v.Set("prevValue", "AAA")
+		resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 101, "")
+		assert.Equal(t, body["message"], "Test Failed", "")
+		assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 3]", "")
+		assert.Equal(t, body["index"], 4, "")
+	})
+}
+
+// Ensures that an error is returned if a blank prevValue is set.
+//
+//   $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevValue=
+//
+func TestV2SetKeyCASWithMissingValueFails(t *testing.T) {
+	tests.RunServer(func(s *server.Server) {
+		v := url.Values{}
+		v.Set("value", "XXX")
+		v.Set("prevValue", "")
+		resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v)
+		body := tests.ReadBodyJSON(resp)
+		assert.Equal(t, body["errorCode"], 201, "")
+		assert.Equal(t, body["message"], "PrevValue is Required in POST form", "")
+		assert.Equal(t, body["cause"], "CompareAndSwap", "")
+	})
+}

+ 4 - 5
store/event.go

@@ -1,7 +1,6 @@
 package store
 
 import (
-	"net/http"
 	"time"
 )
 
@@ -43,16 +42,16 @@ func newEvent(action string, key string, index uint64, term uint64) *Event {
 	}
 }
 
-func (e *Event) HttpStatusCode() int {
+func (e *Event) IsCreated() bool {
 	if e.Action == Create {
-		return http.StatusCreated
+		return true
 	}
 
 	if e.Action == Set && e.PrevValue == "" {
-		return http.StatusCreated
+		return true
 	}
 
-	return http.StatusOK
+	return false
 }
 
 // Converts an event object into a response object.

+ 3 - 0
test.sh

@@ -8,6 +8,9 @@ set -e
 export GOPATH="${PWD}"
 
 # Unit tests
+go test -i ./server/v2/tests
+go test -v ./server/v2/tests
+
 go test -i ./store
 go test -v ./store
 

+ 1 - 2
tests/functional/internal_version_test.go

@@ -38,12 +38,11 @@ func TestInternalVersion(t *testing.T) {
 		t.Fatal("start process failed:" + err.Error())
 		return
 	}
-	defer process.Kill()
 
 	time.Sleep(time.Second)
+	process.Kill()
 
 	_, err = http.Get("http://127.0.0.1:4001")
-
 	if err == nil {
 		t.Fatal("etcd node should not be up")
 		return

+ 67 - 0
tests/http_utils.go

@@ -0,0 +1,67 @@
+package tests
+
+import (
+	"encoding/json"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"net/http"
+	"net/url"
+	"strings"
+)
+
+// Creates a new HTTP client with KeepAlive disabled.
+func NewHTTPClient() *http.Client {
+	return &http.Client{Transport: &http.Transport{DisableKeepAlives: true}}
+}
+
+// Reads the body from the response and closes it.
+func ReadBody(resp *http.Response) []byte {
+	if resp == nil {
+		return []byte{}
+	}
+	body, _ := ioutil.ReadAll(resp.Body)
+	resp.Body.Close()
+	return body
+}
+
+// Reads the body from the response and parses it as JSON.
+func ReadBodyJSON(resp *http.Response) map[string]interface{} {
+	m := make(map[string]interface{})
+	b := ReadBody(resp)
+	if err := json.Unmarshal(b, &m); err != nil {
+		panic(fmt.Sprintf("HTTP body JSON parse error: %v", err))
+	}
+	return m
+}
+
+func Get(url string) (*http.Response, error) {
+	return send("GET", url, "application/json", nil)
+}
+
+func Post(url string, bodyType string, body io.Reader) (*http.Response, error) {
+	return send("POST", url, bodyType, body)
+}
+
+func PostForm(url string, data url.Values) (*http.Response, error) {
+	return Post(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode()))
+}
+
+func Put(url string, bodyType string, body io.Reader) (*http.Response, error) {
+	return send("PUT", url, bodyType, body)
+}
+
+func PutForm(url string, data url.Values) (*http.Response, error) {
+	return Put(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode()))
+}
+
+
+func send(method string, url string, bodyType string, body io.Reader) (*http.Response, error) {
+	c := NewHTTPClient()
+	req, err := http.NewRequest(method, url, body)
+	if err != nil {
+		return nil, err
+	}
+	req.Header.Set("Content-Type", bodyType)
+	return c.Do(req)
+}

+ 66 - 0
tests/mock/mock_store.go

@@ -0,0 +1,66 @@
+package mock
+
+import (
+	"github.com/coreos/etcd/store"
+	"github.com/stretchr/testify/mock"
+	"time"
+)
+
+// A mock Store object used for testing.
+type Store struct {
+	mock.Mock
+}
+
+func NewStore() *Store {
+	return &Store{}
+}
+
+func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, recursive, sorted, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, value, expireTime, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, newValue, expireTime, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, value, incrementalSuffix, expireTime, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, prevValue, prevIndex, value, expireTime, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*store.Event, error) {
+	args := s.Called(nodePath, recursive, index, term)
+	return args.Get(0).(*store.Event), args.Error(1)
+}
+
+func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *store.Event, error) {
+	args := s.Called(prefix, recursive, sinceIndex, index, term)
+	return args.Get(0).(<-chan *store.Event), args.Error(1)
+}
+
+func (s *Store) Save() ([]byte, error) {
+	args := s.Called()
+	return args.Get(0).([]byte), args.Error(1)
+}
+
+func (s *Store) Recovery(b []byte) error {
+	args := s.Called(b)
+	return args.Error(1)
+}
+
+func (s *Store) JsonStats() []byte {
+	args := s.Called()
+	return args.Get(0).([]byte)
+}

+ 55 - 0
tests/mock/server_v2.go

@@ -0,0 +1,55 @@
+package mock
+
+import (
+	"net/http"
+	
+	"github.com/coreos/etcd/store"
+	"github.com/coreos/go-raft"
+	"github.com/stretchr/testify/mock"
+)
+
+// A mock Server for the v2 handlers.
+type ServerV2 struct {
+	mock.Mock
+	store store.Store
+}
+
+func NewServerV2(store store.Store) *ServerV2 {
+	return &ServerV2{
+		store: store,
+	}
+}
+
+func (s *ServerV2) State() string {
+	args := s.Called()
+	return args.String(0)
+}
+
+func (s *ServerV2) Leader() string {
+	args := s.Called()
+	return args.String(0)
+}
+
+func (s *ServerV2) CommitIndex() uint64 {
+	args := s.Called()
+	return args.Get(0).(uint64)
+}
+
+func (s *ServerV2) Term() uint64 {
+	args := s.Called()
+	return args.Get(0).(uint64)
+}
+
+func (s *ServerV2) PeerURL(name string) (string, bool) {
+	args := s.Called(name)
+	return args.String(0), args.Bool(1)
+}
+
+func (s *ServerV2) Store() store.Store {
+	return s.store
+}
+
+func (s *ServerV2) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error {
+	args := s.Called(c, w, req)
+	return args.Error(0)
+}

+ 53 - 0
tests/server_utils.go

@@ -0,0 +1,53 @@
+package tests
+
+import (
+	"io/ioutil"
+	"os"
+	"time"
+
+	"github.com/coreos/etcd/store"
+	"github.com/coreos/etcd/server"
+)
+
+const (
+	testName = "ETCDTEST"
+	testClientURL = "localhost:4401"
+	testRaftURL = "localhost:7701"
+)
+
+// Starts a server in a temporary directory.
+func RunServer(f func(*server.Server)) {
+	path, _ := ioutil.TempDir("", "etcd-")
+	defer os.RemoveAll(path)
+
+	store := store.New()
+	registry := server.NewRegistry(store)
+	ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, registry, store)
+	s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, ps, registry, store)
+	ps.SetServer(s)
+
+	// Start up peer server.
+	c := make(chan bool)
+	go func() {
+		c <- true
+		ps.ListenAndServe(false, []string{})
+	}()
+	<- c
+
+	// Start up etcd server.
+	go func() {
+		c <- true
+		s.ListenAndServe()
+	}()
+	<- c
+
+	// Wait to make sure servers have started.
+	time.Sleep(5 * time.Millisecond)
+
+	// Execute the function passed in.
+	f(s)
+
+	// Clean up servers.
+	ps.Close()
+	s.Close()
+}

+ 2 - 0
third_party/deps

@@ -5,6 +5,8 @@ packages="
 	github.com/coreos/go-systemd
 	github.com/gorilla/context
 	github.com/gorilla/mux
+	github.com/stretchr/testify/assert
+	github.com/stretchr/testify/mock
 	bitbucket.org/kardianos/osext
 	code.google.com/p/go.net
 	code.google.com/p/goprotobuf

+ 4 - 4
third_party/github.com/coreos/go-etcd/etcd/client_test.go

@@ -2,9 +2,9 @@ package etcd
 
 import (
 	"fmt"
-	"net"
-	"net/url"
 	"testing"
+	"net/url"
+	"net"
 )
 
 // To pass this test, we need to create a cluster of 3 machines
@@ -19,7 +19,7 @@ func TestSync(t *testing.T) {
 		t.Fatal("cannot sync machines")
 	}
 
-	for _, m := range c.GetCluster() {
+	for _, m := range(c.GetCluster()) {
 		u, err := url.Parse(m)
 		if err != nil {
 			t.Fatal(err)
@@ -27,7 +27,7 @@ func TestSync(t *testing.T) {
 		if u.Scheme != "http" {
 			t.Fatal("scheme must be http")
 		}
-
+		
 		host, _, err := net.SplitHostPort(u.Host)
 		if err != nil {
 			t.Fatal(err)

+ 1 - 0
third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go

@@ -1,3 +1,4 @@
+
 package main
 
 import (

+ 15 - 13
third_party/github.com/coreos/go-log/log/commands.go

@@ -1,5 +1,4 @@
 package log
-
 // Copyright 2013, CoreOS, Inc. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");
@@ -43,6 +42,7 @@ func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) {
 	logger.Log(priority, fmt.Sprintf(format, v...))
 }
 
+
 func (logger *Logger) Emergency(v ...interface{}) {
 	logger.Log(PriEmerg, v...)
 }
@@ -99,6 +99,7 @@ func (logger *Logger) Debugf(format string, v ...interface{}) {
 	logger.Log(PriDebug, fmt.Sprintf(format, v...))
 }
 
+
 func Emergency(v ...interface{}) {
 	defaultLogger.Log(PriEmerg, v...)
 }
@@ -157,56 +158,57 @@ func Debugf(format string, v ...interface{}) {
 
 // Standard library log functions
 
-func (logger *Logger) Fatalln(v ...interface{}) {
+func (logger *Logger)Fatalln (v ...interface{}) {
 	logger.Log(PriCrit, v...)
 	os.Exit(1)
 }
-func (logger *Logger) Fatalf(format string, v ...interface{}) {
+func (logger *Logger)Fatalf (format string, v ...interface{}) {
 	logger.Logf(PriCrit, format, v...)
 	os.Exit(1)
 }
 
-func (logger *Logger) Panicln(v ...interface{}) {
+func (logger *Logger)Panicln (v ...interface{}) {
 	s := fmt.Sprint(v...)
 	logger.Log(PriErr, s)
 	panic(s)
 }
-func (logger *Logger) Panicf(format string, v ...interface{}) {
+func (logger *Logger)Panicf (format string, v ...interface{}) {
 	s := fmt.Sprintf(format, v...)
 	logger.Log(PriErr, s)
 	panic(s)
 }
 
-func (logger *Logger) Println(v ...interface{}) {
+func (logger *Logger)Println (v ...interface{}) {
 	logger.Log(PriInfo, v...)
 }
-func (logger *Logger) Printf(format string, v ...interface{}) {
+func (logger *Logger)Printf (format string, v ...interface{}) {
 	logger.Logf(PriInfo, format, v...)
 }
 
-func Fatalln(v ...interface{}) {
+
+func Fatalln (v ...interface{}) {
 	defaultLogger.Log(PriCrit, v...)
 	os.Exit(1)
 }
-func Fatalf(format string, v ...interface{}) {
+func Fatalf (format string, v ...interface{}) {
 	defaultLogger.Logf(PriCrit, format, v...)
 	os.Exit(1)
 }
 
-func Panicln(v ...interface{}) {
+func Panicln (v ...interface{}) {
 	s := fmt.Sprint(v...)
 	defaultLogger.Log(PriErr, s)
 	panic(s)
 }
-func Panicf(format string, v ...interface{}) {
+func Panicf (format string, v ...interface{}) {
 	s := fmt.Sprintf(format, v...)
 	defaultLogger.Log(PriErr, s)
 	panic(s)
 }
 
-func Println(v ...interface{}) {
+func Println (v ...interface{}) {
 	defaultLogger.Log(PriInfo, v...)
 }
-func Printf(format string, v ...interface{}) {
+func Printf (format string, v ...interface{}) {
 	defaultLogger.Logf(PriInfo, format, v...)
 }

+ 0 - 1
third_party/github.com/coreos/go-log/log/fields.go

@@ -1,5 +1,4 @@
 package log
-
 // Copyright 2013, CoreOS, Inc. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

+ 0 - 1
third_party/github.com/coreos/go-log/log/logger.go

@@ -1,5 +1,4 @@
 package log
-
 // Copyright 2013, CoreOS, Inc. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

+ 0 - 1
third_party/github.com/coreos/go-log/log/priority.go

@@ -1,5 +1,4 @@
 package log
-
 // Copyright 2013, CoreOS, Inc. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

+ 0 - 1
third_party/github.com/coreos/go-log/log/sinks.go

@@ -1,5 +1,4 @@
 package log
-
 // Copyright 2013, CoreOS, Inc. All rights reserved.
 //
 // Licensed under the Apache License, Version 2.0 (the "License");

+ 1 - 1
third_party/github.com/coreos/go-systemd/activation/files.go

@@ -24,7 +24,7 @@ func Files() []*os.File {
 	files := []*os.File(nil)
 	for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ {
 		syscall.CloseOnExec(fd)
-		files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_"+strconv.Itoa(fd)))
+		files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd)))
 	}
 	return files
 }

+ 465 - 0
third_party/github.com/stretchr/testify/assert/assertions.go

@@ -0,0 +1,465 @@
+package assert
+
+import (
+	"fmt"
+	"reflect"
+	"runtime"
+	"strings"
+	"testing"
+	"time"
+)
+
+// Comparison a custom function that returns true on success and false on failure
+type Comparison func() (success bool)
+
+/*
+	Helper functions
+*/
+
+// ObjectsAreEqual determines if two objects are considered equal.
+//
+// This function does no assertion of any kind.
+func ObjectsAreEqual(a, b interface{}) bool {
+
+	if reflect.DeepEqual(a, b) {
+		return true
+	}
+
+	if reflect.ValueOf(a) == reflect.ValueOf(b) {
+		return true
+	}
+
+	// Last ditch effort
+	if fmt.Sprintf("%#v", a) == fmt.Sprintf("%#v", b) {
+		return true
+	}
+
+	return false
+
+}
+
+/* CallerInfo is necessary because the assert functions use the testing object
+internally, causing it to print the file:line of the assert method, rather than where
+the problem actually occured in calling code.*/
+
+// CallerInfo returns a string containing the file and line number of the assert call
+// that failed.
+func CallerInfo() string {
+
+	file := ""
+	line := 0
+	ok := false
+
+	for i := 0; ; i++ {
+		_, file, line, ok = runtime.Caller(i)
+		if !ok {
+			return ""
+		}
+		parts := strings.Split(file, "/")
+		dir := parts[len(parts)-2]
+		file = parts[len(parts)-1]
+		if (dir != "assert" && dir != "mock") || file == "mock_test.go" {
+			break
+		}
+	}
+
+	return fmt.Sprintf("%s:%d", file, line)
+}
+
+// getWhitespaceString returns a string that is long enough to overwrite the default
+// output from the go testing framework.
+func getWhitespaceString() string {
+
+	_, file, line, ok := runtime.Caller(1)
+	if !ok {
+		return ""
+	}
+	parts := strings.Split(file, "/")
+	file = parts[len(parts)-1]
+
+	return strings.Repeat(" ", len(fmt.Sprintf("%s:%d:      ", file, line)))
+
+}
+
+func messageFromMsgAndArgs(msgAndArgs ...interface{}) string {
+	if len(msgAndArgs) == 0 || msgAndArgs == nil {
+		return ""
+	}
+	if len(msgAndArgs) == 1 {
+		return msgAndArgs[0].(string)
+	}
+	if len(msgAndArgs) > 1 {
+		return fmt.Sprintf(msgAndArgs[0].(string), msgAndArgs[1:]...)
+	}
+	return ""
+}
+
+// Fail reports a failure through
+func Fail(t *testing.T, failureMessage string, msgAndArgs ...interface{}) bool {
+
+	message := messageFromMsgAndArgs(msgAndArgs...)
+
+	if len(message) > 0 {
+		t.Errorf("\r%s\r\tLocation:\t%s\n\r\tError:\t\t%s\n\r\tMessages:\t%s\n\r", getWhitespaceString(), CallerInfo(), failureMessage, message)
+	} else {
+		t.Errorf("\r%s\r\tLocation:\t%s\n\r\tError:\t\t%s\n\r", getWhitespaceString(), CallerInfo(), failureMessage)
+	}
+
+	return false
+}
+
+// Implements asserts that an object is implemented by the specified interface.
+//
+//    assert.Implements(t, (*MyInterface)(nil), new(MyObject), "MyObject")
+func Implements(t *testing.T, interfaceObject interface{}, object interface{}, msgAndArgs ...interface{}) bool {
+
+	interfaceType := reflect.TypeOf(interfaceObject).Elem()
+
+	if !reflect.TypeOf(object).Implements(interfaceType) {
+		return Fail(t, fmt.Sprintf("Object must implement %v", interfaceType), msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// IsType asserts that the specified objects are of the same type.
+func IsType(t *testing.T, expectedType interface{}, object interface{}, msgAndArgs ...interface{}) bool {
+
+	if !ObjectsAreEqual(reflect.TypeOf(object), reflect.TypeOf(expectedType)) {
+		return Fail(t, fmt.Sprintf("Object expected to be of type %v, but was %v", reflect.TypeOf(expectedType), reflect.TypeOf(object)), msgAndArgs...)
+	}
+
+	return true
+}
+
+// Equal asserts that two objects are equal.
+//
+//    assert.Equal(t, 123, 123, "123 and 123 should be equal")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Equal(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool {
+
+	if !ObjectsAreEqual(a, b) {
+		return Fail(t, fmt.Sprintf("Not equal: %#v != %#v", a, b), msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// Exactly asserts that two objects are equal is value and type.
+//
+//    assert.Exactly(t, int32(123), int64(123), "123 and 123 should NOT be equal")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Exactly(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool {
+
+	aType := reflect.TypeOf(a)
+	bType := reflect.TypeOf(b)
+
+	if aType != bType {
+		return Fail(t, "Types expected to match exactly", "%v != %v", aType, bType)
+	}
+
+	return Equal(t, a, b, msgAndArgs...)
+
+}
+
+// NotNil asserts that the specified object is not nil.
+//
+//    assert.NotNil(t, err, "err should be something")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NotNil(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool {
+
+	var success bool = true
+
+	if object == nil {
+		success = false
+	} else {
+		value := reflect.ValueOf(object)
+		kind := value.Kind()
+		if kind >= reflect.Chan && kind <= reflect.Slice && value.IsNil() {
+			success = false
+		}
+	}
+
+	if !success {
+		Fail(t, "Expected not to be nil.", msgAndArgs...)
+	}
+
+	return success
+}
+
+// Nil asserts that the specified object is nil.
+//
+//    assert.Nil(t, err, "err should be nothing")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Nil(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool {
+
+	if object == nil {
+		return true
+	} else {
+		value := reflect.ValueOf(object)
+		kind := value.Kind()
+		if kind >= reflect.Chan && kind <= reflect.Slice && value.IsNil() {
+			return true
+		}
+	}
+
+	return Fail(t, fmt.Sprintf("Expected nil, but got: %#v", object), msgAndArgs...)
+}
+
+// isEmpty gets whether the specified object is considered empty or not.
+func isEmpty(object interface{}) bool {
+
+	if object == nil {
+		return true
+	} else if object == "" {
+		return true
+	} else if object == 0 {
+		return true
+	} else if object == false {
+		return true
+	}
+
+	objValue := reflect.ValueOf(object)
+	switch objValue.Kind() {
+	case reflect.Slice:
+		{
+			return (objValue.Len() == 0)
+		}
+	}
+
+	return false
+
+}
+
+// Empty asserts that the specified object is empty.  I.e. nil, "", false, 0 or a
+// slice with len == 0.
+//
+// assert.Empty(t, obj)
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Empty(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool {
+
+	pass := isEmpty(object)
+	if !pass {
+		Fail(t, fmt.Sprintf("Should be empty, but was %v", object), msgAndArgs...)
+	}
+
+	return pass
+
+}
+
+// Empty asserts that the specified object is NOT empty.  I.e. not nil, "", false, 0 or a
+// slice with len == 0.
+//
+// if assert.NotEmpty(t, obj) {
+//   assert.Equal(t, "two", obj[1])
+// }
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NotEmpty(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool {
+
+	pass := !isEmpty(object)
+	if !pass {
+		Fail(t, fmt.Sprintf("Should NOT be empty, but was %v", object), msgAndArgs...)
+	}
+
+	return pass
+
+}
+
+// True asserts that the specified value is true.
+//
+//    assert.True(t, myBool, "myBool should be true")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func True(t *testing.T, value bool, msgAndArgs ...interface{}) bool {
+
+	if value != true {
+		return Fail(t, "Should be true", msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// False asserts that the specified value is true.
+//
+//    assert.False(t, myBool, "myBool should be false")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func False(t *testing.T, value bool, msgAndArgs ...interface{}) bool {
+
+	if value != false {
+		return Fail(t, "Should be false", msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// NotEqual asserts that the specified values are NOT equal.
+//
+//    assert.NotEqual(t, obj1, obj2, "two objects shouldn't be equal")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NotEqual(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool {
+
+	if ObjectsAreEqual(a, b) {
+		return Fail(t, "Should not be equal", msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// Contains asserts that the specified string contains the specified substring.
+//
+//    assert.Contains(t, "Hello World", "World", "But 'Hello World' does contain 'World'")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Contains(t *testing.T, s, contains string, msgAndArgs ...interface{}) bool {
+
+	if !strings.Contains(s, contains) {
+		return Fail(t, fmt.Sprintf("\"%s\" does not contain \"%s\"", s, contains), msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// NotContains asserts that the specified string does NOT contain the specified substring.
+//
+//    assert.NotContains(t, "Hello World", "Earth", "But 'Hello World' does NOT contain 'Earth'")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NotContains(t *testing.T, s, contains string, msgAndArgs ...interface{}) bool {
+
+	if strings.Contains(s, contains) {
+		return Fail(t, fmt.Sprintf("\"%s\" should not contain \"%s\"", s, contains), msgAndArgs...)
+	}
+
+	return true
+
+}
+
+// Uses a Comparison to assert a complex condition.
+func Condition(t *testing.T, comp Comparison, msgAndArgs ...interface{}) bool {
+	result := comp()
+	if !result {
+		Fail(t, "Condition failed!", msgAndArgs...)
+	}
+	return result
+}
+
+// PanicTestFunc defines a func that should be passed to the assert.Panics and assert.NotPanics
+// methods, and represents a simple func that takes no arguments, and returns nothing.
+type PanicTestFunc func()
+
+// didPanic returns true if the function passed to it panics. Otherwise, it returns false.
+func didPanic(f PanicTestFunc) (bool, interface{}) {
+
+	var didPanic bool = false
+	var message interface{}
+	func() {
+
+		defer func() {
+			if message = recover(); message != nil {
+				didPanic = true
+			}
+		}()
+
+		// call the target function
+		f()
+
+	}()
+
+	return didPanic, message
+
+}
+
+// Panics asserts that the code inside the specified PanicTestFunc panics.
+//
+//   assert.Panics(t, func(){
+//     GoCrazy()
+//   }, "Calling GoCrazy() should panic")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Panics(t *testing.T, f PanicTestFunc, msgAndArgs ...interface{}) bool {
+
+	if funcDidPanic, panicValue := didPanic(f); !funcDidPanic {
+		return Fail(t, fmt.Sprintf("func %#v should panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...)
+	}
+
+	return true
+}
+
+// NotPanics asserts that the code inside the specified PanicTestFunc does NOT panic.
+//
+//   assert.NotPanics(t, func(){
+//     RemainCalm()
+//   }, "Calling RemainCalm() should NOT panic")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NotPanics(t *testing.T, f PanicTestFunc, msgAndArgs ...interface{}) bool {
+
+	if funcDidPanic, panicValue := didPanic(f); funcDidPanic {
+		return Fail(t, fmt.Sprintf("func %#v should not panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...)
+	}
+
+	return true
+}
+
+// WithinDuration asserts that the two times are within duration delta of each other.
+//
+//   assert.WithinDuration(t, time.Now(), time.Now(), 10*time.Second, "The difference should not be more than 10s")
+//
+// Returns whether the assertion was successful (true) or not (false).
+func WithinDuration(t *testing.T, a, b time.Time, delta time.Duration, msgAndArgs ...interface{}) bool {
+
+	dt := a.Sub(b)
+	if dt < -delta || dt > delta {
+		return Fail(t, fmt.Sprintf("Max difference between %v and %v allowed is %v, but difference was %v", a, b, dt, delta), msgAndArgs...)
+	}
+
+	return true
+}
+
+/*
+	Errors
+*/
+
+// NoError asserts that a function returned no error (i.e. `nil`).
+//
+//   actualObj, err := SomeFunction()
+//   if assert.NoError(t, err) {
+//	   assert.Equal(t, actualObj, expectedObj)
+//   }
+//
+// Returns whether the assertion was successful (true) or not (false).
+func NoError(t *testing.T, theError error, msgAndArgs ...interface{}) bool {
+
+	message := messageFromMsgAndArgs(msgAndArgs...)
+	return Nil(t, theError, "No error is expected but got %v %s", theError, message)
+
+}
+
+// Error asserts that a function returned an error (i.e. not `nil`).
+//
+//   actualObj, err := SomeFunction()
+//   if assert.Error(t, err, "An error was expected") {
+//	   assert.Equal(t, err, expectedError)
+//   }
+//
+// Returns whether the assertion was successful (true) or not (false).
+func Error(t *testing.T, theError error, msgAndArgs ...interface{}) bool {
+
+	message := messageFromMsgAndArgs(msgAndArgs...)
+	return NotNil(t, theError, "An error is expected but got nil. %s", message)
+
+}

+ 383 - 0
third_party/github.com/stretchr/testify/assert/assertions_test.go

@@ -0,0 +1,383 @@
+package assert
+
+import (
+	"errors"
+	"testing"
+	"time"
+)
+
+// AssertionTesterInterface defines an interface to be used for testing assertion methods
+type AssertionTesterInterface interface {
+	TestMethod()
+}
+
+// AssertionTesterConformingObject is an object that conforms to the AssertionTesterInterface interface
+type AssertionTesterConformingObject struct {
+}
+
+func (a *AssertionTesterConformingObject) TestMethod() {
+}
+
+// AssertionTesterNonConformingObject is an object that does not conform to the AssertionTesterInterface interface
+type AssertionTesterNonConformingObject struct {
+}
+
+func TestObjectsAreEqual(t *testing.T) {
+
+	if !ObjectsAreEqual("Hello World", "Hello World") {
+		t.Error("objectsAreEqual should return true")
+	}
+	if !ObjectsAreEqual(123, 123) {
+		t.Error("objectsAreEqual should return true")
+	}
+	if !ObjectsAreEqual(123.5, 123.5) {
+		t.Error("objectsAreEqual should return true")
+	}
+	if !ObjectsAreEqual([]byte("Hello World"), []byte("Hello World")) {
+		t.Error("objectsAreEqual should return true")
+	}
+	if !ObjectsAreEqual(nil, nil) {
+		t.Error("objectsAreEqual should return true")
+	}
+
+}
+
+func TestImplements(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !Implements(mockT, (*AssertionTesterInterface)(nil), new(AssertionTesterConformingObject)) {
+		t.Error("Implements method should return true: AssertionTesterConformingObject implements AssertionTesterInterface")
+	}
+	if Implements(mockT, (*AssertionTesterInterface)(nil), new(AssertionTesterNonConformingObject)) {
+		t.Error("Implements method should return false: AssertionTesterNonConformingObject does not implements AssertionTesterInterface")
+	}
+
+}
+
+func TestIsType(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !IsType(mockT, new(AssertionTesterConformingObject), new(AssertionTesterConformingObject)) {
+		t.Error("IsType should return true: AssertionTesterConformingObject is the same type as AssertionTesterConformingObject")
+	}
+	if IsType(mockT, new(AssertionTesterConformingObject), new(AssertionTesterNonConformingObject)) {
+		t.Error("IsType should return false: AssertionTesterConformingObject is not the same type as AssertionTesterNonConformingObject")
+	}
+
+}
+
+func TestEqual(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !Equal(mockT, "Hello World", "Hello World") {
+		t.Error("Equal should return true")
+	}
+	if !Equal(mockT, 123, 123) {
+		t.Error("Equal should return true")
+	}
+	if !Equal(mockT, 123.5, 123.5) {
+		t.Error("Equal should return true")
+	}
+	if !Equal(mockT, []byte("Hello World"), []byte("Hello World")) {
+		t.Error("Equal should return true")
+	}
+	if !Equal(mockT, nil, nil) {
+		t.Error("Equal should return true")
+	}
+
+}
+
+func TestNotNil(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !NotNil(mockT, new(AssertionTesterConformingObject)) {
+		t.Error("NotNil should return true: object is not nil")
+	}
+	if NotNil(mockT, nil) {
+		t.Error("NotNil should return false: object is nil")
+	}
+
+}
+
+func TestNil(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !Nil(mockT, nil) {
+		t.Error("Nil should return true: object is nil")
+	}
+	if Nil(mockT, new(AssertionTesterConformingObject)) {
+		t.Error("Nil should return false: object is not nil")
+	}
+
+}
+
+func TestTrue(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !True(mockT, true) {
+		t.Error("True should return true")
+	}
+	if True(mockT, false) {
+		t.Error("True should return false")
+	}
+
+}
+
+func TestFalse(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !False(mockT, false) {
+		t.Error("False should return true")
+	}
+	if False(mockT, true) {
+		t.Error("False should return false")
+	}
+
+}
+
+func TestExactly(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	a := float32(1)
+	b := float64(1)
+	c := float32(1)
+	d := float32(2)
+
+	if Exactly(mockT, a, b) {
+		t.Error("Exactly should return false")
+	}
+	if Exactly(mockT, a, d) {
+		t.Error("Exactly should return false")
+	}
+	if !Exactly(mockT, a, c) {
+		t.Error("Exactly should return true")
+	}
+
+	if Exactly(mockT, nil, a) {
+		t.Error("Exactly should return false")
+	}
+	if Exactly(mockT, a, nil) {
+		t.Error("Exactly should return false")
+	}
+
+}
+
+func TestNotEqual(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !NotEqual(mockT, "Hello World", "Hello World!") {
+		t.Error("NotEqual should return true")
+	}
+	if !NotEqual(mockT, 123, 1234) {
+		t.Error("NotEqual should return true")
+	}
+	if !NotEqual(mockT, 123.5, 123.55) {
+		t.Error("NotEqual should return true")
+	}
+	if !NotEqual(mockT, []byte("Hello World"), []byte("Hello World!")) {
+		t.Error("NotEqual should return true")
+	}
+	if !NotEqual(mockT, nil, new(AssertionTesterConformingObject)) {
+		t.Error("NotEqual should return true")
+	}
+}
+
+func TestContains(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !Contains(mockT, "Hello World", "Hello") {
+		t.Error("Contains should return true: \"Hello World\" contains \"Hello\"")
+	}
+	if Contains(mockT, "Hello World", "Salut") {
+		t.Error("Contains should return false: \"Hello World\" does not contain \"Salut\"")
+	}
+
+}
+
+func TestNotContains(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !NotContains(mockT, "Hello World", "Hello!") {
+		t.Error("NotContains should return true: \"Hello World\" does not contain \"Hello!\"")
+	}
+	if NotContains(mockT, "Hello World", "Hello") {
+		t.Error("NotContains should return false: \"Hello World\" contains \"Hello\"")
+	}
+
+}
+
+func TestDidPanic(t *testing.T) {
+
+	if funcDidPanic, _ := didPanic(func() {
+		panic("Panic!")
+	}); !funcDidPanic {
+		t.Error("didPanic should return true")
+	}
+
+	if funcDidPanic, _ := didPanic(func() {
+	}); funcDidPanic {
+		t.Error("didPanic should return false")
+	}
+
+}
+
+func TestPanics(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !Panics(mockT, func() {
+		panic("Panic!")
+	}) {
+		t.Error("Panics should return true")
+	}
+
+	if Panics(mockT, func() {
+	}) {
+		t.Error("Panics should return false")
+	}
+
+}
+
+func TestNotPanics(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	if !NotPanics(mockT, func() {
+	}) {
+		t.Error("NotPanics should return true")
+	}
+
+	if NotPanics(mockT, func() {
+		panic("Panic!")
+	}) {
+		t.Error("NotPanics should return false")
+	}
+
+}
+
+func TestEqual_Funcs(t *testing.T) {
+
+	type f func() int
+	var f1 f = func() int { return 1 }
+	var f2 f = func() int { return 2 }
+
+	var f1_copy f = f1
+
+	Equal(t, f1_copy, f1, "Funcs are the same and should be considered equal")
+	NotEqual(t, f1, f2, "f1 and f2 are different")
+
+}
+
+func TestNoError(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	// start with a nil error
+	var err error = nil
+
+	True(t, NoError(mockT, err), "NoError should return True for nil arg")
+
+	// now set an error
+	err = errors.New("Some error")
+
+	False(t, NoError(mockT, err), "NoError with error should return False")
+
+}
+
+func TestError(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	// start with a nil error
+	var err error = nil
+
+	False(t, Error(mockT, err), "Error should return False for nil arg")
+
+	// now set an error
+	err = errors.New("Some error")
+
+	True(t, Error(mockT, err), "Error with error should return True")
+
+}
+
+func Test_isEmpty(t *testing.T) {
+
+	True(t, isEmpty(""))
+	True(t, isEmpty(nil))
+	True(t, isEmpty([]string{}))
+	True(t, isEmpty(0))
+	True(t, isEmpty(false))
+
+	False(t, isEmpty("something"))
+	False(t, isEmpty(errors.New("something")))
+	False(t, isEmpty([]string{"something"}))
+	False(t, isEmpty(1))
+	False(t, isEmpty(true))
+
+}
+
+func TestEmpty(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	True(t, Empty(mockT, ""), "Empty string is empty")
+	True(t, Empty(mockT, nil), "Nil is empty")
+	True(t, Empty(mockT, []string{}), "Empty string array is empty")
+	True(t, Empty(mockT, 0), "Zero int value is empty")
+	True(t, Empty(mockT, false), "False value is empty")
+
+	False(t, Empty(mockT, "something"), "Non Empty string is not empty")
+	False(t, Empty(mockT, errors.New("something")), "Non nil object is not empty")
+	False(t, Empty(mockT, []string{"something"}), "Non empty string array is not empty")
+	False(t, Empty(mockT, 1), "Non-zero int value is not empty")
+	False(t, Empty(mockT, true), "True value is not empty")
+
+}
+
+func TestNotEmpty(t *testing.T) {
+
+	mockT := new(testing.T)
+
+	False(t, NotEmpty(mockT, ""), "Empty string is empty")
+	False(t, NotEmpty(mockT, nil), "Nil is empty")
+	False(t, NotEmpty(mockT, []string{}), "Empty string array is empty")
+	False(t, NotEmpty(mockT, 0), "Zero int value is empty")
+	False(t, NotEmpty(mockT, false), "False value is empty")
+
+	True(t, NotEmpty(mockT, "something"), "Non Empty string is not empty")
+	True(t, NotEmpty(mockT, errors.New("something")), "Non nil object is not empty")
+	True(t, NotEmpty(mockT, []string{"something"}), "Non empty string array is not empty")
+	True(t, NotEmpty(mockT, 1), "Non-zero int value is not empty")
+	True(t, NotEmpty(mockT, true), "True value is not empty")
+
+}
+
+func TestWithinDuration(t *testing.T) {
+
+	mockT := new(testing.T)
+	a := time.Now()
+	b := a.Add(10 * time.Second)
+
+	True(t, WithinDuration(mockT, a, b, 10*time.Second), "A 10s difference is within a 10s time difference")
+	True(t, WithinDuration(mockT, b, a, 10*time.Second), "A 10s difference is within a 10s time difference")
+
+	False(t, WithinDuration(mockT, a, b, 9*time.Second), "A 10s difference is not within a 9s time difference")
+	False(t, WithinDuration(mockT, b, a, 9*time.Second), "A 10s difference is not within a 9s time difference")
+
+	False(t, WithinDuration(mockT, a, b, -9*time.Second), "A 10s difference is not within a 9s time difference")
+	False(t, WithinDuration(mockT, b, a, -9*time.Second), "A 10s difference is not within a 9s time difference")
+
+	False(t, WithinDuration(mockT, a, b, -11*time.Second), "A 10s difference is not within a 9s time difference")
+	False(t, WithinDuration(mockT, b, a, -11*time.Second), "A 10s difference is not within a 9s time difference")
+}

+ 74 - 0
third_party/github.com/stretchr/testify/assert/doc.go

@@ -0,0 +1,74 @@
+// A set of comprehensive testing tools for use with the normal Go testing system.
+//
+// Example Usage
+//
+// The following is a complete example using assert in a standard test function:
+//    import (
+//      "testing"
+//      "github.com/stretchr/testify/assert"
+//    )
+//
+//    func TestSomething(t *testing.T) {
+//
+//      var a string = "Hello"
+//      var b string = "Hello"
+//
+//      assert.Equal(t, a, b, "The two words should be the same.")
+//
+//    }
+//
+// Assertions
+//
+// Assertions allow you to easily write test code, and are global funcs in the `assert` package.
+// All assertion functions take, as the first argument, the `*testing.T` object provided by the
+// testing framework. This allows the assertion funcs to write the failings and other details to
+// the correct place.
+//
+// Every assertion function also takes an optional string message as the final argument,
+// allowing custom error messages to be appended to the message the assertion method outputs.
+//
+// Here is an overview of the assert functions:
+//
+//    assert.Equal(t, expected, actual [, message [, format-args])
+//
+//    assert.NotEqual(t, notExpected, actual [, message [, format-args]])
+//
+//    assert.True(t, actualBool [, message [, format-args]])
+//
+//    assert.False(t, actualBool [, message [, format-args]])
+//
+//    assert.Nil(t, actualObject [, message [, format-args]])
+//
+//    assert.NotNil(t, actualObject [, message [, format-args]])
+//
+//    assert.Empty(t, actualObject [, message [, format-args]])
+//
+//    assert.NotEmpty(t, actualObject [, message [, format-args]])
+//
+//    assert.Error(t, errorObject [, message [, format-args]])
+//
+//    assert.NoError(t, errorObject [, message [, format-args]])
+//
+//    assert.Implements(t, (*MyInterface)(nil), new(MyObject) [,message [, format-args]])
+//
+//    assert.IsType(t, expectedObject, actualObject [, message [, format-args]])
+//
+//    assert.Contains(t, string, substring [, message [, format-args]])
+//
+//    assert.NotContains(t, string, substring [, message [, format-args]])
+//
+//    assert.Panics(t, func(){
+//
+//	    // call code that should panic
+//
+//    } [, message [, format-args]])
+//
+//    assert.NotPanics(t, func(){
+//
+//	    // call code that should not panic
+//
+//    } [, message [, format-args]])
+//
+//    assert.WithinDuration(t, timeA, timeB, deltaTime, [, message [, format-args]])
+
+package assert

+ 10 - 0
third_party/github.com/stretchr/testify/assert/errors.go

@@ -0,0 +1,10 @@
+package assert
+
+import (
+	"errors"
+)
+
+// AnError is an erorr instance useful for testing.  If the code does not care
+// about error specifics, and only needs to return the error for example, this
+// error should be used to make the test code more readable.
+var AnError error = errors.New("assert.AnError general error for testing.")

+ 43 - 0
third_party/github.com/stretchr/testify/mock/doc.go

@@ -0,0 +1,43 @@
+// Provides a system by which it is possible to mock your objects and verify calls are happening as expected.
+//
+// Example Usage
+//
+// The mock package provides an object, Mock, that tracks activity on another object.  It is usually
+// embedded into a test object as shown below:
+//
+//   type MyTestObject struct {
+//     // add a Mock object instance
+//     mock.Mock
+//
+//     // other fields go here as normal
+//   }
+//
+// When implementing the methods of an interface, you wire your functions up
+// to call the Mock.Called(args...) method, and return the appropriate values.
+//
+// For example, to mock a method that saves the name and age of a person and returns
+// the year of their birth or an error, you might write this:
+//
+//     func (o *MyTestObject) SavePersonDetails(firstname, lastname string, age int) (int, error) {
+//       args := o.Mock.Called(firstname, lastname, age)
+//       return args.Int(0), args.Error(1)
+//     }
+//
+// The Int, Error and Bool methods are examples of strongly typed getters that take the argument
+// index position. Given this argument list:
+//
+//     (12, true, "Something")
+//
+// You could read them out strongly typed like this:
+//
+//     args.Int(0)
+//     args.Bool(1)
+//     args.String(2)
+//
+// For objects of your own type, use the generic Arguments.Get(index) method and make a type assertion:
+//
+//     return args.Get(0).(*MyObject), args.Get(1).(*AnotherObjectOfMine)
+//
+// This may cause a panic if the object you are getting is nil (the type assertion will fail), in those
+// cases you should check for nil first.
+package mock

+ 465 - 0
third_party/github.com/stretchr/testify/mock/mock.go

@@ -0,0 +1,465 @@
+package mock
+
+import (
+	"fmt"
+	"github.com/stretchr/objx"
+	"github.com/stretchr/testify/assert"
+	"reflect"
+	"runtime"
+	"strings"
+	"testing"
+)
+
+/*
+	Call
+*/
+
+// Call represents a method call and is used for setting expectations,
+// as well as recording activity.
+type Call struct {
+
+	// The name of the method that was or will be called.
+	Method string
+
+	// Holds the arguments of the method.
+	Arguments Arguments
+
+	// Holds the arguments that should be returned when
+	// this method is called.
+	ReturnArguments Arguments
+}
+
+// Mock is the workhorse used to track activity on another object.
+// For an example of its usage, refer to the "Example Usage" section at the top of this document.
+type Mock struct {
+
+	// The method name that is currently
+	// being referred to by the On method.
+	onMethodName string
+
+	// An array of the arguments that are
+	// currently being referred to by the On method.
+	onMethodArguments Arguments
+
+	// Represents the calls that are expected of
+	// an object.
+	ExpectedCalls []Call
+
+	// Holds the calls that were made to this mocked object.
+	Calls []Call
+
+	// TestData holds any data that might be useful for testing.  Testify ignores
+	// this data completely allowing you to do whatever you like with it.
+	testData objx.Map
+}
+
+// TestData holds any data that might be useful for testing.  Testify ignores
+// this data completely allowing you to do whatever you like with it.
+func (m *Mock) TestData() objx.Map {
+
+	if m.testData == nil {
+		m.testData = make(objx.Map)
+	}
+
+	return m.testData
+}
+
+/*
+	Setting expectations
+*/
+
+// On starts a description of an expectation of the specified method
+// being called.
+//
+//     Mock.On("MyMethod", arg1, arg2)
+func (m *Mock) On(methodName string, arguments ...interface{}) *Mock {
+	m.onMethodName = methodName
+	m.onMethodArguments = arguments
+	return m
+}
+
+// Return finishes a description of an expectation of the method (and arguments)
+// specified in the most recent On method call.
+//
+//     Mock.On("MyMethod", arg1, arg2).Return(returnArg1, returnArg2)
+func (m *Mock) Return(returnArguments ...interface{}) *Mock {
+	m.ExpectedCalls = append(m.ExpectedCalls, Call{m.onMethodName, m.onMethodArguments, returnArguments})
+	return m
+}
+
+/*
+	Recording and responding to activity
+*/
+
+func (m *Mock) findExpectedCall(method string, arguments ...interface{}) (bool, *Call) {
+	for _, call := range m.ExpectedCalls {
+		if call.Method == method {
+
+			_, diffCount := call.Arguments.Diff(arguments)
+			if diffCount == 0 {
+				return true, &call
+			}
+
+		}
+	}
+	return false, nil
+}
+
+func (m *Mock) findClosestCall(method string, arguments ...interface{}) (bool, *Call) {
+
+	diffCount := 0
+	var closestCall *Call = nil
+
+	for _, call := range m.ExpectedCalls {
+		if call.Method == method {
+
+			_, tempDiffCount := call.Arguments.Diff(arguments)
+			if tempDiffCount < diffCount || diffCount == 0 {
+				diffCount = tempDiffCount
+				closestCall = &call
+			}
+
+		}
+	}
+
+	if closestCall == nil {
+		return false, nil
+	}
+
+	return true, closestCall
+}
+
+func callString(method string, arguments Arguments, includeArgumentValues bool) string {
+
+	var argValsString string = ""
+	if includeArgumentValues {
+		var argVals []string
+		for argIndex, arg := range arguments {
+			argVals = append(argVals, fmt.Sprintf("%d: %v", argIndex, arg))
+		}
+		argValsString = fmt.Sprintf("\n\t\t%s", strings.Join(argVals, "\n\t\t"))
+	}
+
+	return fmt.Sprintf("%s(%s)%s", method, arguments.String(), argValsString)
+}
+
+// Called tells the mock object that a method has been called, and gets an array
+// of arguments to return.  Panics if the call is unexpected (i.e. not preceeded by
+// appropriate .On .Return() calls)
+func (m *Mock) Called(arguments ...interface{}) Arguments {
+
+	// get the calling function's name
+	pc, _, _, ok := runtime.Caller(1)
+	if !ok {
+		panic("Couldn't get the caller information")
+	}
+	functionPath := runtime.FuncForPC(pc).Name()
+	parts := strings.Split(functionPath, ".")
+	functionName := parts[len(parts)-1]
+
+	found, call := m.findExpectedCall(functionName, arguments...)
+
+	if !found {
+
+		// we have to fail here - because we don't know what to do
+		// as the return arguments.  This is because:
+		//
+		//   a) this is a totally unexpected call to this method,
+		//   b) the arguments are not what was expected, or
+		//   c) the developer has forgotten to add an accompanying On...Return pair.
+
+		closestFound, closestCall := m.findClosestCall(functionName, arguments...)
+
+		if closestFound {
+			panic(fmt.Sprintf("\n\nmock: Unexpected Method Call\n-----------------------------\n\n%s\n\nThe closest call I have is: \n\n%s\n", callString(functionName, arguments, true), callString(functionName, closestCall.Arguments, true)))
+		} else {
+			panic(fmt.Sprintf("\nassert: mock: I don't know what to return because the method call was unexpected.\n\tEither do Mock.On(\"%s\").Return(...) first, or remove the %s() call.\n\tThis method was unexpected:\n\t\t%s\n\tat: %s", functionName, functionName, callString(functionName, arguments, true), assert.CallerInfo()))
+		}
+
+	}
+
+	// add the call
+	m.Calls = append(m.Calls, Call{functionName, arguments, make([]interface{}, 0)})
+
+	return call.ReturnArguments
+
+}
+
+/*
+	Assertions
+*/
+
+// AssertExpectationsForObjects asserts that everything specified with On and Return
+// of the specified objects was in fact called as expected.
+//
+// Calls may have occurred in any order.
+func AssertExpectationsForObjects(t *testing.T, testObjects ...interface{}) bool {
+	var success bool = true
+	for _, obj := range testObjects {
+		mockObj := obj.(Mock)
+		success = success && mockObj.AssertExpectations(t)
+	}
+	return success
+}
+
+// AssertExpectations asserts that everything specified with On and Return was
+// in fact called as expected.  Calls may have occurred in any order.
+func (m *Mock) AssertExpectations(t *testing.T) bool {
+
+	var somethingMissing bool = false
+	var failedExpectations int = 0
+
+	// iterate through each expectation
+	for _, expectedCall := range m.ExpectedCalls {
+		if !m.methodWasCalled(expectedCall.Method, expectedCall.Arguments) {
+			somethingMissing = true
+			failedExpectations++
+			t.Logf("\u274C\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String())
+		} else {
+			t.Logf("\u2705\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String())
+		}
+	}
+
+	if somethingMissing {
+		t.Errorf("FAIL: %d out of %d expectation(s) were met.\n\tThe code you are testing needs to make %d more call(s).\n\tat: %s", len(m.ExpectedCalls)-failedExpectations, len(m.ExpectedCalls), failedExpectations, assert.CallerInfo())
+	}
+
+	return !somethingMissing
+}
+
+// AssertNumberOfCalls asserts that the method was called expectedCalls times.
+func (m *Mock) AssertNumberOfCalls(t *testing.T, methodName string, expectedCalls int) bool {
+	var actualCalls int = 0
+	for _, call := range m.Calls {
+		if call.Method == methodName {
+			actualCalls++
+		}
+	}
+	return assert.Equal(t, actualCalls, expectedCalls, fmt.Sprintf("Expected number of calls (%d) does not match the actual number of calls (%d).", expectedCalls, actualCalls))
+}
+
+// AssertCalled asserts that the method was called.
+func (m *Mock) AssertCalled(t *testing.T, methodName string, arguments ...interface{}) bool {
+	if !assert.True(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method should have been called with %d argument(s), but was not.", methodName, len(arguments))) {
+		t.Logf("%s", m.ExpectedCalls)
+		return false
+	}
+	return true
+}
+
+// AssertNotCalled asserts that the method was not called.
+func (m *Mock) AssertNotCalled(t *testing.T, methodName string, arguments ...interface{}) bool {
+	if !assert.False(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method was called with %d argument(s), but should NOT have been.", methodName, len(arguments))) {
+		t.Logf("%s", m.ExpectedCalls)
+		return false
+	}
+	return true
+}
+
+func (m *Mock) methodWasCalled(methodName string, arguments []interface{}) bool {
+	for _, call := range m.Calls {
+		if call.Method == methodName {
+
+			_, differences := call.Arguments.Diff(arguments)
+
+			if differences == 0 {
+				// found the expected call
+				return true
+			}
+
+		}
+	}
+	// we didn't find the expected call
+	return false
+}
+
+/*
+	Arguments
+*/
+
+// Arguments holds an array of method arguments or return values.
+type Arguments []interface{}
+
+const (
+	// The "any" argument.  Used in Diff and Assert when
+	// the argument being tested shouldn't be taken into consideration.
+	Anything string = "mock.Anything"
+)
+
+// AnythingOfTypeArgument is a string that contains the type of an argument
+// for use when type checking.  Used in Diff and Assert.
+type AnythingOfTypeArgument string
+
+// AnythingOfType returns an AnythingOfTypeArgument object containing the
+// name of the type to check for.  Used in Diff and Assert.
+//
+// For example:
+//	Assert(t, AnythingOfType("string"), AnythingOfType("int"))
+func AnythingOfType(t string) AnythingOfTypeArgument {
+	return AnythingOfTypeArgument(t)
+}
+
+// Get Returns the argument at the specified index.
+func (args Arguments) Get(index int) interface{} {
+	if index+1 > len(args) {
+		panic(fmt.Sprintf("assert: arguments: Cannot call Get(%d) because there are %d argument(s).", index, len(args)))
+	}
+	return args[index]
+}
+
+// Is gets whether the objects match the arguments specified.
+func (args Arguments) Is(objects ...interface{}) bool {
+	for i, obj := range args {
+		if obj != objects[i] {
+			return false
+		}
+	}
+	return true
+}
+
+// Diff gets a string describing the differences between the arguments
+// and the specified objects.
+//
+// Returns the diff string and number of differences found.
+func (args Arguments) Diff(objects []interface{}) (string, int) {
+
+	var output string = "\n"
+	var differences int
+
+	var maxArgCount int = len(args)
+	if len(objects) > maxArgCount {
+		maxArgCount = len(objects)
+	}
+
+	for i := 0; i < maxArgCount; i++ {
+		var actual, expected interface{}
+
+		if len(args) <= i {
+			actual = "(Missing)"
+		} else {
+			actual = args[i]
+		}
+
+		if len(objects) <= i {
+			expected = "(Missing)"
+		} else {
+			expected = objects[i]
+		}
+
+		if reflect.TypeOf(expected) == reflect.TypeOf((*AnythingOfTypeArgument)(nil)).Elem() {
+
+			// type checking
+			if reflect.TypeOf(actual).Name() != string(expected.(AnythingOfTypeArgument)) {
+				// not match
+				differences++
+				output = fmt.Sprintf("%s\t%d: \u274C  type %s != type %s - %s\n", output, i, expected, reflect.TypeOf(actual).Name(), actual)
+			}
+
+		} else {
+
+			// normal checking
+
+			if assert.ObjectsAreEqual(expected, Anything) || assert.ObjectsAreEqual(actual, Anything) || assert.ObjectsAreEqual(actual, expected) {
+				// match
+				output = fmt.Sprintf("%s\t%d: \u2705  %s == %s\n", output, i, expected, actual)
+			} else {
+				// not match
+				differences++
+				output = fmt.Sprintf("%s\t%d: \u274C  %s != %s\n", output, i, expected, actual)
+			}
+		}
+
+	}
+
+	if differences == 0 {
+		return "No differences.", differences
+	}
+
+	return output, differences
+
+}
+
+// Assert compares the arguments with the specified objects and fails if
+// they do not exactly match.
+func (args Arguments) Assert(t *testing.T, objects ...interface{}) bool {
+
+	// get the differences
+	diff, diffCount := args.Diff(objects)
+
+	if diffCount == 0 {
+		return true
+	}
+
+	// there are differences... report them...
+	t.Logf(diff)
+	t.Errorf("%sArguments do not match.", assert.CallerInfo())
+
+	return false
+
+}
+
+// String gets the argument at the specified index. Panics if there is no argument, or
+// if the argument is of the wrong type.
+//
+// If no index is provided, String() returns a complete string representation
+// of the arguments.
+func (args Arguments) String(indexOrNil ...int) string {
+
+	if len(indexOrNil) == 0 {
+		// normal String() method - return a string representation of the args
+		var argsStr []string
+		for _, arg := range args {
+			argsStr = append(argsStr, fmt.Sprintf("%s", reflect.TypeOf(arg)))
+		}
+		return strings.Join(argsStr, ",")
+	} else if len(indexOrNil) == 1 {
+		// Index has been specified - get the argument at that index
+		var index int = indexOrNil[0]
+		var s string
+		var ok bool
+		if s, ok = args.Get(index).(string); !ok {
+			panic(fmt.Sprintf("assert: arguments: String(%d) failed because object wasn't correct type: %s", index, args.Get(index)))
+		}
+		return s
+	}
+
+	panic(fmt.Sprintf("assert: arguments: Wrong number of arguments passed to String.  Must be 0 or 1, not %d", len(indexOrNil)))
+
+}
+
+// Int gets the argument at the specified index. Panics if there is no argument, or
+// if the argument is of the wrong type.
+func (args Arguments) Int(index int) int {
+	var s int
+	var ok bool
+	if s, ok = args.Get(index).(int); !ok {
+		panic(fmt.Sprintf("assert: arguments: Int(%d) failed because object wasn't correct type: %s", index, args.Get(index)))
+	}
+	return s
+}
+
+// Error gets the argument at the specified index. Panics if there is no argument, or
+// if the argument is of the wrong type.
+func (args Arguments) Error(index int) error {
+	obj := args.Get(index)
+	var s error
+	var ok bool
+	if obj == nil {
+		return nil
+	}
+	if s, ok = obj.(error); !ok {
+		panic(fmt.Sprintf("assert: arguments: Error(%d) failed because object wasn't correct type: %s", index, args.Get(index)))
+	}
+	return s
+}
+
+// Bool gets the argument at the specified index. Panics if there is no argument, or
+// if the argument is of the wrong type.
+func (args Arguments) Bool(index int) bool {
+	var s bool
+	var ok bool
+	if s, ok = args.Get(index).(bool); !ok {
+		panic(fmt.Sprintf("assert: arguments: Bool(%d) failed because object wasn't correct type: %s", index, args.Get(index)))
+	}
+	return s
+}

+ 418 - 0
third_party/github.com/stretchr/testify/mock/mock_test.go

@@ -0,0 +1,418 @@
+package mock
+
+import (
+	"errors"
+	"github.com/stretchr/testify/assert"
+	"testing"
+)
+
+/*
+	Test objects
+*/
+
+// ExampleInterface represents an example interface.
+type ExampleInterface interface {
+	TheExampleMethod(a, b, c int) (int, error)
+}
+
+// TestExampleImplementation is a test implementation of ExampleInterface
+type TestExampleImplementation struct {
+	Mock
+}
+
+func (i *TestExampleImplementation) TheExampleMethod(a, b, c int) (int, error) {
+	args := i.Mock.Called(a, b, c)
+	return args.Int(0), args.Error(1)
+}
+
+func (i *TestExampleImplementation) TheExampleMethod2(yesorno bool) {
+	i.Mock.Called(yesorno)
+}
+
+/*
+	Mock
+*/
+
+func Test_Mock_TestData(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	if assert.NotNil(t, mockedService.TestData()) {
+
+		mockedService.TestData().Set("something", 123)
+		assert.Equal(t, 123, mockedService.TestData().Get("something").Data())
+
+	}
+
+}
+
+func Test_Mock_On(t *testing.T) {
+
+	// make a test impl object
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	assert.Equal(t, mockedService.Mock.On("TheExampleMethod"), &mockedService.Mock)
+	assert.Equal(t, "TheExampleMethod", mockedService.Mock.onMethodName)
+
+}
+
+func Test_Mock_On_WithArgs(t *testing.T) {
+
+	// make a test impl object
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	assert.Equal(t, mockedService.Mock.On("TheExampleMethod", 1, 2, 3), &mockedService.Mock)
+	assert.Equal(t, "TheExampleMethod", mockedService.Mock.onMethodName)
+	assert.Equal(t, 1, mockedService.Mock.onMethodArguments[0])
+	assert.Equal(t, 2, mockedService.Mock.onMethodArguments[1])
+	assert.Equal(t, 3, mockedService.Mock.onMethodArguments[2])
+
+}
+
+func Test_Mock_Return(t *testing.T) {
+
+	// make a test impl object
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	assert.Equal(t, mockedService.Mock.On("TheExampleMethod", "A", "B", true).Return(1, "two", true), &mockedService.Mock)
+
+	// ensure the call was created
+	if assert.Equal(t, 1, len(mockedService.Mock.ExpectedCalls)) {
+		call := mockedService.Mock.ExpectedCalls[0]
+
+		assert.Equal(t, "TheExampleMethod", call.Method)
+		assert.Equal(t, "A", call.Arguments[0])
+		assert.Equal(t, "B", call.Arguments[1])
+		assert.Equal(t, true, call.Arguments[2])
+		assert.Equal(t, 1, call.ReturnArguments[0])
+		assert.Equal(t, "two", call.ReturnArguments[1])
+		assert.Equal(t, true, call.ReturnArguments[2])
+
+	}
+
+}
+
+func Test_Mock_Return_Nothing(t *testing.T) {
+
+	// make a test impl object
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	assert.Equal(t, mockedService.Mock.On("TheExampleMethod", "A", "B", true).Return(), &mockedService.Mock)
+
+	// ensure the call was created
+	if assert.Equal(t, 1, len(mockedService.Mock.ExpectedCalls)) {
+		call := mockedService.Mock.ExpectedCalls[0]
+
+		assert.Equal(t, "TheExampleMethod", call.Method)
+		assert.Equal(t, "A", call.Arguments[0])
+		assert.Equal(t, "B", call.Arguments[1])
+		assert.Equal(t, true, call.Arguments[2])
+		assert.Equal(t, 0, len(call.ReturnArguments))
+
+	}
+
+}
+
+func Test_Mock_findExpectedCall(t *testing.T) {
+
+	m := new(Mock)
+	m.On("One", 1).Return("one")
+	m.On("Two", 2).Return("two")
+	m.On("Two", 3).Return("three")
+
+	f, c := m.findExpectedCall("Two", 3)
+
+	if assert.True(t, f) {
+		if assert.NotNil(t, c) {
+			assert.Equal(t, "Two", c.Method)
+			assert.Equal(t, 3, c.Arguments[0])
+			assert.Equal(t, "three", c.ReturnArguments[0])
+		}
+	}
+
+}
+
+func Test_callString(t *testing.T) {
+
+	assert.Equal(t, `Method(int,bool,string)`, callString("Method", []interface{}{1, true, "something"}, false))
+
+}
+
+func Test_Mock_Called(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_Called", 1, 2, 3).Return(5, "6", true)
+
+	returnArguments := mockedService.Mock.Called(1, 2, 3)
+
+	if assert.Equal(t, 1, len(mockedService.Mock.Calls)) {
+		assert.Equal(t, "Test_Mock_Called", mockedService.Mock.Calls[0].Method)
+		assert.Equal(t, 1, mockedService.Mock.Calls[0].Arguments[0])
+		assert.Equal(t, 2, mockedService.Mock.Calls[0].Arguments[1])
+		assert.Equal(t, 3, mockedService.Mock.Calls[0].Arguments[2])
+	}
+
+	if assert.Equal(t, 3, len(returnArguments)) {
+		assert.Equal(t, 5, returnArguments[0])
+		assert.Equal(t, "6", returnArguments[1])
+		assert.Equal(t, true, returnArguments[2])
+	}
+
+}
+
+func Test_Mock_Called_Unexpected(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	// make sure it panics if no expectation was made
+	assert.Panics(t, func() {
+		mockedService.Mock.Called(1, 2, 3)
+	}, "Calling unexpected method should panic")
+
+}
+
+func Test_AssertExpectationsForObjects_Helper(t *testing.T) {
+
+	var mockedService1 *TestExampleImplementation = new(TestExampleImplementation)
+	var mockedService2 *TestExampleImplementation = new(TestExampleImplementation)
+	var mockedService3 *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService1.Mock.On("Test_AssertExpectationsForObjects_Helper", 1).Return()
+	mockedService2.Mock.On("Test_AssertExpectationsForObjects_Helper", 2).Return()
+	mockedService3.Mock.On("Test_AssertExpectationsForObjects_Helper", 3).Return()
+
+	mockedService1.Called(1)
+	mockedService2.Called(2)
+	mockedService3.Called(3)
+
+	assert.True(t, AssertExpectationsForObjects(t, mockedService1.Mock, mockedService2.Mock, mockedService3.Mock))
+
+}
+
+func Test_AssertExpectationsForObjects_Helper_Failed(t *testing.T) {
+
+	var mockedService1 *TestExampleImplementation = new(TestExampleImplementation)
+	var mockedService2 *TestExampleImplementation = new(TestExampleImplementation)
+	var mockedService3 *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService1.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 1).Return()
+	mockedService2.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 2).Return()
+	mockedService3.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 3).Return()
+
+	mockedService1.Called(1)
+	mockedService3.Called(3)
+
+	tt := new(testing.T)
+	assert.False(t, AssertExpectationsForObjects(tt, mockedService1.Mock, mockedService2.Mock, mockedService3.Mock))
+
+}
+
+func Test_Mock_AssertExpectations(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_AssertExpectations", 1, 2, 3).Return(5, 6, 7)
+
+	tt := new(testing.T)
+	assert.False(t, mockedService.AssertExpectations(tt))
+
+	// make the call now
+	mockedService.Mock.Called(1, 2, 3)
+
+	// now assert expectations
+	assert.True(t, mockedService.AssertExpectations(tt))
+
+}
+
+func Test_Mock_AssertNumberOfCalls(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_AssertNumberOfCalls", 1, 2, 3).Return(5, 6, 7)
+
+	mockedService.Mock.Called(1, 2, 3)
+	assert.True(t, mockedService.AssertNumberOfCalls(t, "Test_Mock_AssertNumberOfCalls", 1))
+
+	mockedService.Mock.Called(1, 2, 3)
+	assert.True(t, mockedService.AssertNumberOfCalls(t, "Test_Mock_AssertNumberOfCalls", 2))
+
+}
+
+func Test_Mock_AssertCalled(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_AssertCalled", 1, 2, 3).Return(5, 6, 7)
+
+	mockedService.Mock.Called(1, 2, 3)
+
+	assert.True(t, mockedService.AssertCalled(t, "Test_Mock_AssertCalled", 1, 2, 3))
+
+}
+
+func Test_Mock_AssertCalled_WithArguments(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_AssertCalled_WithArguments", 1, 2, 3).Return(5, 6, 7)
+
+	mockedService.Mock.Called(1, 2, 3)
+
+	tt := new(testing.T)
+	assert.True(t, mockedService.AssertCalled(tt, "Test_Mock_AssertCalled_WithArguments", 1, 2, 3))
+	assert.False(t, mockedService.AssertCalled(tt, "Test_Mock_AssertCalled_WithArguments", 2, 3, 4))
+
+}
+
+func Test_Mock_AssertNotCalled(t *testing.T) {
+
+	var mockedService *TestExampleImplementation = new(TestExampleImplementation)
+
+	mockedService.Mock.On("Test_Mock_AssertNotCalled", 1, 2, 3).Return(5, 6, 7)
+
+	mockedService.Mock.Called(1, 2, 3)
+
+	assert.True(t, mockedService.AssertNotCalled(t, "Test_Mock_NotCalled"))
+
+}
+
+/*
+	Arguments helper methods
+*/
+func Test_Arguments_Get(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+
+	assert.Equal(t, "string", args.Get(0).(string))
+	assert.Equal(t, 123, args.Get(1).(int))
+	assert.Equal(t, true, args.Get(2).(bool))
+
+}
+
+func Test_Arguments_Is(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+
+	assert.True(t, args.Is("string", 123, true))
+	assert.False(t, args.Is("wrong", 456, false))
+
+}
+
+func Test_Arguments_Diff(t *testing.T) {
+
+	var args Arguments = []interface{}{"Hello World", 123, true}
+	var diff string
+	var count int
+	diff, count = args.Diff([]interface{}{"Hello World", 456, "false"})
+
+	assert.Equal(t, 2, count)
+	assert.Contains(t, diff, `%!s(int=456) != %!s(int=123)`)
+	assert.Contains(t, diff, `false != %!s(bool=true)`)
+
+}
+
+func Test_Arguments_Diff_DifferentNumberOfArgs(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	var diff string
+	var count int
+	diff, count = args.Diff([]interface{}{"string", 456, "false", "extra"})
+
+	assert.Equal(t, 3, count)
+	assert.Contains(t, diff, `extra != (Missing)`)
+
+}
+
+func Test_Arguments_Diff_WithAnythingArgument(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	var count int
+	_, count = args.Diff([]interface{}{"string", Anything, true})
+
+	assert.Equal(t, 0, count)
+
+}
+
+func Test_Arguments_Diff_WithAnythingArgument_InActualToo(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", Anything, true}
+	var count int
+	_, count = args.Diff([]interface{}{"string", 123, true})
+
+	assert.Equal(t, 0, count)
+
+}
+
+func Test_Arguments_Diff_WithAnythingOfTypeArgument(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	var count int
+	_, count = args.Diff([]interface{}{"string", AnythingOfType("int"), true})
+
+	assert.Equal(t, 0, count)
+
+}
+
+func Test_Arguments_Diff_WithAnythingOfTypeArgument_Failing(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	var count int
+	var diff string
+	diff, count = args.Diff([]interface{}{"string", AnythingOfType("string"), true})
+
+	assert.Equal(t, 1, count)
+	assert.Contains(t, diff, `string != type int - %!s(int=123)`)
+
+}
+
+func Test_Arguments_Assert(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+
+	assert.True(t, args.Assert(t, "string", 123, true))
+
+}
+
+func Test_Arguments_String_Representation(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	assert.Equal(t, `string,int,bool`, args.String())
+
+}
+
+func Test_Arguments_String(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	assert.Equal(t, "string", args.String(0))
+
+}
+
+func Test_Arguments_Error(t *testing.T) {
+
+	var err error = errors.New("An Error")
+	var args Arguments = []interface{}{"string", 123, true, err}
+	assert.Equal(t, err, args.Error(3))
+
+}
+
+func Test_Arguments_Error_Nil(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true, nil}
+	assert.Equal(t, nil, args.Error(3))
+
+}
+
+func Test_Arguments_Int(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	assert.Equal(t, 123, args.Int(1))
+
+}
+
+func Test_Arguments_Bool(t *testing.T) {
+
+	var args Arguments = []interface{}{"string", 123, true}
+	assert.Equal(t, true, args.Bool(2))
+
+}