diff --git a/consensus/consensus.go b/consensus/consensus.go
index d821f551476146abdedcf461a591c4e3c5605cb0..5280406014f99e32e0c86a7647ccc39eb9ee0601 100644
--- a/consensus/consensus.go
+++ b/consensus/consensus.go
@@ -1,58 +1,5 @@
-// 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 consensus contains the logic maintaining and connecting to our Tendermint
+// consensus network. The job of the consensus module is to establish a total order
+// of transactions shared by all validators, which can then be executed over by the
+// execution module.
 package consensus
-
-import (
-	"fmt"
-
-	config "github.com/hyperledger/burrow/config"
-	tendermint "github.com/hyperledger/burrow/consensus/tendermint"
-	definitions "github.com/hyperledger/burrow/definitions"
-)
-
-func LoadConsensusEngineInPipe(moduleConfig *config.ModuleConfig,
-	pipe definitions.Pipe) error {
-
-	// Check interface-level compatibility
-	if !pipe.GetApplication().CompatibleConsensus(&tendermint.Tendermint{}) {
-		return fmt.Errorf("Manager Application %s it no compatible with "+
-			"%s consensus", moduleConfig.Name, pipe.GetApplication())
-	}
-
-	switch moduleConfig.Name {
-	case "tendermint":
-
-		tmint, err := tendermint.NewTendermint(moduleConfig, pipe.GetApplication(),
-			pipe.Logger())
-		if err != nil {
-			return fmt.Errorf("Failed to load Tendermint node: %v", err)
-		}
-
-		err = pipe.SetConsensusEngine(tmint)
-		if err != nil {
-			return fmt.Errorf("Failed to load Tendermint in pipe as "+
-				"ConsensusEngine: %v", err)
-		}
-
-		// For Tendermint we have a coupled Blockchain and ConsensusEngine
-		// implementation, so load it at the same time as ConsensusEngine
-		err = pipe.SetBlockchain(tmint)
-		if err != nil {
-			return fmt.Errorf("Failed to load Tendermint in pipe as "+
-				"Blockchain: %v", err)
-		}
-	}
-	return nil
-}
diff --git a/consensus/tendermint/abci/app.go b/consensus/tendermint/abci/app.go
new file mode 100644
index 0000000000000000000000000000000000000000..497e00f1ba6929e6bba106f413472d94757b2377
--- /dev/null
+++ b/consensus/tendermint/abci/app.go
@@ -0,0 +1,175 @@
+package abci
+
+import (
+	"fmt"
+	"sync"
+	"time"
+
+	bcm "github.com/hyperledger/burrow/blockchain"
+	"github.com/hyperledger/burrow/execution"
+	"github.com/hyperledger/burrow/logging"
+	"github.com/hyperledger/burrow/logging/structure"
+	logging_types "github.com/hyperledger/burrow/logging/types"
+	"github.com/hyperledger/burrow/txs"
+	"github.com/hyperledger/burrow/version"
+	abci_types "github.com/tendermint/abci/types"
+	"github.com/tendermint/go-wire"
+)
+
+const responseInfoName = "Burrow"
+
+type abciApp struct {
+	mtx sync.Mutex
+	// State
+	blockchain bcm.MutableBlockchain
+	checker    execution.BatchExecutor
+	committer  execution.BatchCommitter
+	// We need to cache these from BeginBlock for when we need actually need it in Commit
+	block *abci_types.RequestBeginBlock
+	// Utility
+	txDecoder txs.Decoder
+	// Logging
+	logger logging_types.InfoTraceLogger
+
+}
+
+func NewApp(blockchain bcm.MutableBlockchain,
+	checker execution.BatchExecutor,
+	committer execution.BatchCommitter,
+	logger logging_types.InfoTraceLogger) abci_types.Application {
+	return &abciApp{
+		blockchain: blockchain,
+		checker:    checker,
+		committer:  committer,
+		txDecoder:  txs.NewGoWireCodec(),
+		logger:     logging.WithScope(logger.With(structure.ComponentKey, "ABCI_App"), "abci.NewApp"),
+	}
+}
+
+func (app *abciApp) Info(info abci_types.RequestInfo) abci_types.ResponseInfo {
+	tip := app.blockchain.Tip()
+	return abci_types.ResponseInfo{
+		Data:             responseInfoName,
+		Version:          version.GetSemanticVersionString(),
+		LastBlockHeight:  tip.LastBlockHeight(),
+		LastBlockAppHash: tip.AppHashAfterLastBlock(),
+	}
+}
+
+func (app *abciApp) SetOption(key string, value string) string {
+	return "No options available"
+}
+
+func (app *abciApp) Query(reqQuery abci_types.RequestQuery) (respQuery abci_types.ResponseQuery) {
+	respQuery.Log = "Query not support"
+	respQuery.Code = abci_types.CodeType_UnknownRequest
+	return respQuery
+}
+
+func (app *abciApp) CheckTx(txBytes []byte) abci_types.Result {
+	app.mtx.Lock()
+	defer app.mtx.Unlock()
+	tx, err := app.txDecoder.DecodeTx(txBytes)
+	if err != nil {
+		logging.TraceMsg(app.logger, "CheckTx decoding error",
+			"error", err)
+		return abci_types.NewError(abci_types.CodeType_EncodingError, fmt.Sprintf("Encoding error: %v", err))
+	}
+	// TODO: map ExecTx errors to sensible ABCI error codes
+	receipt := txs.GenerateReceipt(app.blockchain.ChainID(), tx)
+
+	err = app.checker.Execute(tx)
+	if err != nil {
+		logging.TraceMsg(app.logger, "CheckTx execution error",
+			"error", err,
+			"tx_hash", receipt.TxHash,
+			"creates_contract", receipt.CreatesContract)
+		return abci_types.NewError(abci_types.CodeType_InternalError,
+			fmt.Sprintf("Could not execute transaction: %s, error: %v", tx, err))
+	}
+
+	receiptBytes := wire.BinaryBytes(receipt)
+	logging.TraceMsg(app.logger, "CheckTx success",
+		"tx_hash", receipt.TxHash,
+		"creates_contract", receipt.CreatesContract)
+	return abci_types.NewResultOK(receiptBytes, "Success")
+}
+
+func (app *abciApp) InitChain(chain abci_types.RequestInitChain) {
+	// Could verify agreement on initial validator set here
+}
+
+func (app *abciApp) BeginBlock(block abci_types.RequestBeginBlock) {
+	app.block = &block
+}
+
+func (app *abciApp) DeliverTx(txBytes []byte) abci_types.Result {
+	app.mtx.Lock()
+	defer app.mtx.Unlock()
+	tx, err := app.txDecoder.DecodeTx(txBytes)
+	if err != nil {
+		logging.TraceMsg(app.logger, "DeliverTx decoding error",
+			"error", err)
+		return abci_types.NewError(abci_types.CodeType_EncodingError, fmt.Sprintf("Encoding error: %s", err))
+	}
+
+	receipt := txs.GenerateReceipt(app.blockchain.ChainID(), tx)
+	err = app.committer.Execute(tx)
+	if err != nil {
+		logging.TraceMsg(app.logger, "DeliverTx execution error",
+			"error", err,
+			"tx_hash", receipt.TxHash,
+			"creates_contract", receipt.CreatesContract)
+		return abci_types.NewError(abci_types.CodeType_InternalError,
+			fmt.Sprintf("Could not execute transaction: %s, error: %s", tx, err))
+	}
+
+	logging.TraceMsg(app.logger, "DeliverTx success",
+		"tx_hash", receipt.TxHash,
+		"creates_contract", receipt.CreatesContract)
+	receiptBytes := wire.BinaryBytes(receipt)
+	return abci_types.NewResultOK(receiptBytes, "Success")
+}
+
+func (app *abciApp) EndBlock(height uint64) (respEndBlock abci_types.ResponseEndBlock) {
+	return respEndBlock
+}
+
+func (app *abciApp) Commit() abci_types.Result {
+	app.mtx.Lock()
+	defer app.mtx.Unlock()
+	tip := app.blockchain.Tip()
+	logging.InfoMsg(app.logger, "Committing block",
+		structure.ScopeKey, "Commit()",
+		"block_height", tip.LastBlockHeight(),
+		"block_hash", app.block.Hash,
+		"block_time", app.block.Header.Time,
+		"num_txs", app.block.Header.NumTxs,
+		"last_block_time", tip.LastBlockTime(),
+		"last_block_hash", tip.LastBlockHash())
+
+	appHash, err := app.committer.Commit()
+	if err != nil {
+		return abci_types.NewError(abci_types.CodeType_InternalError,
+			fmt.Sprintf("Could not commit block: %s", err))
+	}
+
+	logging.InfoMsg(app.logger, "Resetting transaction check cache")
+	app.checker.Reset()
+
+	// Commit to our blockchain state
+	app.blockchain.CommitBlock(time.Unix(int64(app.block.Header.Time), 0), app.block.Hash, appHash)
+
+	// Perform a sanity check our block height
+	if app.blockchain.LastBlockHeight() != app.block.Header.Height {
+		logging.InfoMsg(app.logger, "Burrow block height disagrees with Tendermint block height",
+			structure.ScopeKey, "Commit()",
+			"burrow_height", app.blockchain.LastBlockHeight(),
+			"tendermint_height", app.block.Header.Height)
+		return abci_types.NewError(abci_types.CodeType_InternalError,
+			fmt.Sprintf("Burrow has recorded a block height of %v, "+
+				"but Tendermint reports a block height of %v, and the two should agree.",
+				app.blockchain.LastBlockHeight(), app.block.Header.Height))
+	}
+	return abci_types.NewResultOK(appHash, "Success")
+}
diff --git a/consensus/tendermint/config.go b/consensus/tendermint/config.go
index b73a0b4f71ac424ba14b83320f3e8681015fd4c6..e1c898188577ad93910ac8577c839810274cc4d7 100644
--- a/consensus/tendermint/config.go
+++ b/consensus/tendermint/config.go
@@ -1,177 +1,42 @@
-// 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
 
 import (
 	"path"
-	"time"
-
-	"github.com/spf13/viper"
-	tendermintConfig "github.com/tendermint/go-config"
 
-	"github.com/hyperledger/burrow/config"
+	tm_config "github.com/tendermint/tendermint/config"
 )
 
-// NOTE [ben] Compiler check to ensure TendermintConfig successfully implements
-// tendermint/go-config/config.Config
-var _ tendermintConfig.Config = (*TendermintConfig)(nil)
-
-// Tendermint has a self-rolled configuration type defined
-// in tendermint/go-config but over an interface type, which is implemented
-// by default in tendermint/tendermint/config/tendermint.go
-// However, for burrow purposes we can choose different rules for how to load
-// the tendermint configuration and set the defaults.  Hence we re-implement
-// go-config.Config on a viper subtree of the loaded burrow configuration file.
-type TendermintConfig struct {
-	subTree *viper.Viper
-}
-
-func GetTendermintConfig(loadedConfig *viper.Viper) *TendermintConfig {
-	// ensure we make an explicit copy
-	subTree := new(viper.Viper)
-	*subTree = *loadedConfig
-
-	return &TendermintConfig{
-		subTree: subTree,
+// Burrow's view on Tendermint's config. Since we operate as a Tendermint harness not all configuration values
+// are applicable, we may not allow some values to specified, or we may not allow some to be set independently.
+// So this serves as a layer of indirection over Tendermint's real config that we derive from ours.
+type BurrowTendermintConfig struct {
+	Seeds                string
+	ListenAddress        string
+	Moniker              string
+	TendermintRoot       string
+}
+
+func DefaultBurrowTendermintConfig() *BurrowTendermintConfig {
+	tmDefaultConfig := tm_config.DefaultConfig()
+	return &BurrowTendermintConfig{
+		ListenAddress:        tmDefaultConfig.P2P.ListenAddress,
+		TendermintRoot:       ".burrow",
 	}
 }
 
-//------------------------------------------------------------------------------
-// Tendermint defaults
-
-//
-// Contract
-//
-
-func (tmintConfig *TendermintConfig) AssertTendermintDefaults(chainId, workDir,
-	dataDir, rootDir string) {
-
-	tmintConfig.Set("chain_id", chainId)
-	tmintConfig.SetDefault("genesis_file", path.Join(rootDir, "genesis.json"))
-	tmintConfig.SetDefault("proxy_app", "tcp://127.0.0.1:46658")
-	tmintConfig.SetDefault("moniker", "anonymous_marmot")
-	tmintConfig.SetDefault("node_laddr", "0.0.0.0:46656")
-	tmintConfig.SetDefault("seeds", "")
-
-	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)
-	tmintConfig.SetDefault("log_level", "info")
-	tmintConfig.SetDefault("rpc_laddr", "")
-	tmintConfig.SetDefault("prof_laddr", "")
-	tmintConfig.SetDefault("revision_file", path.Join(workDir, "revision"))
-	tmintConfig.SetDefault("cs_wal_file", path.Join(dataDir, "cs.wal/wal"))
-	tmintConfig.SetDefault("cs_wal_light", false)
-	tmintConfig.SetDefault("filter_peers", false)
-
-	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)
-	tmintConfig.SetDefault("timeout_prevote", 1000)
-	tmintConfig.SetDefault("timeout_prevote_delta", 500)
-	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"))
-}
-
-//------------------------------------------------------------------------------
-// Tendermint consistency checks
-
-func (tmintConfig *TendermintConfig) AssertTendermintConsistency(
-	consensusConfig *config.ModuleConfig, privateValidatorFilePath string) {
-
-	tmintConfig.Set("chain_id", consensusConfig.ChainId)
-	tmintConfig.Set("genesis_file", consensusConfig.GenesisFile)
-	// private validator file
-	tmintConfig.Set("priv_validator_file", privateValidatorFilePath)
-
-}
-
-// implement interface github.com/tendermint/go-config/config.Config
-// so that `TMROOT` and config can be circumvented
-func (tmintConfig *TendermintConfig) Get(key string) interface{} {
-	return tmintConfig.subTree.Get(key)
-}
-
-func (tmintConfig *TendermintConfig) GetBool(key string) bool {
-	return tmintConfig.subTree.GetBool(key)
-}
-
-func (tmintConfig *TendermintConfig) GetFloat64(key string) float64 {
-	return tmintConfig.subTree.GetFloat64(key)
-}
-
-func (tmintConfig *TendermintConfig) GetInt(key string) int {
-	return tmintConfig.subTree.GetInt(key)
-}
-
-func (tmintConfig *TendermintConfig) GetString(key string) string {
-	return tmintConfig.subTree.GetString(key)
-}
-
-func (tmintConfig *TendermintConfig) GetStringSlice(key string) []string {
-	return tmintConfig.subTree.GetStringSlice(key)
-}
-
-func (tmintConfig *TendermintConfig) GetTime(key string) time.Time {
-	return tmintConfig.subTree.GetTime(key)
-}
-
-func (tmintConfig *TendermintConfig) GetMap(key string) map[string]interface{} {
-	return tmintConfig.subTree.GetStringMap(key)
-}
-
-func (tmintConfig *TendermintConfig) GetMapString(key string) map[string]string {
-	return tmintConfig.subTree.GetStringMapString(key)
-}
-
-func (tmintConfig *TendermintConfig) GetConfig(key string) tendermintConfig.Config {
-	// TODO: [ben] log out a warning as this indicates a potentially breaking code
-	// change from Tendermints side
-	subTree, _ := config.ViperSubConfig(tmintConfig.subTree, key)
-	if subTree == nil {
-		return &TendermintConfig{
-			subTree: viper.New(),
-		}
+func (btc *BurrowTendermintConfig) TendermintConfig() *tm_config.Config {
+	conf := tm_config.DefaultConfig()
+	if btc != nil {
+		// We may need to expose more of the P2P/Consensus/Mempool options, but I'd like to keep the configuration
+		// minimal
+		conf.P2P.Seeds = btc.Seeds
+		conf.P2P.ListenAddress = btc.ListenAddress
+		conf.Moniker = btc.Moniker
+		conf.DBPath = path.Join(btc.TendermintRoot, conf.DBPath)
+		conf.Mempool.WalPath = path.Join(btc.TendermintRoot, conf.Mempool.WalPath)
+		conf.Consensus.WalPath = path.Join(btc.TendermintRoot, conf.Consensus.WalPath)
 	}
-	return &TendermintConfig{
-		subTree: tmintConfig.subTree.Sub(key),
-	}
-}
-
-func (tmintConfig *TendermintConfig) IsSet(key string) bool {
-	return tmintConfig.IsSet(key)
-}
-
-func (tmintConfig *TendermintConfig) Set(key string, value interface{}) {
-	tmintConfig.subTree.Set(key, value)
-}
-
-func (tmintConfig *TendermintConfig) SetDefault(key string, value interface{}) {
-	tmintConfig.subTree.SetDefault(key, value)
+	// Disable Tendermint RPC
+	conf.RPC.ListenAddress = ""
+	return conf
 }
diff --git a/consensus/tendermint/logger.go b/consensus/tendermint/logger.go
new file mode 100644
index 0000000000000000000000000000000000000000..c42f46a4ca679890b6f67c5fd3f3978949bf0f8e
--- /dev/null
+++ b/consensus/tendermint/logger.go
@@ -0,0 +1,35 @@
+package tendermint
+
+import (
+	"github.com/hyperledger/burrow/logging"
+	logging_types "github.com/hyperledger/burrow/logging/types"
+	"github.com/tendermint/tmlibs/log"
+)
+
+type tendermintLogger struct {
+	logger logging_types.InfoTraceLogger
+}
+
+func NewLogger(logger logging_types.InfoTraceLogger) *tendermintLogger {
+	return &tendermintLogger{
+		logger: logger,
+	}
+}
+
+func (tml *tendermintLogger) Info(msg string, keyvals ...interface{}) {
+	logging.InfoMsg(tml.logger, msg, keyvals...)
+}
+
+func (tml *tendermintLogger) Error(msg string, keyvals ...interface{}) {
+	logging.InfoMsg(tml.logger, msg, keyvals...)
+}
+
+func (tml *tendermintLogger) Debug(msg string, keyvals ...interface{}) {
+	logging.TraceMsg(tml.logger, msg, keyvals...)
+}
+
+func (tml *tendermintLogger) With(keyvals ...interface{}) log.Logger {
+	return &tendermintLogger{
+		logger: tml.logger.With(keyvals...),
+	}
+}
diff --git a/consensus/tendermint/query/node_view.go b/consensus/tendermint/query/node_view.go
new file mode 100644
index 0000000000000000000000000000000000000000..69949436a27d1966971869927d326d28da5eb33b
--- /dev/null
+++ b/consensus/tendermint/query/node_view.go
@@ -0,0 +1,101 @@
+package query
+
+import (
+	"fmt"
+
+	acm "github.com/hyperledger/burrow/account"
+	"github.com/hyperledger/burrow/txs"
+	"github.com/tendermint/tendermint/consensus"
+	ctypes "github.com/tendermint/tendermint/consensus/types"
+	"github.com/tendermint/tendermint/node"
+	"github.com/tendermint/tendermint/p2p"
+	"github.com/tendermint/tendermint/types"
+)
+
+// You're like the interface I never had
+type NodeView interface {
+	// PrivValidator public key
+	PrivValidatorPublicKey() acm.PublicKey
+	// NodeInfo for this node broadcast to other nodes (including ephemeral STS ED25519 public key)
+	NodeInfo() *p2p.NodeInfo
+	// Whether the Tendermint node is listening
+	IsListening() bool
+	// Current listeners
+	Listeners() []p2p.Listener
+	// Known Tendermint peers
+	Peers() p2p.IPeerSet
+	// Read-only BlockStore
+	BlockStore() types.BlockStoreRPC
+	// Get the currently unconfirmed but not known to be invalid transactions from the Node's mempool
+	MempoolTransactions(maxTxs int) ([]txs.Tx, error)
+	// Get the validator's consensus RoundState
+	RoundState() *ctypes.RoundState
+	// Get the validator's peer's consensus RoundState
+	PeerRoundStates() ([]*ctypes.PeerRoundState, error)
+}
+
+type nodeView struct {
+	tmNode    *node.Node
+	txDecoder txs.Decoder
+}
+
+func NewNodeView(tmNode *node.Node, txDecoder txs.Decoder) NodeView {
+	return &nodeView{
+		tmNode:    tmNode,
+		txDecoder: txDecoder,
+	}
+}
+
+func (nv *nodeView) PrivValidatorPublicKey() acm.PublicKey {
+	return acm.PublicKeyFromPubKey(nv.tmNode.PrivValidator().GetPubKey())
+}
+
+func (nv *nodeView) NodeInfo() *p2p.NodeInfo {
+	return nv.tmNode.NodeInfo()
+}
+
+func (nv *nodeView) IsListening() bool {
+	return nv.tmNode.Switch().IsListening()
+}
+
+func (nv *nodeView) Listeners() []p2p.Listener {
+	return nv.tmNode.Switch().Listeners()
+}
+
+func (nv *nodeView) Peers() p2p.IPeerSet {
+	return nv.tmNode.Switch().Peers()
+}
+
+func (nv *nodeView) BlockStore() types.BlockStoreRPC {
+	return nv.tmNode.BlockStore()
+}
+
+// Pass -1 to get all available transactions
+func (nv *nodeView) MempoolTransactions(maxTxs int) ([]txs.Tx, error) {
+	var transactions []txs.Tx
+	for _, txBytes := range nv.tmNode.MempoolReactor().Mempool.Reap(maxTxs) {
+		tx, err := nv.txDecoder.DecodeTx(txBytes)
+		if err != nil {
+			return nil, err
+		}
+		transactions = append(transactions, tx)
+	}
+	return transactions, nil
+}
+
+func (nv *nodeView) RoundState() *ctypes.RoundState {
+	return nv.tmNode.ConsensusState().GetRoundState()
+}
+
+func (nv *nodeView) PeerRoundStates() ([]*ctypes.PeerRoundState, error) {
+	peers := nv.tmNode.Switch().Peers().List()
+	peerRoundStates := make([]*ctypes.PeerRoundState, len(peers))
+	for i, peer := range peers {
+		peerState, ok := peer.Get(types.PeerStateKey).(*consensus.PeerState)
+		if !ok {
+			return nil, fmt.Errorf("could not get PeerState for peer: %s", peer)
+		}
+		peerRoundStates[i] = peerState.GetRoundState()
+	}
+	return peerRoundStates, nil
+}
diff --git a/consensus/tendermint/tendermint.go b/consensus/tendermint/tendermint.go
index a35337cc3c4ac88f7c52934f0a51036723d4f625..8f38c5befd734d4676e0aec54d3ab209c7e9480a 100644
--- a/consensus/tendermint/tendermint.go
+++ b/consensus/tendermint/tendermint.go
@@ -1,288 +1,75 @@
-// 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
 
 import (
 	"fmt"
-	"path"
-	"strings"
-
-	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"
-
-	edb_event "github.com/hyperledger/burrow/event"
 
-	config "github.com/hyperledger/burrow/config"
-	manager_types "github.com/hyperledger/burrow/manager/types"
-	// files  "github.com/hyperledger/burrow/files"
-	"errors"
-
-	blockchain_types "github.com/hyperledger/burrow/blockchain/types"
-	consensus_types "github.com/hyperledger/burrow/consensus/types"
-	"github.com/hyperledger/burrow/logging"
+	bcm "github.com/hyperledger/burrow/blockchain"
+	"github.com/hyperledger/burrow/consensus/tendermint/abci"
+	"github.com/hyperledger/burrow/execution"
+	"github.com/hyperledger/burrow/genesis"
+	"github.com/hyperledger/burrow/logging/structure"
 	logging_types "github.com/hyperledger/burrow/logging/types"
 	"github.com/hyperledger/burrow/txs"
-	"github.com/tendermint/go-wire"
+	abci_types "github.com/tendermint/abci/types"
+	"github.com/tendermint/tendermint/config"
+	"github.com/tendermint/tendermint/node"
+	"github.com/tendermint/tendermint/proxy"
+	tm_types "github.com/tendermint/tendermint/types"
 )
 
-type Tendermint struct {
-	tmintNode   *node.Node
-	tmintConfig *TendermintConfig
-	chainId     string
-	logger      logging_types.InfoTraceLogger
-}
-
-// Compiler checks to ensure Tendermint successfully implements
-// burrow/definitions Consensus and Blockchain
-var _ consensus_types.ConsensusEngine = (*Tendermint)(nil)
-var _ blockchain_types.Blockchain = (*Tendermint)(nil)
-
-func NewTendermint(moduleConfig *config.ModuleConfig,
-	application manager_types.Application,
-	logger logging_types.InfoTraceLogger) (*Tendermint, error) {
-	// loading the module has ensured the working and data directory
-	// for tendermint have been created, but the config files needs
-	// to be written in tendermint's root directory.
-	// NOTE: [ben] as elsewhere Sub panics if config file does not have this
-	// subtree. To shield in go-routine, or PR to viper.
-	if !moduleConfig.Config.IsSet("configuration") {
-		return nil, fmt.Errorf("Failed to extract Tendermint configuration subtree.")
-	}
-	tendermintConfigViper, err := config.ViperSubConfig(moduleConfig.Config, "configuration")
-	if tendermintConfigViper == nil {
-		return nil,
-			fmt.Errorf("Failed to extract Tendermint configuration subtree: %s", err)
-	}
-	// wrap a copy of the viper config in a tendermint/go-config interface
-	tmintConfig := GetTendermintConfig(tendermintConfigViper)
-	// complete the tendermint configuration with default flags
-	tmintConfig.AssertTendermintDefaults(moduleConfig.ChainId,
-		moduleConfig.WorkDir, moduleConfig.DataDir, moduleConfig.RootDir)
-
-	privateValidatorFilePath := path.Join(moduleConfig.RootDir,
-		moduleConfig.Config.GetString("private_validator_file"))
-	if moduleConfig.Config.GetString("private_validator_file") == "" {
-		return nil, fmt.Errorf("No private validator file provided.")
-	}
-	// override tendermint configurations to force consistency with overruling
-	// settings
-	tmintConfig.AssertTendermintConsistency(moduleConfig,
-		privateValidatorFilePath)
-	chainId := tmintConfig.GetString("chain_id")
-
-	logging.TraceMsg(logger, "Loaded Tendermint sub-configuration",
-		"chainId", chainId,
-		"genesisFile", tmintConfig.GetString("genesis_file"),
-		"nodeLocalAddress", tmintConfig.GetString("node_laddr"),
-		"moniker", tmintConfig.GetString("moniker"),
-		"seeds", tmintConfig.GetString("seeds"),
-		"fastSync", tmintConfig.GetBool("fast_sync"),
-		"rpcLocalAddress", tmintConfig.GetString("rpc_laddr"),
-		"databaseDirectory", tmintConfig.GetString("db_dir"),
-		"privateValidatorFile", tmintConfig.GetString("priv_validator_file"),
-		"privValFile", moduleConfig.Config.GetString("private_validator_file"))
-
-	// TODO: [ben] do not "or Generate Validator keys", rather fail directly
-	// TODO: [ben] implement the signer for Private validator over monax-keys
-	// TODO: [ben] copy from rootDir to tendermint workingDir;
-	privateValidator := tendermint_types.LoadOrGenPrivValidator(
-		path.Join(moduleConfig.RootDir,
-			moduleConfig.Config.GetString("private_validator_file")))
-
-	// TODO: [Silas] we want to something better than this like not not have it in
-	// the config at all, but for now I think it's much safer to make sure we are
-	// not running the tendermint RPC as it could lead to unexpected behaviour,
-	// not least if we accidentally try to run it on the same address as our own
-	if tmintConfig.GetString("rpc_laddr") != "" {
-		logging.InfoMsg(logger, "Force disabling Tendermint's native RPC",
-			"provided_rpc_laddr", tmintConfig.GetString("rpc_laddr"))
-		tmintConfig.Set("rpc_laddr", "")
-	}
-
-	newNode := node.NewNode(tmintConfig, privateValidator,
-		proxy.NewLocalClientCreator(application))
-
-	// TODO: [ben] delay starting the node to a different function, to hand
-	// control over events to Core
-	if started, err := newNode.Start(); !started {
-		newNode.Stop()
+func NewNode(
+	conf *config.Config,
+	privValidator tm_types.PrivValidator,
+	genesisDoc *tm_types.GenesisDoc,
+	blockchain bcm.MutableBlockchain,
+	checker execution.BatchExecutor,
+	committer execution.BatchCommitter,
+	logger logging_types.InfoTraceLogger) (*node.Node, error) {
+
+	// disable Tendermint's RPC
+	conf.RPC.ListenAddress = ""
+
+	app := abci.NewApp(blockchain, checker, committer, logger)
+	return node.NewNode(conf, privValidator,
+		proxy.NewLocalClientCreator(app),
+		func() (*tm_types.GenesisDoc, error) {
+			return genesisDoc, nil
+		},
+		node.DefaultDBProvider,
+		NewLogger(logger.WithPrefix(structure.ComponentKey, "Tendermint").
+			With(structure.ScopeKey, "tendermint.NewNode")))
+}
+
+func BroadcastTxAsyncFunc(validator *node.Node, txEncoder txs.Encoder) func(tx txs.Tx,
+	callback func(res *abci_types.Response)) error {
+
+	return func(tx txs.Tx, callback func(res *abci_types.Response)) error {
+		txBytes, err := txEncoder.EncodeTx(tx)
 		if err != nil {
-			return nil, fmt.Errorf("Failed to start Tendermint consensus node: %v", err)
+			return fmt.Errorf("error encoding transaction: %v", err)
 		}
-		return nil, errors.New("Failed to start Tendermint consensus node, " +
-			"probably because it is already started, see logs")
-
-	}
-	logging.InfoMsg(logger, "Tendermint consensus node started",
-		"nodeAddress", tmintConfig.GetString("node_laddr"),
-		"transportProtocol", "tcp",
-		"upnp", !tmintConfig.GetBool("skip_upnp"),
-		"moniker", tmintConfig.GetString("moniker"))
-
-	// If seedNode is provided by config, dial out.
-	if tmintConfig.GetString("seeds") != "" {
-		seeds := strings.Split(tmintConfig.GetString("seeds"), ",")
-		newNode.DialSeeds(seeds)
-		logging.TraceMsg(logger, "Tendermint node called seeds",
-			"seeds", seeds)
-	}
-
-	return &Tendermint{
-		tmintNode:   newNode,
-		tmintConfig: tmintConfig,
-		chainId:     chainId,
-		logger:      logger,
-	}, nil
-}
-
-//------------------------------------------------------------------------------
-// Blockchain implementation
-
-func (tendermint *Tendermint) Height() int {
-	return tendermint.tmintNode.BlockStore().Height()
-}
-
-func (tendermint *Tendermint) BlockMeta(height int) *tendermint_types.BlockMeta {
-	return tendermint.tmintNode.BlockStore().LoadBlockMeta(height)
-}
 
-func (tendermint *Tendermint) Block(height int) *tendermint_types.Block {
-	return tendermint.tmintNode.BlockStore().LoadBlock(height)
-}
-
-func (tendermint *Tendermint) ChainId() string {
-	return tendermint.chainId
-}
-
-// Consensus implementation
-func (tendermint *Tendermint) IsListening() bool {
-	return tendermint.tmintNode.Switch().IsListening()
-}
-
-func (tendermint *Tendermint) Listeners() []p2p.Listener {
-	var copyListeners []p2p.Listener
-	// copy slice of Listeners
-	copy(copyListeners[:], tendermint.tmintNode.Switch().Listeners())
-	return copyListeners
-}
-
-func (tendermint *Tendermint) Peers() []*consensus_types.Peer {
-	p2pPeers := tendermint.tmintNode.Switch().Peers().List()
-	peers := make([]*consensus_types.Peer, 0)
-	for _, peer := range p2pPeers {
-		peers = append(peers, &consensus_types.Peer{
-			NodeInfo:   peer.NodeInfo,
-			IsOutbound: peer.IsOutbound(),
-		})
-	}
-	return peers
-}
-
-func (tendermint *Tendermint) NodeInfo() *p2p.NodeInfo {
-	var copyNodeInfo = new(p2p.NodeInfo)
-	// call Switch().NodeInfo is not go-routine safe, so copy
-	*copyNodeInfo = *tendermint.tmintNode.Switch().NodeInfo()
-	tendermint.tmintNode.ConsensusState().GetRoundState()
-	return copyNodeInfo
-}
-
-func (tendermint *Tendermint) PublicValidatorKey() crypto.PubKey {
-	// TODO: [ben] this is abetment, not yet a go-routine safe solution
-	var copyPublicValidatorKey crypto.PubKey
-	// crypto.PubKey is an interface so copy underlying struct
-	publicKey := tendermint.tmintNode.PrivValidator().PubKey
-	switch publicKey.(type) {
-	case crypto.PubKeyEd25519:
-		// type crypto.PubKeyEd25519 is [32]byte
-		copyKeyBytes := publicKey.(crypto.PubKeyEd25519)
-		copyPublicValidatorKey = crypto.PubKey(copyKeyBytes)
-	default:
-		// TODO: [ben] add error return to all these calls
-		copyPublicValidatorKey = nil
-	}
-	return copyPublicValidatorKey
-}
-
-func (tendermint *Tendermint) Events() edb_event.EventEmitter {
-	return edb_event.NewEvents(tendermint.tmintNode.EventSwitch(), tendermint.logger)
-}
-
-func (tendermint *Tendermint) BroadcastTransaction(transaction []byte,
-	callback func(*abci_types.Response)) error {
-	return tendermint.tmintNode.MempoolReactor().BroadcastTx(transaction, callback)
-}
-
-func (tendermint *Tendermint) ListUnconfirmedTxs(
-	maxTxs int) ([]txs.Tx, error) {
-	tendermintTxs := tendermint.tmintNode.MempoolReactor().Mempool.Reap(maxTxs)
-	transactions := make([]txs.Tx, len(tendermintTxs))
-	for i, txBytes := range tendermintTxs {
-		tx, err := txs.DecodeTx(txBytes)
+		err = validator.MempoolReactor().BroadcastTx(txBytes, callback)
 		if err != nil {
-			return nil, err
+			return fmt.Errorf("error broadcasting transaction: %v", err)
 		}
-		transactions[i] = tx
+		return nil
 	}
-	return transactions, nil
-}
-
-func (tendermint *Tendermint) ListValidators() []consensus_types.Validator {
-	return consensus_types.FromTendermintValidators(tendermint.tmintNode.
-		ConsensusState().Validators.Validators)
 }
 
-func (tendermint *Tendermint) ConsensusState() *consensus_types.ConsensusState {
-	return consensus_types.FromRoundState(tendermint.tmintNode.ConsensusState().
-		GetRoundState())
-}
-
-func (tendermint *Tendermint) PeerConsensusStates() map[string]string {
-	peers := tendermint.tmintNode.Switch().Peers().List()
-	peerConsensusStates := make(map[string]string,
-		len(peers))
-	for _, peer := range peers {
-		peerState := peer.Data.Get(tendermint_types.PeerStateKey).(*tendermint_consensus.PeerState)
-		peerRoundState := peerState.GetRoundState()
-		// TODO: implement a proper mapping, this is a nasty way of marshalling
-		// to JSON
-		peerConsensusStates[peer.Key] = string(wire.JSONBytes(peerRoundState))
+func DeriveGenesisDoc(burrowGenesisDoc *genesis.GenesisDoc) *tm_types.GenesisDoc {
+	validators := make([]tm_types.GenesisValidator, len(burrowGenesisDoc.Validators))
+	for i, validator := range burrowGenesisDoc.Validators {
+		validators[i] = tm_types.GenesisValidator{
+			PubKey: validator.PublicKey.PubKey,
+			Name:   validator.Name,
+			Power:  int64(validator.Amount),
+		}
+	}
+	return &tm_types.GenesisDoc{
+		ChainID:     burrowGenesisDoc.ChainID(),
+		GenesisTime: burrowGenesisDoc.GenesisTime,
+		Validators:  validators,
+		AppHash:     burrowGenesisDoc.Hash(),
 	}
-	return peerConsensusStates
-}
-
-// Allow for graceful shutdown of node. Returns whether the node was stopped.
-func (tendermint *Tendermint) Stop() bool {
-	return tendermint.tmintNode.Stop()
 }
-
-//------------------------------------------------------------------------------
-// Helper functions
-
-// func marshalConfigToDisk(filePath string, tendermintConfig *viper.Viper) error {
-//
-//   tendermintConfig.Unmarshal
-//   // marshal interface to toml bytes
-//   bytesConfig, err := toml.Marshal(tendermintConfig)
-//   if err != nil {
-//     return fmt.Fatalf("Failed to marshal Tendermint configuration to bytes: %v",
-//       err)
-//   }
-//   return files.WriteAndBackup(filePath, bytesConfig)
-// }
diff --git a/consensus/tendermint/validator/priv_validator_memory.go b/consensus/tendermint/validator/priv_validator_memory.go
new file mode 100644
index 0000000000000000000000000000000000000000..f8581144758dcdd0f519f26c4d777d8cc471c0df
--- /dev/null
+++ b/consensus/tendermint/validator/priv_validator_memory.go
@@ -0,0 +1,46 @@
+package validator
+
+import (
+	acm "github.com/hyperledger/burrow/account"
+	"github.com/tendermint/go-crypto"
+	"github.com/tendermint/go-wire/data"
+	tm_types "github.com/tendermint/tendermint/types"
+)
+
+type privValidatorMemory struct {
+	acm.Addressable
+	acm.Signer
+	lastSignedInfo *LastSignedInfo
+}
+
+var _ tm_types.PrivValidator = &privValidatorMemory{}
+
+// Create a PrivValidator with in-memory state that takes an addressable representing the validator identity
+// and a signer providing private signing for that identity.
+func NewPrivValidatorMemory(addressable acm.Addressable, signer acm.Signer) *privValidatorMemory {
+	return &privValidatorMemory{
+		Addressable:    addressable,
+		Signer:         signer,
+		lastSignedInfo: new(LastSignedInfo),
+	}
+}
+
+func (pvm *privValidatorMemory) GetAddress() data.Bytes {
+	return pvm.Address().Bytes()
+}
+
+func (pvm *privValidatorMemory) GetPubKey() crypto.PubKey {
+	return pvm.PublicKey().PubKey
+}
+
+func (pvm *privValidatorMemory) SignVote(chainID string, vote *tm_types.Vote) error {
+	return pvm.lastSignedInfo.SignVote(pvm, chainID, vote)
+}
+
+func (pvm *privValidatorMemory) SignProposal(chainID string, proposal *tm_types.Proposal) error {
+	return pvm.lastSignedInfo.SignProposal(pvm, chainID, proposal)
+}
+
+func (pvm *privValidatorMemory) SignHeartbeat(chainID string, heartbeat *tm_types.Heartbeat) error {
+	return pvm.lastSignedInfo.SignHeartbeat(pvm, chainID, heartbeat)
+}
diff --git a/consensus/tendermint/validator/verifier.go b/consensus/tendermint/validator/verifier.go
new file mode 100644
index 0000000000000000000000000000000000000000..20d174dd6667aa9671d880f2d5aa784adc26bf55
--- /dev/null
+++ b/consensus/tendermint/validator/verifier.go
@@ -0,0 +1,143 @@
+package validator
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"sync"
+
+	"github.com/tendermint/go-crypto"
+	"github.com/tendermint/go-wire/data"
+	tm_types "github.com/tendermint/tendermint/types"
+)
+
+const (
+	StepError     = -1
+	StepNone      = 0 // Used to distinguish the initial state
+	StepPropose   = 1
+	StepPrevote   = 2
+	StepPrecommit = 3
+)
+
+func VoteToStep(vote *tm_types.Vote) int8 {
+	switch vote.Type {
+	case tm_types.VoteTypePrevote:
+		return StepPrevote
+	case tm_types.VoteTypePrecommit:
+		return StepPrecommit
+	default:
+		return StepError
+	}
+}
+
+type Verifier interface {
+	SignVote(signer tm_types.Signer, chainID string, vote *tm_types.Vote) error
+	SignProposal(signer tm_types.Signer, chainID string, proposal *tm_types.Proposal) error
+	SignHeartbeat(signer tm_types.Signer, chainID string, heartbeat *tm_types.Heartbeat) error
+}
+
+type LastSignedInfo struct {
+	mtx           sync.Mutex
+	LastHeight    int              `json:"last_height"`
+	LastRound     int              `json:"last_round"`
+	LastStep      int8             `json:"last_step"`
+	LastSignature crypto.Signature `json:"last_signature,omitempty"` // so we dont lose signatures
+	LastSignBytes data.Bytes       `json:"last_signbytes,omitempty"` // so we dont lose signatures
+}
+
+var _ Verifier = &LastSignedInfo{}
+
+// SignVote signs a canonical representation of the vote, along with the chainID.
+// Implements PrivValidator.
+func (lsi *LastSignedInfo) SignVote(signer tm_types.Signer, chainID string, vote *tm_types.Vote) error {
+	lsi.mtx.Lock()
+	defer lsi.mtx.Unlock()
+	signature, err := lsi.signBytesHRS(signer, vote.Height, vote.Round, VoteToStep(vote), tm_types.SignBytes(chainID, vote))
+	if err != nil {
+		return fmt.Errorf("error signing vote: %v", err)
+	}
+	vote.Signature = signature
+	return nil
+}
+
+// SignProposal signs a canonical representation of the proposal, along with the chainID.
+// Implements PrivValidator.
+func (lsi *LastSignedInfo) SignProposal(signer tm_types.Signer, chainID string, proposal *tm_types.Proposal) error {
+	lsi.mtx.Lock()
+	defer lsi.mtx.Unlock()
+	signature, err := lsi.signBytesHRS(signer, proposal.Height, proposal.Round, StepPropose, tm_types.SignBytes(chainID, proposal))
+	if err != nil {
+		return fmt.Errorf("error signing proposal: %v", err)
+	}
+	proposal.Signature = signature
+	return nil
+}
+
+// SignHeartbeat signs a canonical representation of the heartbeat, along with the chainID.
+// Implements PrivValidator.
+func (lsi *LastSignedInfo) SignHeartbeat(signer tm_types.Signer, chainID string, heartbeat *tm_types.Heartbeat) error {
+	lsi.mtx.Lock()
+	defer lsi.mtx.Unlock()
+	var err error
+	heartbeat.Signature, err = signer.Sign(tm_types.SignBytes(chainID, heartbeat))
+	return err
+}
+
+// signBytesHRS signs the given signBytes if the height/round/step (HRS)
+// are greater than the latest state. If the HRS are equal,
+// it returns the privValidator.LastSignature.
+func (lsi *LastSignedInfo) signBytesHRS(signer tm_types.Signer, height, round int, step int8, signBytes []byte) (crypto.Signature, error) {
+
+	sig := crypto.Signature{}
+	// If height regression, err
+	if lsi.LastHeight > height {
+		return sig, errors.New("height regression")
+	}
+	// More cases for when the height matches
+	if lsi.LastHeight == height {
+		// If round regression, err
+		if lsi.LastRound > round {
+			return sig, errors.New("round regression")
+		}
+		// If step regression, err
+		if lsi.LastRound == round {
+			if lsi.LastStep > step {
+				return sig, errors.New("step regression")
+			} else if lsi.LastStep == step {
+				if lsi.LastSignBytes != nil {
+					if lsi.LastSignature.Empty() {
+						return crypto.Signature{}, errors.New("lsi: LastSignature is nil but LastSignBytes is not")
+					}
+					// so we dont sign a conflicting vote or proposal
+					// NOTE: proposals are non-deterministic (include time),
+					// so we can actually lose them, but will still never sign conflicting ones
+					if bytes.Equal(lsi.LastSignBytes, signBytes) {
+						// log.Notice("Using lsi.LastSignature", "sig", lsi.LastSignature)
+						return lsi.LastSignature, nil
+					}
+				}
+				return sig, errors.New("step regression")
+			}
+		}
+	}
+
+	// Sign
+	sig, err := signer.Sign(signBytes)
+	if err != nil {
+		return sig, err
+	}
+
+	// Persist height/round/step
+	lsi.LastHeight = height
+	lsi.LastRound = round
+	lsi.LastStep = step
+	lsi.LastSignature = sig
+	lsi.LastSignBytes = signBytes
+
+	return sig, nil
+}
+
+func (lsi *LastSignedInfo) String() string {
+	return fmt.Sprintf("LastSignedInfo{LastHeight:%v, LastRound:%v, LastStep:%v}",
+		lsi.LastHeight, lsi.LastRound, lsi.LastStep)
+}
diff --git a/consensus/tendermint/version.go b/consensus/tendermint/version.go
deleted file mode 100644
index 58f690db7dbace4b8ebb5852457d7fdb26c867c8..0000000000000000000000000000000000000000
--- a/consensus/tendermint/version.go
+++ /dev/null
@@ -1,70 +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
-
-import (
-	"strconv"
-
-	tendermint_version "github.com/tendermint/tendermint/version"
-
-	version "github.com/hyperledger/burrow/version"
-)
-
-const (
-	// Client identifier to advertise over the network
-	tendermintClientIdentifier = "tendermint"
-)
-
-var (
-	tendermintVersionMajor uint8
-	tendermintVersionMinor uint8
-	tendermintVersionPatch uint8
-)
-
-func init() {
-	// discard error because we test for this in Continuous Integration tests
-	tendermintVersionMajor, _ = getTendermintMajorVersionFromSource()
-	tendermintVersionMinor, _ = getTendermintMinorVersionFromSource()
-	tendermintVersionPatch, _ = getTendermintPatchVersionFromSource()
-}
-
-func GetTendermintVersion() *version.VersionIdentifier {
-	return version.New(tendermintClientIdentifier, tendermintVersionMajor,
-		tendermintVersionMinor, tendermintVersionPatch)
-}
-
-func getTendermintMajorVersionFromSource() (uint8, error) {
-	majorVersionUint, err := strconv.ParseUint(tendermint_version.Maj, 10, 8)
-	if err != nil {
-		return 0, err
-	}
-	return uint8(majorVersionUint), nil
-}
-
-func getTendermintMinorVersionFromSource() (uint8, error) {
-	minorVersionUint, err := strconv.ParseUint(tendermint_version.Min, 10, 8)
-	if err != nil {
-		return 0, err
-	}
-	return uint8(minorVersionUint), nil
-}
-
-func getTendermintPatchVersionFromSource() (uint8, error) {
-	patchVersionUint, err := strconv.ParseUint(tendermint_version.Fix, 10, 8)
-	if err != nil {
-		return 0, err
-	}
-	return uint8(patchVersionUint), nil
-}
diff --git a/consensus/tendermint/version_test.go b/consensus/tendermint/version_test.go
deleted file mode 100644
index 330c77425a6a281581882bda8424620873a84773..0000000000000000000000000000000000000000
--- a/consensus/tendermint/version_test.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 tendermint
-
-import "testing"
-
-func TestSemanticVersioningTendermint(t *testing.T) {
-	// assert that reading the semantic version from Tendermint vendored source
-	// succeeds without error; at runtime initialisation, on error we default
-	// to hard-coded semantic version
-	if _, err := getTendermintMajorVersionFromSource(); err != nil {
-		t.Errorf("Failed to read Major version from Tendermint source code: %s", err)
-		t.Fail()
-	}
-	if _, err := getTendermintMinorVersionFromSource(); err != nil {
-		t.Errorf("Failed to read Minor version from Tendermint source code: %s", err)
-		t.Fail()
-	}
-	if _, err := getTendermintPatchVersionFromSource(); err != nil {
-		t.Errorf("Failed to read Patch version from Tendermint source code: %s", err)
-		t.Fail()
-	}
-}
diff --git a/consensus/types/consensus_engine.go b/consensus/types/consensus_engine.go
deleted file mode 100644
index b1677d7a53fe8ef14181738ae17f93ff59748d22..0000000000000000000000000000000000000000
--- a/consensus/types/consensus_engine.go
+++ /dev/null
@@ -1,54 +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 types
-
-import (
-	"github.com/hyperledger/burrow/event"
-	"github.com/hyperledger/burrow/txs"
-	abci_types "github.com/tendermint/abci/types"
-	"github.com/tendermint/go-crypto"
-	"github.com/tendermint/go-p2p"
-)
-
-type ConsensusEngine interface {
-	// Peer-2-Peer
-	IsListening() bool
-	Listeners() []p2p.Listener
-	NodeInfo() *p2p.NodeInfo
-	Peers() []*Peer
-
-	// Private Validator
-	PublicValidatorKey() crypto.PubKey
-
-	// Memory pool
-	BroadcastTransaction(transaction []byte,
-		callback func(*abci_types.Response)) error
-
-	// Events
-	// For consensus events like NewBlock
-	Events() event.EventEmitter
-
-	// List pending transactions in the mempool, passing 0 for maxTxs gets an
-	// unbounded number of transactions
-	ListUnconfirmedTxs(maxTxs int) ([]txs.Tx, error)
-	ListValidators() []Validator
-	ConsensusState() *ConsensusState
-	// TODO: Consider creating a real type for PeerRoundState, but at the looks
-	// quite coupled to tendermint
-	PeerConsensusStates() map[string]string
-
-	// Allow for graceful shutdown of node. Returns whether the node was stopped.
-	Stop() bool
-}
diff --git a/consensus/types/consensus_state.go b/consensus/types/consensus_state.go
deleted file mode 100644
index 74d8b4b4a2167d5061bc120ac930409ace542570..0000000000000000000000000000000000000000
--- a/consensus/types/consensus_state.go
+++ /dev/null
@@ -1,46 +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 types
-
-import (
-	"time"
-
-	tendermint_consensus "github.com/tendermint/tendermint/consensus"
-	tendermint_types "github.com/tendermint/tendermint/types"
-)
-
-// ConsensusState
-type ConsensusState struct {
-	Height     int                        `json:"height"`
-	Round      int                        `json:"round"`
-	Step       uint8                      `json:"step"`
-	StartTime  time.Time                  `json:"start_time"`
-	CommitTime time.Time                  `json:"commit_time"`
-	Validators []Validator                `json:"validators"`
-	Proposal   *tendermint_types.Proposal `json:"proposal"`
-}
-
-func FromRoundState(rs *tendermint_consensus.RoundState) *ConsensusState {
-	cs := &ConsensusState{
-		StartTime:  rs.StartTime,
-		CommitTime: rs.CommitTime,
-		Height:     rs.Height,
-		Proposal:   rs.Proposal,
-		Round:      rs.Round,
-		Step:       uint8(rs.Step),
-		Validators: FromTendermintValidators(rs.Validators.Validators),
-	}
-	return cs
-}
diff --git a/consensus/types/peer.go b/consensus/types/peer.go
deleted file mode 100644
index c123cf5c022e43d7a0224c55c14623913a4f12c4..0000000000000000000000000000000000000000
--- a/consensus/types/peer.go
+++ /dev/null
@@ -1,22 +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 types
-
-import "github.com/tendermint/go-p2p"
-
-type Peer struct {
-	NodeInfo   *p2p.NodeInfo `json:"node_info"`
-	IsOutbound bool          `json:"is_outbound"`
-}
diff --git a/consensus/types/validator.go b/consensus/types/validator.go
deleted file mode 100644
index 86fcfac1a6d7badbd2792eb5cae07f7156b32038..0000000000000000000000000000000000000000
--- a/consensus/types/validator.go
+++ /dev/null
@@ -1,59 +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 types
-
-import (
-	"github.com/tendermint/go-wire"
-	tendermint_types "github.com/tendermint/tendermint/types"
-)
-
-var _ = wire.RegisterInterface(
-	struct{ Validator }{},
-	wire.ConcreteType{&TendermintValidator{}, byte(0x01)},
-)
-
-type Validator interface {
-	AssertIsValidator()
-	Address() []byte
-}
-
-// Anticipating moving to our own definition of Validator, or at least
-// augmenting Tendermint's.
-type TendermintValidator struct {
-	*tendermint_types.Validator `json:"validator"`
-}
-
-var _ Validator = (*TendermintValidator)(nil)
-
-func (tendermintValidator *TendermintValidator) AssertIsValidator() {
-
-}
-
-func (tendermintValidator *TendermintValidator) Address() []byte {
-	return tendermintValidator.Address()
-}
-
-//-------------------------------------------------------------------------------------
-// Helper function for TendermintValidator
-
-func FromTendermintValidators(tmValidators []*tendermint_types.Validator) []Validator {
-	validators := make([]Validator, len(tmValidators))
-	for i, tmValidator := range tmValidators {
-		// This embedding could be replaced by a mapping once if we want to describe
-		// a more general notion of validator
-		validators[i] = &TendermintValidator{tmValidator}
-	}
-	return validators
-}