diff --git a/circle.yml b/circle.yml
index f2cc9173abdedf6a000600d403a381bc9ab635fe..54189a8cb08cbfc6968115f0be520cec9f7918a1 100644
--- a/circle.yml
+++ b/circle.yml
@@ -55,6 +55,13 @@ deployment:
       - docker push quay.io/eris/db
       # push the updated documentation
       - docs/build.sh
+  develop:
+    branch: develop
+    commands:
+      - docker login -e $DOCKER_EMAIL -u $DOCKER_USER -p $DOCKER_PASS quay.io
+      # build docker image and tag the image with the version
+      - tests/build_tool.sh
+      - docker push quay.io/eris/db
   master:
     branch: master
     commands:
@@ -77,6 +84,3 @@ deployment:
       - docker push quay.io/eris/db
       # push the updated documentation
       - docs/build.sh
-
-
-
diff --git a/config/templates.go b/config/templates.go
index ede335eccb60d277ab83184fe08dee631b6edea5..904cbc2bfd5bd96ab8701d5e126daa6a4ea04810 100644
--- a/config/templates.go
+++ b/config/templates.go
@@ -83,7 +83,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
@@ -181,17 +181,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"
 
@@ -202,7 +202,7 @@ const sectionTendermint = `
 ################################################################################
 ##
 ## Tendermint
-## version 0.6.0
+## version 0.6
 ##
 ## in-process execution of Tendermint consensus engine
 ##
@@ -241,7 +241,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"
 
@@ -297,7 +297,7 @@ const sectionErisMint = `
 ################################################################################
 ##
 ## Eris-Mint
-## version 0.16.0
+## version 0.16
 ##
 ## The original Ethereum virtual machine with IAVL merkle trees
 ## and tendermint/go-wire encoding
diff --git a/consensus/config.go b/consensus/config.go
index c2a7cdf4977e27fa38fed5ca92e310048c19842c..05cc488f158d93dde57ec0ab46326d29b76b13a5 100644
--- a/consensus/config.go
+++ b/consensus/config.go
@@ -17,7 +17,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"
 )
 
 //------------------------------------------------------------------------------
@@ -29,8 +28,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/config.go b/consensus/tendermint/config.go
index c55115f7254527da1dce0549f07224ce72e6473a..b8c89d241d175d442b1550b09619b852a37f7b2c 100644
--- a/consensus/tendermint/config.go
+++ b/consensus/tendermint/config.go
@@ -51,6 +51,10 @@ func GetTendermintConfig(loadedConfig *viper.Viper) *TendermintConfig {
 //------------------------------------------------------------------------------
 // Tendermint defaults
 
+//
+// Contract
+//
+
 func (tmintConfig *TendermintConfig) AssertTendermintDefaults(chainId, workDir,
 	dataDir, rootDir string) {
 
@@ -64,6 +68,8 @@ func (tmintConfig *TendermintConfig) AssertTendermintDefaults(chainId, workDir,
 	tmintConfig.SetDefault("fast_sync", true)
 	tmintConfig.SetDefault("skip_upnp", false)
 	tmintConfig.SetDefault("addrbook_file", path.Join(rootDir, "addrbook.json"))
+	tmintConfig.SetDefault("addrbook_strict", true) // disable to allow connections locally
+	tmintConfig.SetDefault("pex_reactor", false)    // enable for peer exchange
 	tmintConfig.SetDefault("priv_validator_file", path.Join(rootDir, "priv_validator.json"))
 	tmintConfig.SetDefault("db_backend", "leveldb")
 	tmintConfig.SetDefault("db_dir", dataDir)
@@ -71,10 +77,12 @@ func (tmintConfig *TendermintConfig) AssertTendermintDefaults(chainId, workDir,
 	tmintConfig.SetDefault("rpc_laddr", "")
 	tmintConfig.SetDefault("prof_laddr", "")
 	tmintConfig.SetDefault("revision_file", path.Join(workDir, "revision"))
-	tmintConfig.SetDefault("cswal", path.Join(dataDir, "cswal"))
-	tmintConfig.SetDefault("cswal_light", false)
+	tmintConfig.SetDefault("cs_wal_dir", path.Join(dataDir, "cs.wal"))
+	tmintConfig.SetDefault("cs_wal_light", false)
+	tmintConfig.SetDefault("filter_peers", false)
 
-	tmintConfig.SetDefault("block_size", 10000)
+	tmintConfig.SetDefault("block_size", 10000)      // max number of txs
+	tmintConfig.SetDefault("block_part_size", 65536) // part size 64K
 	tmintConfig.SetDefault("disable_data_hash", false)
 	tmintConfig.SetDefault("timeout_propose", 3000)
 	tmintConfig.SetDefault("timeout_propose_delta", 500)
@@ -83,9 +91,12 @@ func (tmintConfig *TendermintConfig) AssertTendermintDefaults(chainId, workDir,
 	tmintConfig.SetDefault("timeout_precommit", 1000)
 	tmintConfig.SetDefault("timeout_precommit_delta", 500)
 	tmintConfig.SetDefault("timeout_commit", 1000)
+	// make progress asap (no `timeout_commit`) on full precommit votes
+	tmintConfig.SetDefault("skip_timeout_commit", false)
 	tmintConfig.SetDefault("mempool_recheck", true)
 	tmintConfig.SetDefault("mempool_recheck_empty", true)
 	tmintConfig.SetDefault("mempool_broadcast", true)
+	tmintConfig.SetDefault("mempool_wal_dir", path.Join(dataDir, "mempool.wal"))
 }
 
 //------------------------------------------------------------------------------
diff --git a/consensus/tendermint/local_client.go b/consensus/tendermint/local_client.go
deleted file mode 100644
index 2a2cd873018966cfb318a4d7fe080f06bb8b30d1..0000000000000000000000000000000000000000
--- a/consensus/tendermint/local_client.go
+++ /dev/null
@@ -1,265 +0,0 @@
-// Copyright 2017 Monax Industries Limited
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package tendermint
-
-// This file is originally based on github.com/tendermint/tmsp/client/...
-// .../local_client.go
-
-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 b4d9046b7b9fd37418babecbab1904d80dd14774..2eda6171daa4742efe486bb1c958128299fb99e0 100644
--- a/consensus/tendermint/tendermint.go
+++ b/consensus/tendermint/tendermint.go
@@ -18,15 +18,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"
 
@@ -120,10 +119,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"))
@@ -230,7 +227,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 a759e54411942593e4e2f7ba41661190a3b20d7f..78ed5dea5ec3df5e07a19520959e371e8f2ece73 100644
--- a/consensus/tendermint/version.go
+++ b/consensus/tendermint/version.go
@@ -28,7 +28,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 e961bc50c6d50bb01e12a1eb7a62d95e05cdad40..0000000000000000000000000000000000000000
--- a/consensus/tmsp/server.go
+++ /dev/null
@@ -1,209 +0,0 @@
-// Copyright 2017 Monax Industries Limited
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package tmsp
-
-// Taken originally from github.com/tendermint/tmsp/server.go
-
-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 a5422c211433acb164359eb4cb8a426e748a4bc1..0000000000000000000000000000000000000000
--- a/consensus/tmsp/version.go
+++ /dev/null
@@ -1,35 +0,0 @@
-// Copyright 2017 Monax Industries Limited
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//    http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package 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 0ca7bd481b905c2e074341a3b244fcd285ce7461..191bc6e4cbd771803d56d889b8bcc22508ba6386 100644
--- a/consensus/types/consensus_engine.go
+++ b/consensus/types/consensus_engine.go
@@ -17,9 +17,9 @@ package types
 import (
 	"github.com/eris-ltd/eris-db/event"
 	"github.com/eris-ltd/eris-db/txs"
+	abci_types "github.com/tendermint/abci/types"
 	"github.com/tendermint/go-crypto"
 	"github.com/tendermint/go-p2p"
-	tmsp_types "github.com/tendermint/tmsp/types"
 )
 
 type ConsensusEngine interface {
@@ -34,7 +34,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 decb567c78eeffce45b95d4d2361fc59697db0f6..14725b6c2ac137df77f856577f25e897ddc432c2 100644
--- a/core/core.go
+++ b/core/core.go
@@ -38,7 +38,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 4ef6ddac0e0db621043a08155bf3728afc51da04..5ada94a52e994f7497c111300c65205a86894d12 100644
--- a/event/events.go
+++ b/event/events.go
@@ -42,7 +42,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")}
 }
 
@@ -55,7 +55,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 62620c5ded26d6461e28801b4c5ce280fdfcb974..6887d752d90fc9ab64b4b6d7181b92262ed992d3 100644
--- a/glide.lock
+++ b/glide.lock
@@ -1,12 +1,12 @@
-hash: f1f85c5d4b9520217cc6fa9fd7b7e97790e737def9bc08ab45d53d5db729c779
-updated: 2016-09-14T20:54:48.289839938+02:00
+hash: 310aa7c7435ad7dd1c3eb6772a42065b5f506e38e195107bdbfb1584833add9a
+updated: 2017-02-21T01:43:41.814044634Z
 imports:
 - name: github.com/Azure/go-ansiterm
   version: 388960b655244e76e24c75f48631564eaefade62
   subpackages:
   - winterm
 - name: github.com/btcsuite/btcd
-  version: 7de7bddba9d9a8b0b7490931e346e8f10d0bdb7f
+  version: 153dca5c1e4b5d1ea1523592495e5bedfa503391
   subpackages:
   - btcec
 - name: github.com/btcsuite/fastsha256
@@ -20,9 +20,15 @@ imports:
 - name: github.com/bugsnag/panicwrap
   version: d6191e27ad06236eaad65d79e49a08b03b9f8029
 - name: github.com/BurntSushi/toml
-  version: f0aeabca5a127c4078abb8c8d64298b147264b55
+  version: 99064174e013895bbd9b025c31100bd1d9b590ca
 - name: github.com/davecgh/go-spew
   version: 5215b55f46b2b919f50a1df0eaa5886afe4e3b3d
+- name: github.com/eapache/channels
+  version: 47238d5aae8c0fefd518ef2bee46290909cf8263
+- name: github.com/eapache/queue
+  version: 44cc805cf13205b55f69e14bcb69867d1ae92f98
+- name: github.com/ebuchman/fail-test
+  version: c1eddaa09da2b4017351245b0d43234955276798
 - name: github.com/eris-ltd/eris-keys
   version: 114ebc77443db9a153692233294e48bc7e184215
 - name: github.com/fsnotify/fsnotify
@@ -32,37 +38,53 @@ imports:
   subpackages:
   - binding
   - render
+- name: github.com/go-kit/kit
+  version: f66b0e13579bfc5a48b9e2a94b1209c107ea1f41
+  subpackages:
+  - log
+- name: github.com/go-logfmt/logfmt
+  version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5
 - name: github.com/go-stack/stack
   version: 100eb0c0a9c5b306ca2fb4f165df21d80ada4b82
+- name: github.com/gogo/protobuf
+  version: f9114dace7bd920b32f943b3c73fafbcbab2bf31
 - name: github.com/golang/protobuf
-  version: 0c1f6d65b5a189c2250d10e71a5506f06f9fa0a0
+  version: 8ee79997227bf9b34611aee7946ae64735e6fd93
   subpackages:
   - proto
 - name: github.com/golang/snappy
   version: d9eb7a3d35ec988b8585d4a0068e462c27d28380
 - name: github.com/gorilla/websocket
-  version: a68708917c6a4f06314ab4e52493cc61359c9d42
+  version: 17634340a83afe0cab595e40fbc63f6ffa1d8915
 - name: github.com/hashicorp/hcl
   version: da486364306ed66c218be9b7953e19173447c18b
   subpackages:
   - hcl/ast
   - hcl/parser
-  - hcl/token
-  - json/parser
   - hcl/scanner
   - hcl/strconv
+  - hcl/token
+  - json/parser
   - json/scanner
   - json/token
+- name: github.com/inconshreveable/log15
+  version: 46a701a619de90c65a78c04d1a58bf02585e9701
+  subpackages:
+  - term
 - name: github.com/inconshreveable/mousetrap
   version: 76626ae9c91c4f2a10f34cad8ce83ea42c93bb75
+- name: github.com/jmhodges/levigo
+  version: c42d9e0ca023e2198120196f842701bb4c55d7b9
 - name: github.com/magiconair/properties
   version: c265cfa48dda6474e208715ca93e987829f572f8
 - name: github.com/manucorporat/sse
   version: ee05b128a739a0fb76c7ebd3ae4810c1de808d6d
+- name: github.com/Masterminds/glide
+  version: 84607742b10f492430762d038e954236bbaf23f7
 - name: github.com/mattn/go-colorable
-  version: 9056b7a9f2d1f2d96498d6d146acd1f9d5ed3d59
+  version: d228849504861217f796da67fae4f6e347643f15
 - name: github.com/mattn/go-isatty
-  version: 56b76bdf51f7708750eac80fa38b952bb9f32639
+  version: 30a891c33c7cde7b02a981314b4228ec99380cca
 - name: github.com/mitchellh/mapstructure
   version: d2dd0262208475919e1a362f675cfc0e7c10e905
 - name: github.com/naoina/toml
@@ -72,7 +94,7 @@ imports:
   subpackages:
   - difflib
 - name: github.com/Sirupsen/logrus
-  version: f3cfb454f4c209e6668c95216c4744b8fddb2356
+  version: d26492970760ca5d33129d2d799e34be5c4782eb
 - name: github.com/spf13/cast
   version: 27b586b42e29bec072fe7379259cc719e1289da6
 - name: github.com/spf13/cobra
@@ -80,117 +102,133 @@ imports:
 - name: github.com/spf13/jwalterweatherman
   version: 33c24e77fb80341fe7130ee7c594256ff08ccc46
 - name: github.com/spf13/pflag
-  version: 367864438f1b1a3c7db4da06a2f55b144e6784e0
+  version: 25f8b5b07aece3207895bf19f7ab517eb3b22a40
 - name: github.com/spf13/viper
   version: c1ccc378a054ea8d4e38d8c67f6938d4760b53dd
+- name: github.com/streadway/simpleuuid
+  version: 6617b501e485b77e61b98cd533aefff9e258b5a7
 - name: github.com/stretchr/testify
   version: d77da356e56a7428ad25149ca77381849a6a5232
   subpackages:
   - assert
 - name: github.com/syndtr/goleveldb
-  version: fa5b5c78794bc5c18f330361059f871ae8c2b9d6
+  version: 23851d93a2292dcc56e71a18ec9e0624d84a0f65
   subpackages:
   - leveldb
-  - leveldb/errors
-  - leveldb/opt
   - leveldb/cache
   - leveldb/comparer
+  - leveldb/errors
   - leveldb/filter
   - leveldb/iterator
   - leveldb/journal
   - leveldb/memdb
+  - leveldb/opt
   - leveldb/storage
   - leveldb/table
   - leveldb/util
+- name: github.com/tendermint/abci
+  version: 699d45bc678865b004b90213bf88a950f420973b
+  subpackages:
+  - client
+  - example/counter
+  - example/dummy
+  - example/nil
+  - server
+  - types
 - name: github.com/tendermint/ed25519
-  version: fdac6641497281ed1cc368687ec6377e96e02b24
+  version: 1f52c6f8b8a5c7908aff4497c186af344b428925
   subpackages:
   - edwards25519
   - extra25519
 - name: github.com/tendermint/flowcontrol
   version: 84d9671090430e8ec80e35b339907e0579b999eb
+- name: github.com/tendermint/go-autofile
+  version: 0416e0aa9c68205aa44844096f9f151ada9d0405
 - name: github.com/tendermint/go-clist
-  version: 634527f5b60fd7c71ca811262493df2ad65ee0ca
+  version: 3baa390bbaf7634251c42ad69a8682e7e3990552
 - name: github.com/tendermint/go-common
-  version: dcfa46af1341d03b80d32e4901019d1668b978b9
+  version: e289af53b6bf6af28da129d9ef64389a4cf7987f
+  subpackages:
+  - test
 - name: github.com/tendermint/go-config
-  version: cfcef384d64b94e50909596e39b32ffb3cc20573
+  version: e64b424499acd0eb9856b88e10c0dff41628c0d6
 - name: github.com/tendermint/go-crypto
-  version: 41cfb7b677f4e16cdfd22b6ce0946c89919fbc7b
+  version: 4b11d62bdb324027ea01554e5767b71174680ba0
 - name: github.com/tendermint/go-db
-  version: 31fdd21c7eaeed53e0ea7ca597fb1e960e2988a5
+  version: 72f6dacd22a686cdf7fcd60286503e3aceda77ba
 - name: github.com/tendermint/go-events
-  version: 7b75ca7bb55aa25e9ef765eb8c0b69486b227357
+  version: fddee66d90305fccb6f6d84d16c34fa65ea5b7f6
+- name: github.com/tendermint/go-flowrate
+  version: a20c98e61957faa93b4014fbd902f20ab9317a6a
+  subpackages:
+  - flowrate
 - name: github.com/tendermint/go-logger
-  version: 529efe50eab1a8a9c111d55f4de4ecd95f482761
+  version: cefb3a45c0bf3c493a04e9bcd9b1540528be59f2
 - name: github.com/tendermint/go-merkle
-  version: 05042c6ab9cad51d12e4cecf717ae68e3b1409a8
+  version: 7a86b4486f2cd84ac885c5bbc609fdee2905f5d1
 - name: github.com/tendermint/go-p2p
-  version: 5bd7692323ec60d6461678f09b5024a952164151
+  version: 3d98f675f30dc4796546b8b890f895926152fa8d
   subpackages:
   - upnp
 - name: github.com/tendermint/go-rpc
-  version: 479510be0e80dd9e5d6b1f941adad168df0af85f
+  version: fcea0cda21f64889be00a0f4b6d13266b1a76ee7
   subpackages:
   - client
   - server
   - types
 - name: github.com/tendermint/go-wire
-  version: 3b0adbc86ed8425eaed98516165b6788d9f4de7a
+  version: 2f3b7aafe21c80b19b6ee3210ecb3e3d07c7a471
 - name: github.com/tendermint/log15
-  version: 9545b249b3aacafa97f79e0838b02b274adc6f5f
+  version: ae0f3d6450da9eac7074b439c8e1c3cabf0d5ce6
   subpackages:
   - term
 - name: github.com/tendermint/tendermint
-  version: aaea0c5d2e3ecfbf29f2608f9d43649ec7f07f50
+  version: 764091dfbb035f1b28da4b067526e04c6a849966
   subpackages:
-  - node
-  - proxy
-  - types
-  - version
-  - consensus
-  - rpc/core/types
   - blockchain
+  - consensus
   - mempool
+  - node
+  - proxy
   - rpc/core
+  - rpc/core/types
   - state
-- name: github.com/tendermint/tmsp
-  version: 73e5c3cb7bbee2f9c49792e5a0fcbcab442bf7dc
-  subpackages:
-  - client
   - types
-  - example/dummy
-  - example/nil
+  - version
 - name: github.com/tommy351/gin-cors
   version: dc91dec6313ae4db53481bf3b29cf6b94bf80357
 - name: golang.org/x/crypto
-  version: f3241ce8505855877cc8a9717bd61a0f7c4ea83c
+  version: 7c6cc321c680f03b9ef0764448e780704f486b51
   subpackages:
-  - ripemd160
+  - curve25519
+  - nacl/box
   - nacl/secretbox
   - openpgp/armor
-  - nacl/box
+  - openpgp/errors
   - poly1305
+  - ripemd160
   - salsa20/salsa
-  - openpgp/errors
-  - curve25519
 - name: golang.org/x/net
-  version: de35ec43e7a9aabd6a9c54d2898220ea7e44de7d
+  version: 60c41d1de8da134c05b7b40154a9a82bf5b7edb9
   subpackages:
-  - http2
   - context
-  - netutil
-  - trace
+  - http2
   - http2/hpack
   - idna
-  - lex/httplex
   - internal/timeseries
 - name: golang.org/x/sys
-  version: 62bee037599929a6e9146f29d10dd5208c43507d
+  version: d75a52659825e75fff6158388dddc6a5b04f9ba5
   subpackages:
   - unix
+- name: golang.org/x/text
+  version: 44f4f658a783b0cee41fe0a23b8fc91d9c120558
+  subpackages:
+  - secure/bidirule
+  - transform
+  - unicode/bidi
+  - unicode/norm
 - name: google.golang.org/grpc
-  version: e78224b060cf3215247b7be455f80ea22e469b66
+  version: 50955793b0183f9de69bd78e2ec251cf20aab121
   subpackages:
   - codes
   - credentials
@@ -198,8 +236,10 @@ imports:
   - internal
   - metadata
   - naming
-  - transport
   - peer
+  - stats
+  - tap
+  - transport
 - name: gopkg.in/fatih/set.v0
   version: 27c40922c40b43fe04554d8223a402af3ea333f3
 - name: gopkg.in/go-playground/validator.v8
@@ -208,26 +248,4 @@ imports:
   version: ecde8c8f16df93a994dda8936c8f60f0c26c28ab
 - name: gopkg.in/yaml.v2
   version: a83829b6f1293c91addabc89d0571c246397bbf4
-- name: github.com/go-kit/kit
-  version: f66b0e13579bfc5a48b9e2a94b1209c107ea1f41
-  subpackages:
-  - log
-- name: github.com/eapache/channels
-  version: 47238d5aae8c0fefd518ef2bee46290909cf8263
-- name: github.com/eapache/queue
-  version: 44cc805cf13205b55f69e14bcb69867d1ae92f98
-- name: github.com/go-logfmt/logfmt
-  version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5
-- name: github.com/go-stack/stack
-  version: 100eb0c0a9c5b306ca2fb4f165df21d80ada4b82
-- name: github.com/Sirupsen/logrus
-  version: d26492970760ca5d33129d2d799e34be5c4782eb
-- name: github.com/inconshreveable/log15
-  version: 46a701a619de90c65a78c04d1a58bf02585e9701
-  subpackages:
-  - term
-- name: github.com/streadway/simpleuuid
-  version: 6617b501e485b77e61b98cd533aefff9e258b5a7
-- name: github.com/Masterminds/glide
-  version: 84607742b10f492430762d038e954236bbaf23f7
-devImports: []
+testImports: []
diff --git a/glide.yaml b/glide.yaml
index d832bc2213309b201c8c52bda69fbb9e87ef52f5..ed8e097c2828278365c0e7d158acd11a5a9fb6f5 100644
--- a/glide.yaml
+++ b/glide.yaml
@@ -3,11 +3,12 @@ import:
 - package: github.com/eris-ltd/eris-keys
 - package: github.com/spf13/cobra
 - package: github.com/spf13/viper
-- package: github.com/tendermint/tendermint
 - package: github.com/gin-gonic/gin
 - package: github.com/gorilla/websocket
 - package: github.com/naoina/toml
 - package: github.com/stretchr/testify
+- package: github.com/tendermint/tendermint
+  version: ~0.8.0
 - package: github.com/tommy351/gin-cors
 - package: golang.org/x/crypto
   subpackages:
@@ -30,4 +31,4 @@ import:
   version: ^0.11.0
 - package: github.com/streadway/simpleuuid
 - package: github.com/Masterminds/glide
-  version: ~0.12.3
\ No newline at end of file
+  version: ~0.12.3
diff --git a/manager/eris-mint/eris-mint.go b/manager/eris-mint/eris-mint.go
index fe3948e12feb8a29536ea2865632012781255157..9ba18441832ecf6bce0b500ba99fca9bc41b3a40 100644
--- a/manager/eris-mint/eris-mint.go
+++ b/manager/eris-mint/eris-mint.go
@@ -20,9 +20,9 @@ import (
 	"sync"
 	"time"
 
+	abci "github.com/tendermint/abci/types"
 	tendermint_events "github.com/tendermint/go-events"
 	wire "github.com/tendermint/go-wire"
-	tmsp "github.com/tendermint/tmsp/types"
 
 	"github.com/eris-ltd/eris-db/logging"
 	"github.com/eris-ltd/eris-db/logging/loggers"
@@ -45,7 +45,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
@@ -55,9 +55,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()
@@ -72,7 +72,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),
@@ -84,8 +84,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
@@ -94,7 +94,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
@@ -104,45 +104,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()
 
@@ -173,10 +173,9 @@ func (app *ErisMint) Commit() (res tmsp.Result) {
 	// NOTE: set internal time as two seconds per block
 	app.state.LastBlockTime = app.state.LastBlockTime.Add(time.Duration(2) * time.Second)
 	appHash := app.state.Hash()
-	// return tmsp.NewResultOK(app.state.Hash(), "Success")
-	return tmsp.NewResultOK(appHash, "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/evm/snative.go b/manager/eris-mint/evm/snative.go
index c03d2bcd7dfab9515ec798454ef53dc6b1206831..b30858109c1356304b1d46394969c4f82df2479c 100644
--- a/manager/eris-mint/evm/snative.go
+++ b/manager/eris-mint/evm/snative.go
@@ -190,8 +190,8 @@ func SNativeContracts() map[string]*SNativeContractDescription {
 		if _, ok := contractMap[contract.Name]; ok {
 			// If this happens we have a pseudo compile time error that will be caught
 			// on native.go init()
-			panic(fmt.Errorf("Duplicate contract with name %s defined. " +
-					"Contract names must be unique.", contract.Name))
+			panic(fmt.Errorf("Duplicate contract with name %s defined. "+
+				"Contract names must be unique.", contract.Name))
 		}
 		contractMap[contract.Name] = contract
 	}
diff --git a/manager/eris-mint/evm/snative_test.go b/manager/eris-mint/evm/snative_test.go
index da0572b4535b42f6be66607ba29b1ce2e5fa8121..a60b62e15678596380769eb8880681517782102f 100644
--- a/manager/eris-mint/evm/snative_test.go
+++ b/manager/eris-mint/evm/snative_test.go
@@ -20,12 +20,12 @@ import (
 
 	"strings"
 
+	"github.com/eris-ltd/eris-db/manager/eris-mint/evm/abi"
 	. "github.com/eris-ltd/eris-db/manager/eris-mint/evm/opcodes"
 	"github.com/eris-ltd/eris-db/manager/eris-mint/evm/sha3"
 	ptypes "github.com/eris-ltd/eris-db/permission/types"
 	. "github.com/eris-ltd/eris-db/word256"
 	"github.com/stretchr/testify/assert"
-	"github.com/eris-ltd/eris-db/manager/eris-mint/evm/abi"
 )
 
 // Compiling the Permissions solidity contract at
diff --git a/manager/eris-mint/pipe.go b/manager/eris-mint/pipe.go
index a6e6dbe6a98c0ee359af0ff0cad315a37f4400c6..f900b3f09d07989dbe8bdc8d244b5a8e2f5966b1 100644
--- a/manager/eris-mint/pipe.go
+++ b/manager/eris-mint/pipe.go
@@ -18,12 +18,12 @@ import (
 	"bytes"
 	"fmt"
 
+	abci_types "github.com/tendermint/abci/types"
 	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"
@@ -69,7 +69,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,
@@ -142,8 +142,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)
 	}
@@ -532,7 +532,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 {
@@ -552,9 +552,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 {
@@ -562,7 +562,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
@@ -576,17 +576,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 6c2ec28da61f6bd4012f9bf47699f6b6aa33f686..734e5cea30254f5e501a9b2461644eced8033e84 100644
--- a/manager/eris-mint/version.go
+++ b/manager/eris-mint/version.go
@@ -34,8 +34,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 e7c1a76187e9229f5dcf2b6b15f697226a3e3076..d9a3b3d548044fbedbcc70ed292fd0eecaed1efe 100644
--- a/manager/manager.go
+++ b/manager/manager.go
@@ -34,7 +34,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 a7594f5c46e134119e8d697841a2b6d629ba006c..dbfe2b273bc5797b9ea9aec77e25b9c81e2b930a 100644
--- a/manager/types/application.go
+++ b/manager/types/application.go
@@ -15,10 +15,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
@@ -30,16 +30,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
@@ -49,18 +49,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
@@ -68,14 +68,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
@@ -83,7 +83,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
@@ -93,5 +93,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 7fcc7ebc326e683acf6330262c2e3fdac827271c..220ace4b49328834bd4393453f3055f03aed4880 100644
--- a/rpc/tendermint/core/types/responses.go
+++ b/rpc/tendermint/core/types/responses.go
@@ -22,11 +22,11 @@ import (
 	tendermint_types "github.com/tendermint/tendermint/types"
 
 	consensus_types "github.com/eris-ltd/eris-db/consensus/types"
+	abcitypes "github.com/tendermint/abci/types"
 	"github.com/tendermint/go-crypto"
 	"github.com/tendermint/go-p2p"
 	"github.com/tendermint/go-rpc/types"
 	"github.com/tendermint/go-wire"
-	tmsptypes "github.com/tendermint/tmsp/types"
 )
 
 type ResultGetStorage struct {
@@ -125,7 +125,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 94f55b7895306164a5c9f83b98080fab18d514b2..6bc85f4d053a2488357ebd9ff0e8843163637435 100644
--- a/rpc/tendermint/core/websocket.go
+++ b/rpc/tendermint/core/websocket.go
@@ -33,7 +33,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/common.go b/rpc/tendermint/test/common.go
index fa16978dab9baf9ac30206314ca89d0230d28faf..8151401f60dfa76dc6108753db7069f4dbdcbf82 100644
--- a/rpc/tendermint/test/common.go
+++ b/rpc/tendermint/test/common.go
@@ -1,5 +1,3 @@
-// +build integration
-
 // Space above here matters
 // Copyright 2017 Monax Industries Limited
 //
diff --git a/rpc/tendermint/test/config.go b/rpc/tendermint/test/config.go
index 96755d4276ef2c27f3f01ed2fada3503c7367fc6..84fcaeb7997c7b31cc45843d65a1fde9785963c9 100644
--- a/rpc/tendermint/test/config.go
+++ b/rpc/tendermint/test/config.go
@@ -52,12 +52,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"
 
@@ -135,21 +135,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
 ##
@@ -186,7 +184,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 0dcfad967b56a78d244f4834c812d39d3c54253a..d53ff9d713ffcee49a4646c7d303fd55a42a51ec 100644
--- a/rpc/tendermint/test/rpc_client_test.go
+++ b/rpc/tendermint/test/rpc_client_test.go
@@ -292,6 +292,8 @@ func TestNameReg(t *testing.T) {
 func TestBlockchainInfo(t *testing.T) {
 	wsc := newWSClient()
 	testWithAllClients(t, func(t *testing.T, clientName string, client rpcclient.Client) {
+		// wait a mimimal number of blocks to ensure that the later query for block
+		// headers has a non-trivial length
 		nBlocks := 4
 		waitNBlocks(t, wsc, nBlocks)
 
@@ -299,20 +301,25 @@ func TestBlockchainInfo(t *testing.T) {
 		if err != nil {
 			t.Fatalf("Failed to get blockchain info: %v", err)
 		}
-		//TODO: [Silas] reintroduce this when Tendermint changes logic to fire
-		// NewBlock after saving a block
-		// see https://github.com/tendermint/tendermint/issues/273
-		//assert.Equal(t, 4, resp.LastHeight, "Last height should be 4 after waiting for first 4 blocks")
+		lastBlockHeight := resp.LastHeight
+		nMetaBlocks := len(resp.BlockMetas)
+		assert.True(t, nMetaBlocks <= lastBlockHeight,
+			"Logically number of block metas should be equal or less than block height.")
 		assert.True(t, nBlocks <= len(resp.BlockMetas),
-			"Should see at least 4 BlockMetas after waiting for first 4 blocks")
-
-		lastBlockHash := resp.BlockMetas[nBlocks-1].Hash
-		for i := nBlocks - 2; i >= 0; i-- {
-			assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastBlockHash,
+			"Should see at least 4 BlockMetas after waiting for 4 blocks")
+		// For the maximum number (default to 20) of retrieved block headers,
+		// check that they correctly chain to each other.
+		lastBlockHash := resp.BlockMetas[nMetaBlocks-1].Hash
+		for i := nMetaBlocks - 2; i >= 0; i-- {
+			// the blockhash in header of height h should be identical to the hash
+			// in the LastBlockID of the header of block height h+1.
+			assert.Equal(t, lastBlockHash, resp.BlockMetas[i].Header.LastBlockID.Hash,
 				"Blockchain should be a hash tree!")
 			lastBlockHash = resp.BlockMetas[i].Hash
 		}
 
+		// Now retrieve only two blockheaders (h=1, and h=2) and check that we got
+		// two results.
 		resp, err = edbcli.BlockchainInfo(client, 1, 2)
 		assert.NoError(t, err)
 		assert.Equal(t, 2, len(resp.BlockMetas),
diff --git a/test/mock/pipe.go b/test/mock/pipe.go
index f208f324cdeca39aeb43a900a6ad177f93a4f1b8..c3c1f33558a3f0139c32af2260755370c676cc80 100644
--- a/test/mock/pipe.go
+++ b/test/mock/pipe.go
@@ -29,10 +29,10 @@ import (
 	"github.com/eris-ltd/eris-db/txs"
 
 	"github.com/eris-ltd/eris-db/logging/loggers"
+	abci_types "github.com/tendermint/abci/types"
 	"github.com/tendermint/go-crypto"
 	"github.com/tendermint/go-p2p"
 	mintTypes "github.com/tendermint/tendermint/types"
-	tmsp_types "github.com/tendermint/tmsp/types"
 )
 
 // Base struct.
@@ -179,7 +179,7 @@ type consensusEngine struct {
 }
 
 func (cons *consensusEngine) BroadcastTransaction(transaction []byte,
-	callback func(*tmsp_types.Response)) error {
+	callback func(*abci_types.Response)) error {
 	return nil
 }
 
diff --git a/util/hell/cmd/hell/main.go b/util/hell/cmd/hell/main.go
index 8fc417eec070d1a20a2f173fdbe81e5ac0d3717f..427105ba9a77e14ed02efe81e38d2b3f5d3b2739 100644
--- a/util/hell/cmd/hell/main.go
+++ b/util/hell/cmd/hell/main.go
@@ -124,7 +124,6 @@ func main() {
 			overrideLockFile.Imports = append(overrideLockFile.Imports, cfg.LockFromDependency(dep))
 
 			mergedLockFile, err := hell.MergeGlideLockFiles(baseLockFile, overrideLockFile)
-			fmt.Printf("%#v\n", mergedLockFile.Imports)
 			if err != nil {
 				msg.Die("Could not merge lock files: %s\n", err)
 			}
diff --git a/util/snatives/templates/solidity_templates.go b/util/snatives/templates/solidity_templates.go
index cc6cabd2d953766acb24de1a177c41ee4c30e3af..eda2c55588632d71677aeadd5c9017bcf05c0bde 100644
--- a/util/snatives/templates/solidity_templates.go
+++ b/util/snatives/templates/solidity_templates.go
@@ -157,12 +157,10 @@ func (function *solidityFunction) solidity(indentLevel uint) (string, error) {
 	return buf.String(), nil
 }
 
-
 //
-// Contract
+// Utility
 //
 
-
 func comment(comment string) string {
 	commentLines := make([]string, 0, 5)
 	for _, line := range strings.Split(comment, "\n") {