diff --git a/config/templates.go b/config/templates.go
index f38cbecabc027839baabe26406108e0744741f12..36da48589f4a6b6e16763a04f9e4adec97c8d7fe 100644
--- a/config/templates.go
+++ b/config/templates.go
@@ -86,7 +86,7 @@ const separatorChainConsensus = `
 const sectionChainConsensus = `  [chain.consensus]
   # consensus defines the module to use for consensus and
   # this will define the peer-to-peer consensus network;
-  # accepted values are ("noops", "tmsp",) "tendermint"
+  # accepted values are ("noops", "abci",) "tendermint"
   name = "{{.Name}}"
   # version is the major and minor semantic version;
   # the version will be asserted on
@@ -184,17 +184,17 @@ const separatorModules = `
 `
 
 // TODO: [ben] make configurable
-const sectionTmsp = `
+const sectionabci = `
 ################################################################################
 ##
-## Tendermint Socket Protocol (TMSP)
+## Tendermint Socket Protocol (abci)
 ## version 0.6.0
 ##
-## TMSP expects a tendermint consensus process to run and connect to Eris-DB
+## abci expects a tendermint consensus process to run and connect to Eris-DB
 ##
 ################################################################################
 
-[tmsp]
+[abci]
 # listener address for accepting tendermint socket protocol connections
 listener = "tcp://0.0.0.0:46658"
 
@@ -244,7 +244,7 @@ private_validator_file = "priv_validator.json"
 	# NOTE: value is ignored when run in-process as RPC is
 	# handled by [servers.tendermint]
   rpc_laddr = "0.0.0.0:46657"
-  # proxy application address - used for tmsp connections,
+  # proxy application address - used for abci connections,
   # and this port should not be exposed for in-process Tendermint
   proxy_app = "tcp://127.0.0.1:46658"
 
diff --git a/consensus/config.go b/consensus/config.go
index d676d8c214f8c794aaea43a845260f9407d2d311..951b4cf3e426a2b72ce480622b0f6fd8d8a3fce2 100644
--- a/consensus/config.go
+++ b/consensus/config.go
@@ -22,7 +22,6 @@ package consensus
 import (
 	// noops      "github.com/eris-ltd/eris-db/consensus/noops"
 	tendermint "github.com/eris-ltd/eris-db/consensus/tendermint"
-	tmsp "github.com/eris-ltd/eris-db/consensus/tmsp"
 )
 
 //------------------------------------------------------------------------------
@@ -34,8 +33,6 @@ func AssertValidConsensusModule(name, minorVersionString string) bool {
 		// noops should not have any external interfaces that can change
 		// over iterations
 		return true
-	case "tmsp":
-		return minorVersionString == tmsp.GetTmspVersion().GetMinorVersionString()
 	case "tendermint":
 		return minorVersionString == tendermint.GetTendermintVersion().GetMinorVersionString()
 	case "bigchaindb":
diff --git a/consensus/tendermint/local_client.go b/consensus/tendermint/local_client.go
deleted file mode 100644
index 97b732cac6ab66237ce6250478a6a886855a3b6f..0000000000000000000000000000000000000000
--- a/consensus/tendermint/local_client.go
+++ /dev/null
@@ -1,270 +0,0 @@
-// Copyright 2015, 2016 Eris Industries (UK) Ltd.
-// This file is part of Eris-RT
-
-// Eris-RT is free software: you can redistribute it and/or modify
-// it under the terms of the GNU General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-
-// Eris-RT is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
-// GNU General Public License for more details.
-
-// You should have received a copy of the GNU General Public License
-// along with Eris-RT.  If not, see <http://www.gnu.org/licenses/>.
-
-// version provides the current Eris-DB version and a VersionIdentifier
-// for the modules to identify their version with.
-
-// This file is originally based on github.com/tendermint/tmsp/client/...
-// .../local_client.go
-
-package tendermint
-
-import (
-	"sync"
-
-	tmsp_client "github.com/tendermint/tmsp/client"
-	tmsp_types "github.com/tendermint/tmsp/types"
-
-	manager_types "github.com/eris-ltd/eris-db/manager/types"
-)
-
-// NOTE [ben] Compiler check to ensure localClient successfully implements
-// tendermint/tmsp/client
-var _ tmsp_client.Client = (*localClient)(nil)
-
-type localClient struct {
-	mtx         *sync.Mutex
-	Application manager_types.Application
-	Callback    tmsp_client.Callback
-}
-
-func NewLocalClient(mtx *sync.Mutex, app manager_types.Application) *localClient {
-	if mtx == nil {
-		mtx = new(sync.Mutex)
-	}
-	return &localClient{
-		mtx:         mtx,
-		Application: app,
-	}
-}
-
-func (app *localClient) SetResponseCallback(cb tmsp_client.Callback) {
-	app.mtx.Lock()
-	defer app.mtx.Unlock()
-	app.Callback = cb
-}
-
-// TODO: change manager_types.Application to include Error()?
-func (app *localClient) Error() error {
-	return nil
-}
-
-func (app *localClient) Stop() bool {
-	return true
-}
-
-func (app *localClient) FlushAsync() *tmsp_client.ReqRes {
-	// Do nothing
-	return newLocalReqRes(tmsp_types.ToRequestFlush(), nil)
-}
-
-func (app *localClient) EchoAsync(msg string) *tmsp_client.ReqRes {
-	return app.callback(
-		tmsp_types.ToRequestEcho(msg),
-		tmsp_types.ToResponseEcho(msg),
-	)
-}
-
-func (app *localClient) InfoAsync() *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	info := app.Application.Info()
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestInfo(),
-		tmsp_types.ToResponseInfo(info),
-	)
-}
-
-func (app *localClient) SetOptionAsync(key string, value string) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	log := app.Application.SetOption(key, value)
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestSetOption(key, value),
-		tmsp_types.ToResponseSetOption(log),
-	)
-}
-
-func (app *localClient) AppendTxAsync(tx []byte) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	res := app.Application.AppendTx(tx)
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestAppendTx(tx),
-		tmsp_types.ToResponseAppendTx(res.Code, res.Data, res.Log),
-	)
-}
-
-func (app *localClient) CheckTxAsync(tx []byte) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	res := app.Application.CheckTx(tx)
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestCheckTx(tx),
-		tmsp_types.ToResponseCheckTx(res.Code, res.Data, res.Log),
-	)
-}
-
-func (app *localClient) QueryAsync(tx []byte) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	res := app.Application.Query(tx)
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestQuery(tx),
-		tmsp_types.ToResponseQuery(res.Code, res.Data, res.Log),
-	)
-}
-
-func (app *localClient) CommitAsync() *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	res := app.Application.Commit()
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestCommit(),
-		tmsp_types.ToResponseCommit(res.Code, res.Data, res.Log),
-	)
-}
-
-func (app *localClient) InitChainAsync(validators []*tmsp_types.Validator) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		bcApp.InitChain(validators)
-	}
-	reqRes := app.callback(
-		tmsp_types.ToRequestInitChain(validators),
-		tmsp_types.ToResponseInitChain(),
-	)
-	app.mtx.Unlock()
-	return reqRes
-}
-
-func (app *localClient) BeginBlockAsync(height uint64) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		bcApp.BeginBlock(height)
-	}
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestBeginBlock(height),
-		tmsp_types.ToResponseBeginBlock(),
-	)
-}
-
-func (app *localClient) EndBlockAsync(height uint64) *tmsp_client.ReqRes {
-	app.mtx.Lock()
-	var validators []*tmsp_types.Validator
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		validators = bcApp.EndBlock(height)
-	}
-	app.mtx.Unlock()
-	return app.callback(
-		tmsp_types.ToRequestEndBlock(height),
-		tmsp_types.ToResponseEndBlock(validators),
-	)
-}
-
-//-------------------------------------------------------
-
-func (app *localClient) FlushSync() error {
-	return nil
-}
-
-func (app *localClient) EchoSync(msg string) (res tmsp_types.Result) {
-	return tmsp_types.OK.SetData([]byte(msg))
-}
-
-func (app *localClient) InfoSync() (res tmsp_types.Result) {
-	app.mtx.Lock()
-	info := app.Application.Info()
-	app.mtx.Unlock()
-	return tmsp_types.OK.SetData([]byte(info))
-}
-
-func (app *localClient) SetOptionSync(key string, value string) (res tmsp_types.Result) {
-	app.mtx.Lock()
-	log := app.Application.SetOption(key, value)
-	app.mtx.Unlock()
-	return tmsp_types.OK.SetLog(log)
-}
-
-func (app *localClient) AppendTxSync(tx []byte) (res tmsp_types.Result) {
-	app.mtx.Lock()
-	res = app.Application.AppendTx(tx)
-	app.mtx.Unlock()
-	return res
-}
-
-func (app *localClient) CheckTxSync(tx []byte) (res tmsp_types.Result) {
-	app.mtx.Lock()
-	res = app.Application.CheckTx(tx)
-	app.mtx.Unlock()
-	return res
-}
-
-func (app *localClient) QuerySync(query []byte) (res tmsp_types.Result) {
-	app.mtx.Lock()
-	res = app.Application.Query(query)
-	app.mtx.Unlock()
-	return res
-}
-
-func (app *localClient) CommitSync() (res tmsp_types.Result) {
-	app.mtx.Lock()
-	res = app.Application.Commit()
-	app.mtx.Unlock()
-	return res
-}
-
-func (app *localClient) InitChainSync(validators []*tmsp_types.Validator) (err error) {
-	app.mtx.Lock()
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		bcApp.InitChain(validators)
-	}
-	app.mtx.Unlock()
-	return nil
-}
-
-func (app *localClient) BeginBlockSync(height uint64) (err error) {
-	app.mtx.Lock()
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		bcApp.BeginBlock(height)
-	}
-	app.mtx.Unlock()
-	return nil
-}
-
-func (app *localClient) EndBlockSync(height uint64) (changedValidators []*tmsp_types.Validator, err error) {
-	app.mtx.Lock()
-	if bcApp, ok := app.Application.(tmsp_types.BlockchainAware); ok {
-		changedValidators = bcApp.EndBlock(height)
-	}
-	app.mtx.Unlock()
-	return changedValidators, nil
-}
-
-//-------------------------------------------------------
-
-func (app *localClient) callback(req *tmsp_types.Request, res *tmsp_types.Response) *tmsp_client.ReqRes {
-	app.Callback(req, res)
-	return newLocalReqRes(req, res)
-}
-
-func newLocalReqRes(req *tmsp_types.Request, res *tmsp_types.Response) *tmsp_client.ReqRes {
-	reqRes := tmsp_client.NewReqRes(req)
-	reqRes.Response = res
-	reqRes.SetDone()
-	return reqRes
-}
diff --git a/consensus/tendermint/tendermint.go b/consensus/tendermint/tendermint.go
index 5adc08e1a2619723123088901bea595f75a300eb..c0c3b3a7b4d1bb606d93ed5d9a91542ccdcc6848 100644
--- a/consensus/tendermint/tendermint.go
+++ b/consensus/tendermint/tendermint.go
@@ -23,15 +23,14 @@ import (
 	"fmt"
 	"path"
 	"strings"
-	"sync"
 
+	abci_types "github.com/tendermint/abci/types"
 	crypto "github.com/tendermint/go-crypto"
 	p2p "github.com/tendermint/go-p2p"
 	tendermint_consensus "github.com/tendermint/tendermint/consensus"
 	node "github.com/tendermint/tendermint/node"
 	proxy "github.com/tendermint/tendermint/proxy"
 	tendermint_types "github.com/tendermint/tendermint/types"
-	tmsp_types "github.com/tendermint/tmsp/types"
 
 	edb_event "github.com/eris-ltd/eris-db/event"
 
@@ -125,10 +124,8 @@ func NewTendermint(moduleConfig *config.ModuleConfig,
 		tmintConfig.Set("rpc_laddr", "")
 	}
 
-	newNode := node.NewNode(tmintConfig, privateValidator, func(_, _ string,
-		hash []byte) proxy.AppConn {
-		return NewLocalClient(new(sync.Mutex), application)
-	})
+	newNode := node.NewNode(tmintConfig, privateValidator,
+		proxy.NewLocalClientCreator(application))
 
 	listener := p2p.NewDefaultListener("tcp", tmintConfig.GetString("node_laddr"),
 		tmintConfig.GetBool("skip_upnp"))
@@ -235,7 +232,7 @@ func (tendermint *Tendermint) Events() edb_event.EventEmitter {
 }
 
 func (tendermint *Tendermint) BroadcastTransaction(transaction []byte,
-	callback func(*tmsp_types.Response)) error {
+	callback func(*abci_types.Response)) error {
 	return tendermint.tmintNode.MempoolReactor().BroadcastTx(transaction, callback)
 }
 
diff --git a/consensus/tendermint/version.go b/consensus/tendermint/version.go
index 870d1313bf43977a8976f0415d07f1aa5d1f2b2d..cc7f1807584c759399b760b27fbd61b8a25e3965 100644
--- a/consensus/tendermint/version.go
+++ b/consensus/tendermint/version.go
@@ -30,7 +30,7 @@ const (
 	// Major version component of the current release
 	tendermintVersionMajorConst uint8 = 0
 	// Minor version component of the current release
-	tendermintVersionMinorConst uint8 = 6
+	tendermintVersionMinorConst uint8 = 8
 	// Patch version component of the current release
 	tendermintVersionPatchConst uint8 = 0
 )
diff --git a/consensus/tmsp/server.go b/consensus/tmsp/server.go
deleted file mode 100644
index 62abb9d641d747b748972e02e1b1d9b77ebacf47..0000000000000000000000000000000000000000
--- a/consensus/tmsp/server.go
+++ /dev/null
@@ -1,211 +0,0 @@
-// Copyright 2015, 2016 Eris Industries (UK) Ltd.
-// This file is part of Eris-RT
-
-// Eris-RT is free software: you can redistribute it and/or modify
-// it under the terms of the GNU General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-
-// Eris-RT is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
-// GNU General Public License for more details.
-
-// You should have received a copy of the GNU General Public License
-// along with Eris-RT.  If not, see <http://www.gnu.org/licenses/>.
-
-// Taken originally from github.com/tendermint/tmsp/server.go
-
-package tmsp
-
-import (
-	"bufio"
-	"fmt"
-	"io"
-	"net"
-	"strings"
-	"sync"
-
-	. "github.com/tendermint/go-common"
-	tmsp_types "github.com/tendermint/tmsp/types"
-
-	manager_types "github.com/eris-ltd/eris-db/manager/types"
-)
-
-// var maxNumberConnections = 2
-
-type Server struct {
-	QuitService
-
-	proto    string
-	addr     string
-	listener net.Listener
-
-	appMtx sync.Mutex
-	app    manager_types.Application
-}
-
-func NewServer(protoAddr string, app manager_types.Application) (*Server, error) {
-	parts := strings.SplitN(protoAddr, "://", 2)
-	proto, addr := parts[0], parts[1]
-	s := &Server{
-		proto:    proto,
-		addr:     addr,
-		listener: nil,
-		app:      app,
-	}
-	s.QuitService = *NewQuitService(nil, "TMSPServer", s)
-	_, err := s.Start() // Just start it
-	return s, err
-}
-
-func (s *Server) OnStart() error {
-	s.QuitService.OnStart()
-	ln, err := net.Listen(s.proto, s.addr)
-	if err != nil {
-		return err
-	}
-	s.listener = ln
-	go s.acceptConnectionsRoutine()
-	return nil
-}
-
-func (s *Server) OnStop() {
-	s.QuitService.OnStop()
-	s.listener.Close()
-}
-
-func (s *Server) acceptConnectionsRoutine() {
-	// semaphore := make(chan struct{}, maxNumberConnections)
-
-	for {
-		// semaphore <- struct{}{}
-
-		// Accept a connection
-		fmt.Println("Waiting for new connection...")
-		conn, err := s.listener.Accept()
-		if err != nil {
-			if !s.IsRunning() {
-				return // Ignore error from listener closing.
-			}
-			Exit("Failed to accept connection: " + err.Error())
-		} else {
-			fmt.Println("Accepted a new connection")
-		}
-
-		closeConn := make(chan error, 2)                   // Push to signal connection closed
-		responses := make(chan *tmsp_types.Response, 1000) // A channel to buffer responses
-
-		// Read requests from conn and deal with them
-		go s.handleRequests(closeConn, conn, responses)
-		// Pull responses from 'responses' and write them to conn.
-		go s.handleResponses(closeConn, responses, conn)
-
-		go func() {
-			// Wait until signal to close connection
-			errClose := <-closeConn
-			if errClose != nil {
-				fmt.Printf("Connection error: %v\n", errClose)
-			} else {
-				fmt.Println("Connection was closed.")
-			}
-
-			// Close the connection
-			err := conn.Close()
-			if err != nil {
-				fmt.Printf("Error in closing connection: %v\n", err)
-			}
-
-			// <-semaphore
-		}()
-	}
-}
-
-// Read requests from conn and deal with them
-func (s *Server) handleRequests(closeConn chan error, conn net.Conn, responses chan<- *tmsp_types.Response) {
-	var count int
-	var bufReader = bufio.NewReader(conn)
-	for {
-
-		var req = &tmsp_types.Request{}
-		err := tmsp_types.ReadMessage(bufReader, req)
-		if err != nil {
-			if err == io.EOF {
-				closeConn <- fmt.Errorf("Connection closed by client")
-			} else {
-				closeConn <- fmt.Errorf("Error in handleRequests: %v", err.Error())
-			}
-			return
-		}
-		s.appMtx.Lock()
-		count++
-		s.handleRequest(req, responses)
-		s.appMtx.Unlock()
-	}
-}
-
-func (s *Server) handleRequest(req *tmsp_types.Request, responses chan<- *tmsp_types.Response) {
-	switch r := req.Value.(type) {
-	case *tmsp_types.Request_Echo:
-		responses <- tmsp_types.ToResponseEcho(r.Echo.Message)
-	case *tmsp_types.Request_Flush:
-		responses <- tmsp_types.ToResponseFlush()
-	case *tmsp_types.Request_Info:
-		data := s.app.Info()
-		responses <- tmsp_types.ToResponseInfo(data)
-	case *tmsp_types.Request_SetOption:
-		so := r.SetOption
-		logStr := s.app.SetOption(so.Key, so.Value)
-		responses <- tmsp_types.ToResponseSetOption(logStr)
-	case *tmsp_types.Request_AppendTx:
-		res := s.app.AppendTx(r.AppendTx.Tx)
-		responses <- tmsp_types.ToResponseAppendTx(res.Code, res.Data, res.Log)
-	case *tmsp_types.Request_CheckTx:
-		res := s.app.CheckTx(r.CheckTx.Tx)
-		responses <- tmsp_types.ToResponseCheckTx(res.Code, res.Data, res.Log)
-	case *tmsp_types.Request_Commit:
-		res := s.app.Commit()
-		responses <- tmsp_types.ToResponseCommit(res.Code, res.Data, res.Log)
-	case *tmsp_types.Request_Query:
-		res := s.app.Query(r.Query.Query)
-		responses <- tmsp_types.ToResponseQuery(res.Code, res.Data, res.Log)
-	case *tmsp_types.Request_InitChain:
-		if app, ok := s.app.(tmsp_types.BlockchainAware); ok {
-			app.InitChain(r.InitChain.Validators)
-			responses <- tmsp_types.ToResponseInitChain()
-		} else {
-			responses <- tmsp_types.ToResponseInitChain()
-		}
-	case *tmsp_types.Request_EndBlock:
-		if app, ok := s.app.(tmsp_types.BlockchainAware); ok {
-			validators := app.EndBlock(r.EndBlock.Height)
-			responses <- tmsp_types.ToResponseEndBlock(validators)
-		} else {
-			responses <- tmsp_types.ToResponseEndBlock(nil)
-		}
-	default:
-		responses <- tmsp_types.ToResponseException("Unknown request")
-	}
-}
-
-// Pull responses from 'responses' and write them to conn.
-func (s *Server) handleResponses(closeConn chan error, responses <-chan *tmsp_types.Response, conn net.Conn) {
-	var count int
-	var bufWriter = bufio.NewWriter(conn)
-	for {
-		var res = <-responses
-		err := tmsp_types.WriteMessage(res, bufWriter)
-		if err != nil {
-			closeConn <- fmt.Errorf("Error in handleResponses: %v", err.Error())
-			return
-		}
-		if _, ok := res.Value.(*tmsp_types.Response_Flush); ok {
-			err = bufWriter.Flush()
-			if err != nil {
-				closeConn <- fmt.Errorf("Error in handleValue: %v", err.Error())
-				return
-			}
-		}
-		count++
-	}
-}
diff --git a/consensus/tmsp/version.go b/consensus/tmsp/version.go
deleted file mode 100644
index 266b37d792c4b606cdd547469f05d050edcc0b3b..0000000000000000000000000000000000000000
--- a/consensus/tmsp/version.go
+++ /dev/null
@@ -1,37 +0,0 @@
-// Copyright 2015, 2016 Eris Industries (UK) Ltd.
-// This file is part of Eris-RT
-
-// Eris-RT is free software: you can redistribute it and/or modify
-// it under the terms of the GNU General Public License as published by
-// the Free Software Foundation, either version 3 of the License, or
-// (at your option) any later version.
-
-// Eris-RT is distributed in the hope that it will be useful,
-// but WITHOUT ANY WARRANTY; without even the implied warranty of
-// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
-// GNU General Public License for more details.
-
-// You should have received a copy of the GNU General Public License
-// along with Eris-RT.  If not, see <http://www.gnu.org/licenses/>.
-
-package tmsp
-
-import (
-	version "github.com/eris-ltd/eris-db/version"
-)
-
-const (
-	// Client identifier to advertise over the network
-	tmspClientIdentifier = "tmsp"
-	// Major version component of the current release
-	tmspVersionMajor = 0
-	// Minor version component of the current release
-	tmspVersionMinor = 6
-	// Patch version component of the current release
-	tmspVersionPatch = 0
-)
-
-func GetTmspVersion() *version.VersionIdentifier {
-	return version.New(tmspClientIdentifier, tmspVersionMajor, tmspVersionMinor,
-		tmspVersionPatch)
-}
diff --git a/consensus/types/consensus_engine.go b/consensus/types/consensus_engine.go
index ea02d3b2ec36459765f755cfd2f11930a622b38c..52eb9d0b9b01197620c8abcb4874672a4f21c27c 100644
--- a/consensus/types/consensus_engine.go
+++ b/consensus/types/consensus_engine.go
@@ -5,7 +5,7 @@ import (
 	"github.com/eris-ltd/eris-db/txs"
 	"github.com/tendermint/go-crypto"
 	"github.com/tendermint/go-p2p"
-	tmsp_types "github.com/tendermint/tmsp/types"
+	abci_types "github.com/tendermint/abci/types"
 )
 
 type ConsensusEngine interface {
@@ -20,7 +20,7 @@ type ConsensusEngine interface {
 
 	// Memory pool
 	BroadcastTransaction(transaction []byte,
-		callback func(*tmsp_types.Response)) error
+		callback func(*abci_types.Response)) error
 
 	// Events
 	// For consensus events like NewBlock
diff --git a/core/core.go b/core/core.go
index 6c58d1a3312eba6c17cf06c121be1bbb201d48eb..9ce189aae12a2a71bc3b9445d1d44cdd15433581 100644
--- a/core/core.go
+++ b/core/core.go
@@ -40,7 +40,7 @@ import (
 // Core is the high-level structure
 type Core struct {
 	chainId        string
-	evsw           *events.EventSwitch
+	evsw           events.EventSwitch
 	pipe           definitions.Pipe
 	tendermintPipe definitions.TendermintPipe
 }
diff --git a/event/events.go b/event/events.go
index ec790a9ed8b1d120a46ecf1642bec840756fcbdd..65e439e5a7b284a7717ab13ec819cc22c9bedc43 100644
--- a/event/events.go
+++ b/event/events.go
@@ -44,7 +44,7 @@ type EventEmitter interface {
 	Unsubscribe(subId string) error
 }
 
-func NewEvents(eventSwitch *go_events.EventSwitch, logger loggers.InfoTraceLogger) *events {
+func NewEvents(eventSwitch go_events.EventSwitch, logger loggers.InfoTraceLogger) *events {
 	return &events{eventSwitch: eventSwitch, logger: logging.WithScope(logger, "Events")}
 }
 
@@ -57,7 +57,7 @@ func Multiplex(events ...EventEmitter) *multiplexedEvents {
 
 // The events struct has methods for working with events.
 type events struct {
-	eventSwitch *go_events.EventSwitch
+	eventSwitch go_events.EventSwitch
 	logger      loggers.InfoTraceLogger
 }
 
diff --git a/glide.lock b/glide.lock
index c226621ad80436ea994d07152b9b19871bb89fc0..6887d752d90fc9ab64b4b6d7181b92262ed992d3 100644
--- a/glide.lock
+++ b/glide.lock
@@ -195,8 +195,6 @@ imports:
   - state
   - types
   - version
-- name: github.com/tendermint/tmsp
-  version: f41bc5f11969e22b357d94b4247403fd62d40445
 - name: github.com/tommy351/gin-cors
   version: dc91dec6313ae4db53481bf3b29cf6b94bf80357
 - name: golang.org/x/crypto
diff --git a/manager/eris-mint/eris-mint.go b/manager/eris-mint/eris-mint.go
index 29050723ae9e89850b0ebbb91d432591ffdb4ef1..899bbec89333a4f77ec2a7b0ea245f8139f29d45 100644
--- a/manager/eris-mint/eris-mint.go
+++ b/manager/eris-mint/eris-mint.go
@@ -24,7 +24,7 @@ import (
 
 	tendermint_events "github.com/tendermint/go-events"
 	wire "github.com/tendermint/go-wire"
-	tmsp "github.com/tendermint/tmsp/types"
+	abci "github.com/tendermint/abci/types"
 
 	"github.com/eris-ltd/eris-db/logging"
 	"github.com/eris-ltd/eris-db/logging/loggers"
@@ -47,7 +47,7 @@ type ErisMint struct {
 	checkCache *sm.BlockCache // for CheckTx (eg. so we get nonces right)
 
 	evc  *tendermint_events.EventCache
-	evsw *tendermint_events.EventSwitch
+	evsw tendermint_events.EventSwitch
 
 	nTxs   int // count txs in a block
 	logger loggers.InfoTraceLogger
@@ -57,9 +57,9 @@ type ErisMint struct {
 // eris-db/manager/types.Application
 var _ manager_types.Application = (*ErisMint)(nil)
 
-// NOTE: [ben] also automatically implements tmsp.Application,
+// NOTE: [ben] also automatically implements abci.Application,
 // undesired but unharmful
-// var _ tmsp.Application = (*ErisMint)(nil)
+// var _ abci.Application = (*ErisMint)(nil)
 
 func (app *ErisMint) GetState() *sm.State {
 	app.mtx.Lock()
@@ -74,7 +74,7 @@ func (app *ErisMint) GetCheckCache() *sm.BlockCache {
 	return app.checkCache
 }
 
-func NewErisMint(s *sm.State, evsw *tendermint_events.EventSwitch, logger loggers.InfoTraceLogger) *ErisMint {
+func NewErisMint(s *sm.State, evsw tendermint_events.EventSwitch, logger loggers.InfoTraceLogger) *ErisMint {
 	return &ErisMint{
 		state:      s,
 		cache:      sm.NewBlockCache(s),
@@ -86,8 +86,8 @@ func NewErisMint(s *sm.State, evsw *tendermint_events.EventSwitch, logger logger
 }
 
 // Implements manager/types.Application
-func (app *ErisMint) Info() (info string) {
-	return "ErisDB"
+func (app *ErisMint) Info() (info abci.ResponseInfo) {
+	return abci.ResponseInfo{}
 }
 
 // Implements manager/types.Application
@@ -96,7 +96,7 @@ func (app *ErisMint) SetOption(key string, value string) (log string) {
 }
 
 // Implements manager/types.Application
-func (app *ErisMint) AppendTx(txBytes []byte) tmsp.Result {
+func (app *ErisMint) DeliverTx(txBytes []byte) abci.Result {
 	app.nTxs += 1
 
 	// XXX: if we had tx ids we could cache the decoded txs on CheckTx
@@ -106,45 +106,45 @@ func (app *ErisMint) AppendTx(txBytes []byte) tmsp.Result {
 	buf := bytes.NewBuffer(txBytes)
 	wire.ReadBinaryPtr(tx, buf, len(txBytes), &n, &err)
 	if err != nil {
-		return tmsp.NewError(tmsp.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err))
+		return abci.NewError(abci.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err))
 	}
 
 	err = sm.ExecTx(app.cache, *tx, true, app.evc)
 	if err != nil {
-		return tmsp.NewError(tmsp.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err))
+		return abci.NewError(abci.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err))
 	}
 
 	receipt := txs.GenerateReceipt(app.state.ChainID, *tx)
 	receiptBytes := wire.BinaryBytes(receipt)
-	return tmsp.NewResultOK(receiptBytes, "Success")
+	return abci.NewResultOK(receiptBytes, "Success")
 }
 
 // Implements manager/types.Application
-func (app *ErisMint) CheckTx(txBytes []byte) tmsp.Result {
+func (app *ErisMint) CheckTx(txBytes []byte) abci.Result {
 	var n int
 	var err error
 	tx := new(txs.Tx)
 	buf := bytes.NewBuffer(txBytes)
 	wire.ReadBinaryPtr(tx, buf, len(txBytes), &n, &err)
 	if err != nil {
-		return tmsp.NewError(tmsp.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err))
+		return abci.NewError(abci.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err))
 	}
 
-	// TODO: map ExecTx errors to sensible TMSP error codes
+	// TODO: map ExecTx errors to sensible abci error codes
 	err = sm.ExecTx(app.checkCache, *tx, false, nil)
 	if err != nil {
-		return tmsp.NewError(tmsp.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err))
+		return abci.NewError(abci.CodeType_InternalError, fmt.Sprintf("Internal error: %v", err))
 	}
 	receipt := txs.GenerateReceipt(app.state.ChainID, *tx)
 	receiptBytes := wire.BinaryBytes(receipt)
-	return tmsp.NewResultOK(receiptBytes, "Success")
+	return abci.NewResultOK(receiptBytes, "Success")
 }
 
 // Implements manager/types.Application
 // Commit the state (called at end of block)
 // NOTE: CheckTx/AppendTx must not run concurrently with Commit -
 //  the mempool should run during AppendTxs, but lock for Commit and Update
-func (app *ErisMint) Commit() (res tmsp.Result) {
+func (app *ErisMint) Commit() (res abci.Result) {
 	app.mtx.Lock() // the lock protects app.state
 	defer app.mtx.Unlock()
 
@@ -175,10 +175,10 @@ func (app *ErisMint) Commit() (res tmsp.Result) {
 	// MARMOT:
 	appHash := app.state.Hash()
 	fmt.Printf("\n\nMARMOT COMMIT: %X\n\n", appHash)
-	// return tmsp.NewResultOK(app.state.Hash(), "Success")
-	return tmsp.NewResultOK(appHash, "Success")
+	// return abci.NewResultOK(app.state.Hash(), "Success")
+	return abci.NewResultOK(appHash, "Success")
 }
 
-func (app *ErisMint) Query(query []byte) (res tmsp.Result) {
-	return tmsp.NewResultOK(nil, "Success")
+func (app *ErisMint) Query(query []byte) (res abci.Result) {
+	return abci.NewResultOK(nil, "Success")
 }
diff --git a/manager/eris-mint/pipe.go b/manager/eris-mint/pipe.go
index 014de66207a1c2b0f27de336016d15291f9c8de6..9fda3eb6cb74974962034d8a1a943cdd15713a14 100644
--- a/manager/eris-mint/pipe.go
+++ b/manager/eris-mint/pipe.go
@@ -20,13 +20,13 @@ import (
 	"bytes"
 	"fmt"
 
+	abci_types "github.com/tendermint/abci/types"
 	tm_common "github.com/tendermint/go-common"
 	crypto "github.com/tendermint/go-crypto"
 	db "github.com/tendermint/go-db"
 	go_events "github.com/tendermint/go-events"
 	wire "github.com/tendermint/go-wire"
 	tm_types "github.com/tendermint/tendermint/types"
-	tmsp_types "github.com/tendermint/tmsp/types"
 
 	"github.com/eris-ltd/eris-db/account"
 	blockchain_types "github.com/eris-ltd/eris-db/blockchain/types"
@@ -71,7 +71,7 @@ var _ definitions.Pipe = (*erisMintPipe)(nil)
 var _ definitions.TendermintPipe = (*erisMintPipe)(nil)
 
 func NewErisMintPipe(moduleConfig *config.ModuleConfig,
-	eventSwitch *go_events.EventSwitch,
+	eventSwitch go_events.EventSwitch,
 	logger loggers.InfoTraceLogger) (*erisMintPipe, error) {
 
 	startedState, genesisDoc, err := startState(moduleConfig.DataDir,
@@ -144,8 +144,8 @@ func NewErisMintPipe(moduleConfig *config.ModuleConfig,
 func startState(dataDir, backend, genesisFile, chainId string) (*state.State,
 	*genesis.GenesisDoc, error) {
 	// avoid Tendermints PanicSanity and return a clean error
-	if backend != db.DBBackendMemDB &&
-		backend != db.DBBackendLevelDB {
+	if backend != db.MemDBBackendStr &&
+		backend != db.LevelDBBackendStr {
 		return nil, nil, fmt.Errorf("Database backend %s is not supported by %s",
 			backend, GetErisMintVersion)
 	}
@@ -534,7 +534,7 @@ func (pipe *erisMintPipe) ListNames() (*rpc_tm_types.ResultListNames, error) {
 }
 
 func (pipe *erisMintPipe) broadcastTx(tx txs.Tx,
-	callback func(res *tmsp_types.Response)) (*rpc_tm_types.ResultBroadcastTx, error) {
+	callback func(res *abci_types.Response)) (*rpc_tm_types.ResultBroadcastTx, error) {
 
 	txBytes, err := txs.EncodeTx(tx)
 	if err != nil {
@@ -554,9 +554,9 @@ func (pipe *erisMintPipe) BroadcastTxAsync(tx txs.Tx) (*rpc_tm_types.ResultBroad
 }
 
 func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadcastTx, error) {
-	responseChannel := make(chan *tmsp_types.Response, 1)
+	responseChannel := make(chan *abci_types.Response, 1)
 	_, err := pipe.broadcastTx(tx,
-		func(res *tmsp_types.Response) {
+		func(res *abci_types.Response) {
 			responseChannel <- res
 		})
 	if err != nil {
@@ -564,7 +564,7 @@ func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadc
 	}
 	// NOTE: [ben] This Response is set in /consensus/tendermint/local_client.go
 	// a call to Application, here implemented by ErisMint, over local callback,
-	// or TMSP RPC call.  Hence the result is determined by ErisMint/erismint.go
+	// or abci RPC call.  Hence the result is determined by ErisMint/erismint.go
 	// CheckTx() Result (Result converted to ReqRes into Response returned here)
 	// NOTE: [ben] BroadcastTx just calls CheckTx in Tendermint (oddly... [Silas])
 	response := <-responseChannel
@@ -578,17 +578,17 @@ func (pipe *erisMintPipe) BroadcastTxSync(tx txs.Tx) (*rpc_tm_types.ResultBroadc
 		Log:  responseCheckTx.Log,
 	}
 	switch responseCheckTx.Code {
-	case tmsp_types.CodeType_OK:
+	case abci_types.CodeType_OK:
 		return resultBroadCastTx, nil
-	case tmsp_types.CodeType_EncodingError:
+	case abci_types.CodeType_EncodingError:
 		return resultBroadCastTx, fmt.Errorf(resultBroadCastTx.Log)
-	case tmsp_types.CodeType_InternalError:
+	case abci_types.CodeType_InternalError:
 		return resultBroadCastTx, fmt.Errorf(resultBroadCastTx.Log)
 	default:
 		logging.InfoMsg(pipe.logger, "Unknown error returned from Tendermint CheckTx on BroadcastTxSync",
 			"application", GetErisMintVersion().GetVersionString(),
-			"TMSP_code_type", responseCheckTx.Code,
-			"TMSP_log", responseCheckTx.Log,
+			"abci_code_type", responseCheckTx.Code,
+			"abci_log", responseCheckTx.Log,
 		)
 		return resultBroadCastTx, fmt.Errorf("Unknown error returned: " + responseCheckTx.Log)
 	}
diff --git a/manager/eris-mint/version.go b/manager/eris-mint/version.go
index ad15d4a34d7cf0b2eb78a32d4831194e6db9aeb6..e6ca9a4c2a6b1c7d60eafb4c653e51a5f6951181 100644
--- a/manager/eris-mint/version.go
+++ b/manager/eris-mint/version.go
@@ -36,8 +36,7 @@ const (
 // Define the compatible consensus engines this application manager
 // is compatible and has been tested with.
 var compatibleConsensus = [...]string{
-	"tendermint-0.6",
-	// "tmsp-0.6",
+	"tendermint-0.8",
 }
 
 func GetErisMintVersion() *version.VersionIdentifier {
diff --git a/manager/manager.go b/manager/manager.go
index 6f71aefa92dfe857cc4bf8d65b9c3cb22fae6eac..9251aeed5bb2e10cd65d83a663fb90b4a36e028c 100644
--- a/manager/manager.go
+++ b/manager/manager.go
@@ -36,7 +36,7 @@ import (
 // of an application.  It is feasible this will be insufficient to support
 // different types of applications later down the line.
 func NewApplicationPipe(moduleConfig *config.ModuleConfig,
-	evsw *events.EventSwitch, logger loggers.InfoTraceLogger,
+	evsw events.EventSwitch, logger loggers.InfoTraceLogger,
 	consensusMinorVersion string) (definitions.Pipe,
 	error) {
 	switch moduleConfig.Name {
diff --git a/manager/types/application.go b/manager/types/application.go
index cfcf2b82cc35cfc01c6b9da487f6908143183534..3ab29f6d43c60e72e8bb5db6b8c7f3f1e32471da 100644
--- a/manager/types/application.go
+++ b/manager/types/application.go
@@ -17,10 +17,10 @@
 package types
 
 import (
-	// TODO: [ben] this is currently only used for tmsp result type; but should
-	// be removed as tmsp dependencies shouldn't feature in the application
+	// TODO: [ben] this is currently only used for abci result type; but should
+	// be removed as abci dependencies shouldn't feature in the application
 	// manager
-	tmsp_types "github.com/tendermint/tmsp/types"
+	abci_types "github.com/tendermint/abci/types"
 )
 
 // NOTE: [ben] this interface is likely to be changed.  Currently it is taken
@@ -32,16 +32,16 @@ type Application interface {
 
 	// Info returns application information as a string
 	// NOTE: [ben] likely to move
-	Info() (info string)
+	Info() (info abci_types.ResponseInfo)
 
 	// Set application option (e.g. mode=mempool, mode=consensus)
 	// NOTE: [ben] taken from tendermint, but it is unclear what the use is,
-	// specifically, when will tendermint call this over tmsp ?
+	// specifically, when will tendermint call this over abci ?
 	SetOption(key string, value string) (log string)
 
 	// Append transaction applies a transaction to the state regardless of
 	// whether the transaction is valid or not.
-	// Currently AppendTx is taken from tmsp, and returns a result.
+	// Currently AppendTx is taken from abci, and returns a result.
 	// This will be altered, as AppendTransaction needs to more strongly reflect
 	// the theoretical logic:
 	//   Append(StateN, Transaction) = StateN+1
@@ -51,18 +51,18 @@ type Application interface {
 	// TODO: implementation notes:
 	// 1. at this point the transaction should already be strongly typed
 	// 2.
-	AppendTx(tx []byte) tmsp_types.Result
+	DeliverTx(tx []byte) abci_types.Result
 
 	// Check Transaction validates a transaction before being allowed into the
 	// consensus' engine memory pool.  This is the original defintion and
-	// intention as taken from tmsp, but should be remapped to the more
+	// intention as taken from abci, but should be remapped to the more
 	// general concept of basic, cheap verification;
 	// Check Transaction does not alter the state, but does require an immutable
 	// copy of the state. In particular there is no consensus on ordering yet.
 	// TODO: implementation notes:
 	// 1. at this point the transaction should already be strongly typed
 	// 2.
-	CheckTx(tx []byte) tmsp_types.Result
+	CheckTx(tx []byte) abci_types.Result
 
 	// Commit returns the root hash of the current application state
 	// NOTE: [ben] Because the concept of the block has been erased here
@@ -70,14 +70,14 @@ type Application interface {
 	// the opposit the principle of explicit stateless functions.
 	// This will be amended when we introduce the concept of (streaming)
 	// blocks in the pipe.
-	Commit() tmsp_types.Result
+	Commit() abci_types.Result
 
 	// Query for state.  This query request is not passed over the p2p network
 	// and is called from Tendermint rpc directly up to the application.
 	// NOTE: [ben] Eris-DB will give preference to queries from the local client
 	// directly over the Eris-DB rpc.
 	// We will support this for Tendermint compatibility.
-	Query(query []byte) tmsp_types.Result
+	Query(query []byte) abci_types.Result
 }
 
 // Tendermint has a separate interface for reintroduction of blocks
@@ -85,7 +85,7 @@ type BlockchainAware interface {
 
 	// Initialise the blockchain
 	// validators: genesis validators from tendermint core
-	InitChain(validators []*tmsp_types.Validator)
+	InitChain(validators []*abci_types.Validator)
 
 	// Signals the beginning of a block;
 	// NOTE: [ben] currently not supported by tendermint
@@ -95,5 +95,5 @@ type BlockchainAware interface {
 	// validators: changed validators from app to Tendermint
 	// NOTE: [ben] currently not supported by tendermint
 	// not yet well defined what the change set contains.
-	EndBlock(height uint64) (validators []*tmsp_types.Validator)
+	EndBlock(height uint64) (validators []*abci_types.Validator)
 }
diff --git a/rpc/tendermint/core/types/responses.go b/rpc/tendermint/core/types/responses.go
index 98b72fbbd200d120c46c834a02d5cc237ff6b5a4..87d9eeafc67252fd4a4228f0388d1c9eeb9cebd7 100644
--- a/rpc/tendermint/core/types/responses.go
+++ b/rpc/tendermint/core/types/responses.go
@@ -12,7 +12,7 @@ import (
 	"github.com/tendermint/go-p2p"
 	"github.com/tendermint/go-rpc/types"
 	"github.com/tendermint/go-wire"
-	tmsptypes "github.com/tendermint/tmsp/types"
+	abcitypes "github.com/tendermint/abci/types"
 )
 
 type ResultGetStorage struct {
@@ -111,7 +111,7 @@ type ResultGetAccount struct {
 }
 
 type ResultBroadcastTx struct {
-	Code tmsptypes.CodeType `json:"code"`
+	Code abcitypes.CodeType `json:"code"`
 	Data []byte             `json:"data"`
 	Log  string             `json:"log"`
 }
diff --git a/rpc/tendermint/core/websocket.go b/rpc/tendermint/core/websocket.go
index 6824b78e052becb787ccf8092870cb4a73b37c2a..846ddfdc29fcd72aeb1064074ed9bbef016e1a86 100644
--- a/rpc/tendermint/core/websocket.go
+++ b/rpc/tendermint/core/websocket.go
@@ -35,7 +35,7 @@ type TendermintWebsocketServer struct {
 }
 
 func NewTendermintWebsocketServer(config *server.ServerConfig,
-	tendermintPipe definitions.TendermintPipe, evsw *events.EventSwitch) (
+	tendermintPipe definitions.TendermintPipe, evsw events.EventSwitch) (
 	*TendermintWebsocketServer, error) {
 
 	if tendermintPipe == nil {
diff --git a/rpc/tendermint/test/config.go b/rpc/tendermint/test/config.go
index fcf964bbfc7c0b53183c8838dd521b822eb84a31..ee5bd5a20e94bbf941acc0b75081fc4be2a3ade9 100644
--- a/rpc/tendermint/test/config.go
+++ b/rpc/tendermint/test/config.go
@@ -40,12 +40,12 @@ genesis_file = "genesis.json"
   [chain.consensus]
   # consensus defines the module to use for consensus and
   # this will define the peer-to-peer consensus network;
-  # accepted values are "noops", "tmsp", "tendermint"
+  # accepted values are "noops", "abci", "tendermint"
   name = "tendermint"
   # version is the major and minor semantic version;
   # the version will be asserted on
   major_version = 0
-  minor_version = 6
+  minor_version = 8
   # relative path to consensus' module root folder
   relative_root = "tendermint"
 
@@ -123,21 +123,19 @@ genesis_file = "genesis.json"
 
 ################################################################################
 ##
-## Tendermint Socket Protocol (TMSP)
-## version 0.6.0
+## Tendermint Socket Protocol (abci)
 ##
-## TMSP expects a tendermint consensus process to run and connect to Eris-DB
+## abci expects a tendermint consensus process to run and connect to Eris-DB
 ##
 ################################################################################
 
-[tmsp]
+[abci]
 # listener address for accepting tendermint socket protocol connections
 listener = "tcp://0.0.0.0:46658"
 
 ################################################################################
-##
+##yeah we had partial support for that with TMSP
 ## Tendermint
-## version 0.6.0
 ##
 ## in-process execution of Tendermint consensus engine
 ##
@@ -174,7 +172,7 @@ private_validator_file = "priv_validator.json"
 	# NOTE: value is ignored when run in-process as RPC is
 	# handled by [servers.tendermint]
   rpc_laddr = ""
-  # proxy application address - used for tmsp connections,
+  # proxy application address - used for abci connections,
   # and this port should not be exposed for in-process Tendermint
   proxy_app = "tcp://127.0.0.1:46658"
 
diff --git a/rpc/tendermint/test/rpc_client_test.go b/rpc/tendermint/test/rpc_client_test.go
index 7183f2df8f1effbafe76705e344fd1658c01852a..ba5f472a0a098794456e4c7466253ef20ed092c6 100644
--- a/rpc/tendermint/test/rpc_client_test.go
+++ b/rpc/tendermint/test/rpc_client_test.go
@@ -295,7 +295,7 @@ func TestBlockchainInfo(t *testing.T) {
 
 		lastBlockHash := resp.BlockMetas[nBlocks-1].Hash
 		for i := nBlocks - 2; i >= 0; i-- {
-			assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastBlockHash,
+			assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastCommitHash,
 				"Blockchain should be a hash tree!")
 			lastBlockHash = resp.BlockMetas[i].Hash
 		}
diff --git a/test/mock/pipe.go b/test/mock/pipe.go
index ecd245ec87e2c9934e360ae72869210e134ee820..cfa192c9cf817e71bca77ce2fa1510efe4dbbe1b 100644
--- a/test/mock/pipe.go
+++ b/test/mock/pipe.go
@@ -18,7 +18,7 @@ import (
 	"github.com/tendermint/go-crypto"
 	"github.com/tendermint/go-p2p"
 	mintTypes "github.com/tendermint/tendermint/types"
-	tmsp_types "github.com/tendermint/tmsp/types"
+	abci_types "github.com/tendermint/abci/types"
 )
 
 // Base struct.
@@ -165,7 +165,7 @@ type consensusEngine struct {
 }
 
 func (cons *consensusEngine) BroadcastTransaction(transaction []byte,
-	callback func(*tmsp_types.Response)) error {
+	callback func(*abci_types.Response)) error {
 	return nil
 }