diff --git a/chain/src/chain.rs b/chain/src/chain.rs
index fe63b69e7..e0771bff1 100644
--- a/chain/src/chain.rs
+++ b/chain/src/chain.rs
@@ -32,7 +32,7 @@ use grin_store::Error::NotFoundErr;
 use pipe;
 use store;
 use txhashset;
-use types::{ChainAdapter, Options, Tip};
+use types::{ChainAdapter, NoStatus, Options, Tip, TxHashsetWriteStatus};
 use util::secp::pedersen::{Commitment, RangeProof};
 use util::LOGGER;
 
@@ -433,7 +433,7 @@ impl Chain {
 		txhashset::extending_readonly(&mut txhashset, |extension| {
 			// TODO - is this rewind guaranteed to be redundant now?
 			extension.rewind(&header, &header, true, true, true)?;
-			extension.validate(&header, skip_rproofs)?;
+			extension.validate(&header, skip_rproofs, &NoStatus)?;
 			Ok(())
 		})
 	}
@@ -521,14 +521,19 @@ impl Chain {
 	/// If we're willing to accept that new state, the data stream will be
 	/// read as a zip file, unzipped and the resulting state files should be
 	/// rewound to the provided indexes.
-	pub fn txhashset_write(
+	pub fn txhashset_write<T>(
 		&self,
 		h: Hash,
 		rewind_to_output: u64,
 		rewind_to_kernel: u64,
 		txhashset_data: File,
-	) -> Result<(), Error> {
-		let _lock = self.txhashset_lock.lock().unwrap();
+		status: &T,
+	) -> Result<(), Error>
+	where
+		T: TxHashsetWriteStatus,
+	{
+		self.txhashset_lock.lock().unwrap();
+		status.on_setup();
 		let head = self.head().unwrap();
 		let header_head = self.get_header_head().unwrap();
 		if header_head.height - head.height < global::cut_through_horizon() as u64 {
@@ -543,16 +548,17 @@ impl Chain {
 
 		// all good, prepare a new batch and update all the required records
 		let mut batch = self.store.batch()?;
-		// Note: we are validataing against a writeable extension.
+		// Note: we are validating against a writeable extension.
 		txhashset::extending(&mut txhashset, &mut batch, |extension| {
 			// TODO do we need to rewind here? We have no blocks to rewind
 			// (and we need them for the pos to unremove)
 			extension.rewind(&header, &header, true, true, true)?;
-			let (_output_sum, _kernel_sum) = extension.validate(&header, false)?;
+			extension.validate(&header, false, status)?;
 			extension.rebuild_index()?;
 			Ok(())
 		})?;
 
+		status.on_save();
 		// replace the chain txhashset with the newly built one
 		{
 			let mut txhashset_ref = self.txhashset.write().unwrap();
@@ -569,6 +575,8 @@ impl Chain {
 		batch.commit()?;
 
 		self.check_orphans(header.height + 1);
+
+		status.on_done();
 		Ok(())
 	}
 
diff --git a/chain/src/lib.rs b/chain/src/lib.rs
index 05f2b9ab0..6d8fedbe2 100644
--- a/chain/src/lib.rs
+++ b/chain/src/lib.rs
@@ -53,4 +53,4 @@ pub mod types;
 pub use chain::{Chain, MAX_ORPHAN_SIZE};
 pub use error::{Error, ErrorKind};
 pub use store::ChainStore;
-pub use types::{ChainAdapter, Options, Tip};
+pub use types::{ChainAdapter, Options, Tip, TxHashsetWriteStatus};
diff --git a/chain/src/txhashset.rs b/chain/src/txhashset.rs
index bec6a5230..0e0524ef8 100644
--- a/chain/src/txhashset.rs
+++ b/chain/src/txhashset.rs
@@ -29,8 +29,7 @@ use util::secp::pedersen::{Commitment, RangeProof};
 use core::core::committed::Committed;
 use core::core::hash::{Hash, Hashed};
 use core::core::merkle_proof::MerkleProof;
-use core::core::pmmr;
-use core::core::pmmr::PMMR;
+use core::core::pmmr::{self, PMMR};
 use core::core::{
 	Block, BlockHeader, Input, Output, OutputFeatures, OutputIdentifier, Transaction, TxKernel,
 };
@@ -42,7 +41,7 @@ use grin_store;
 use grin_store::pmmr::PMMRBackend;
 use grin_store::types::prune_noop;
 use store::{Batch, ChainStore};
-use types::TxHashSetRoots;
+use types::{TxHashSetRoots, TxHashsetWriteStatus};
 use util::{secp_static, zip, LOGGER};
 
 const TXHASHSET_SUBDIR: &'static str = "txhashset";
@@ -899,11 +898,15 @@ impl<'a> Extension<'a> {
 	}
 
 	/// Validate the txhashset state against the provided block header.
-	pub fn validate(
+	pub fn validate<T>(
 		&mut self,
 		header: &BlockHeader,
 		skip_rproofs: bool,
-	) -> Result<((Commitment, Commitment)), Error> {
+		status: &T,
+	) -> Result<((Commitment, Commitment)), Error>
+	where
+		T: TxHashsetWriteStatus,
+	{
 		self.validate_mmrs()?;
 		self.validate_roots(header)?;
 
@@ -912,19 +915,18 @@ impl<'a> Extension<'a> {
 			return Ok((zero_commit.clone(), zero_commit.clone()));
 		}
 
-		// The real magicking happens here.
-		// Sum of kernel excesses should equal sum of
-		// unspent outputs minus total supply.
+		// The real magicking happens here. Sum of kernel excesses should equal
+		// sum of unspent outputs minus total supply.
 		let (output_sum, kernel_sum) =
 			self.verify_kernel_sums(header.total_overage(), header.total_kernel_offset())?;
 
 		// This is an expensive verification step.
-		self.verify_kernel_signatures()?;
+		self.verify_kernel_signatures(status)?;
 
 		// Verify the rangeproof for each output in the sum above.
 		// This is an expensive verification step (skip for faster verification).
 		if !skip_rproofs {
-			self.verify_rangeproofs()?;
+			self.verify_rangeproofs(status)?;
 		}
 
 		// Verify kernel roots for all past headers, need to be last as it rewinds
@@ -986,10 +988,14 @@ impl<'a> Extension<'a> {
 		)
 	}
 
-	fn verify_kernel_signatures(&self) -> Result<(), Error> {
+	fn verify_kernel_signatures<T>(&self, status: &T) -> Result<(), Error>
+	where
+		T: TxHashsetWriteStatus,
+	{
 		let now = Instant::now();
 
 		let mut kern_count = 0;
+		let total_kernels = pmmr::n_leaves(self.kernel_pmmr.unpruned_size());
 		for n in 1..self.kernel_pmmr.unpruned_size() + 1 {
 			if pmmr::is_leaf(n) {
 				if let Some(kernel) = self.kernel_pmmr.get_data(n) {
@@ -997,6 +1003,9 @@ impl<'a> Extension<'a> {
 					kern_count += 1;
 				}
 			}
+			if n % 20 == 0 {
+				status.on_validation(kern_count, total_kernels, 0, 0);
+			}
 		}
 
 		debug!(
@@ -1010,10 +1019,14 @@ impl<'a> Extension<'a> {
 		Ok(())
 	}
 
-	fn verify_rangeproofs(&self) -> Result<(), Error> {
+	fn verify_rangeproofs<T>(&self, status: &T) -> Result<(), Error>
+	where
+		T: TxHashsetWriteStatus,
+	{
 		let now = Instant::now();
 
 		let mut proof_count = 0;
+		let total_rproofs = pmmr::n_leaves(self.output_pmmr.unpruned_size());
 		for n in 1..self.output_pmmr.unpruned_size() + 1 {
 			if pmmr::is_leaf(n) {
 				if let Some(out) = self.output_pmmr.get_data(n) {
@@ -1033,6 +1046,9 @@ impl<'a> Extension<'a> {
 					}
 				}
 			}
+			if n % 20 == 0 {
+				status.on_validation(0, 0, proof_count, total_rproofs);
+			}
 		}
 		debug!(
 			LOGGER,
diff --git a/chain/src/types.rs b/chain/src/types.rs
index 6481b90cf..a98d1bf78 100644
--- a/chain/src/types.rs
+++ b/chain/src/types.rs
@@ -117,6 +117,32 @@ pub trait ChainAdapter {
 	fn block_accepted(&self, b: &Block, opts: Options);
 }
 
+/// Inform the caller of the current status of a txhashset write operation,
+/// as it can take quite a while to process. Each function is called in the
+/// order defined below and can be used to provide some feedback to the
+/// caller. Functions taking arguments can be called repeatedly to update
+/// those values as the processing progresses.
+pub trait TxHashsetWriteStatus {
+	/// First setup of the txhashset
+	fn on_setup(&self);
+	/// Starting validation
+	fn on_validation(&self, kernels: u64, kernel_total: u64, rproofs: u64, rproof_total: u64);
+	/// Starting to save the txhashset and related data
+	fn on_save(&self);
+	/// Done writing a new txhashset
+	fn on_done(&self);
+}
+
+/// Do-nothing implementation of TxHashsetWriteStatus
+pub struct NoStatus;
+
+impl TxHashsetWriteStatus for NoStatus {
+	fn on_setup(&self) {}
+	fn on_validation(&self, _ks: u64, _kts: u64, _rs: u64, _rt: u64) {}
+	fn on_save(&self) {}
+	fn on_done(&self) {}
+}
+
 /// Dummy adapter used as a placeholder for real implementations
 pub struct NoopAdapter {}
 
diff --git a/servers/src/common/adapters.rs b/servers/src/common/adapters.rs
index 160cb7be7..ac7c6a126 100644
--- a/servers/src/common/adapters.rs
+++ b/servers/src/common/adapters.rs
@@ -19,13 +19,12 @@ use rand::{self, Rng};
 use std::fs::File;
 use std::net::SocketAddr;
 use std::ops::Deref;
-use std::sync::atomic::{AtomicBool, Ordering};
 use std::sync::{Arc, RwLock, Weak};
 use std::thread;
 use std::time::Instant;
 
 use chain::{self, ChainAdapter, Options, Tip};
-use common::types::{ChainValidationMode, ServerConfig};
+use common::types::{ChainValidationMode, ServerConfig, SyncState};
 use core::core;
 use core::core::block::BlockHeader;
 use core::core::hash::{Hash, Hashed};
@@ -51,7 +50,7 @@ fn wo<T>(weak_one: &OneTime<Weak<T>>) -> Arc<T> {
 /// blocks and transactions are received and forwards to the chain and pool
 /// implementations.
 pub struct NetToChainAdapter {
-	currently_syncing: Arc<AtomicBool>,
+	sync_state: Arc<SyncState>,
 	archive_mode: bool,
 	chain: Weak<chain::Chain>,
 	tx_pool: Arc<RwLock<pool::TransactionPool<PoolToChainAdapter>>>,
@@ -335,13 +334,17 @@ impl p2p::ChainAdapter for NetToChainAdapter {
 	) -> bool {
 		// TODO check whether we should accept any txhashset now
 		if let Err(e) =
-			w(&self.chain).txhashset_write(h, rewind_to_output, rewind_to_kernel, txhashset_data)
-		{
+			w(&self.chain).txhashset_write(
+				h,
+				rewind_to_output,
+				rewind_to_kernel,
+				txhashset_data,
+				self.sync_state.as_ref(),
+		) {
 			error!(LOGGER, "Failed to save txhashset archive: {}", e);
 			!e.is_bad_data()
 		} else {
 			info!(LOGGER, "Received valid txhashset data for {}.", h);
-			self.currently_syncing.store(true, Ordering::Relaxed);
 			true
 		}
 	}
@@ -350,14 +353,14 @@ impl p2p::ChainAdapter for NetToChainAdapter {
 impl NetToChainAdapter {
 	/// Construct a new NetToChainAdapter instance
 	pub fn new(
-		currently_syncing: Arc<AtomicBool>,
+		sync_state: Arc<SyncState>,
 		archive_mode: bool,
 		chain_ref: Weak<chain::Chain>,
 		tx_pool: Arc<RwLock<pool::TransactionPool<PoolToChainAdapter>>>,
 		config: ServerConfig,
 	) -> NetToChainAdapter {
 		NetToChainAdapter {
-			currently_syncing,
+			sync_state,
 			archive_mode,
 			chain: chain_ref,
 			tx_pool,
@@ -438,9 +441,7 @@ impl NetToChainAdapter {
 				match e.kind() {
 					chain::ErrorKind::Orphan => {
 						// make sure we did not miss the parent block
-						if !chain.is_orphan(&prev_hash)
-							&& !self.currently_syncing.load(Ordering::Relaxed)
-						{
+						if !chain.is_orphan(&prev_hash) && !self.sync_state.is_syncing() {
 							debug!(LOGGER, "adapter: process_block: received an orphan block, checking the parent: {:}", prev_hash);
 							self.request_block_by_hash(prev_hash, &addr)
 						}
@@ -466,7 +467,7 @@ impl NetToChainAdapter {
 		// Skip this if we are currently syncing (too slow).
 		let chain = w(&self.chain);
 		if chain.head().unwrap().height > 0
-			&& !self.currently_syncing.load(Ordering::Relaxed)
+			&& !self.sync_state.is_syncing()
 			&& self.config.chain_validation_mode == ChainValidationMode::EveryBlock
 		{
 			let now = Instant::now();
@@ -491,7 +492,7 @@ impl NetToChainAdapter {
 
 	fn check_compact(&self, tip_res: Option<Tip>) {
 		// no compaction during sync or if we're in historical mode
-		if self.archive_mode || self.currently_syncing.load(Ordering::Relaxed) {
+		if self.archive_mode || self.sync_state.is_syncing() {
 			return;
 		}
 
@@ -559,7 +560,7 @@ impl NetToChainAdapter {
 
 	/// Prepare options for the chain pipeline
 	fn chain_opts(&self) -> chain::Options {
-		let opts = if self.currently_syncing.load(Ordering::Relaxed) {
+		let opts = if self.sync_state.is_syncing() {
 			chain::Options::SYNC
 		} else {
 			chain::Options::NONE
diff --git a/servers/src/common/stats.rs b/servers/src/common/stats.rs
index b09abc9a9..021b119d9 100644
--- a/servers/src/common/stats.rs
+++ b/servers/src/common/stats.rs
@@ -20,6 +20,7 @@ use std::sync::{Arc, RwLock};
 use std::time::SystemTime;
 
 use chain;
+use common::types::SyncStatus;
 use p2p;
 
 /// Server state info collection struct, to be passed around into internals
@@ -51,7 +52,7 @@ pub struct ServerStats {
 	/// sync header head
 	pub header_head: chain::Tip,
 	/// Whether we're currently syncing
-	pub is_syncing: bool,
+	pub sync_status: SyncStatus,
 	/// Whether we're awaiting peers
 	pub awaiting_peers: bool,
 	/// Handle to current stratum server stats
diff --git a/servers/src/common/types.rs b/servers/src/common/types.rs
index cd901bac1..f3a176a52 100644
--- a/servers/src/common/types.rs
+++ b/servers/src/common/types.rs
@@ -15,6 +15,7 @@
 //! Server types
 
 use std::convert::From;
+use std::sync::RwLock;
 
 use api;
 use chain;
@@ -253,3 +254,120 @@ impl Default for StratumServerConfig {
 		}
 	}
 }
+
+/// Various status sync can be in, whether it's fast sync or archival.
+#[derive(Debug, Clone, Copy, Eq, PartialEq)]
+#[allow(missing_docs)]
+pub enum SyncStatus {
+	/// Not syncing
+	NoSync,
+	/// Downloading block headers
+	HeaderSync {
+		current_height: u64,
+		highest_height: u64,
+	},
+	/// Downloading the various txhashsets
+	TxHashsetDownload,
+	/// Setting up before validation
+	TxHashsetSetup,
+	/// Validating the full state
+	TxHashsetValidation {
+		kernels: u64,
+		kernel_total: u64,
+		rproofs: u64,
+		rproof_total: u64,
+	},
+	/// Finalizing the new state
+	TxHashsetSave,
+	/// Downloading blocks
+	BodySync {
+		current_height: u64,
+		highest_height: u64,
+	},
+}
+
+/// Current sync state. Encapsulates the current SyncStatus.
+pub struct SyncState {
+	current: RwLock<SyncStatus>,
+}
+
+impl SyncState {
+	/// Return a new SyncState initialize to NoSync
+	pub fn new() -> SyncState {
+		SyncState {
+			current: RwLock::new(SyncStatus::NoSync),
+		}
+	}
+
+	/// Whether the current state matches any active syncing operation
+	pub fn is_syncing(&self) -> bool {
+		*self.current.read().unwrap() != SyncStatus::NoSync
+	}
+
+	/// Current syncing status
+	pub fn status(&self) -> SyncStatus {
+		*self.current.read().unwrap()
+	}
+
+	/// Update the syncing status
+	pub fn update(&self, new_status: SyncStatus) {
+		let mut status = self.current.write().unwrap();
+		*status = new_status;
+	}
+}
+
+impl chain::TxHashsetWriteStatus for SyncState {
+	fn on_setup(&self) {
+		self.update(SyncStatus::TxHashsetSetup);
+	}
+
+	fn on_validation(&self, vkernels: u64, vkernel_total: u64, vrproofs: u64, vrproof_total: u64) {
+		let mut status = self.current.write().unwrap();
+		match *status {
+			SyncStatus::TxHashsetValidation {
+				kernels,
+				kernel_total,
+				rproofs,
+				rproof_total,
+			} => {
+				let ks = if vkernels > 0 { vkernels } else { kernels };
+				let kt = if vkernel_total > 0 {
+					vkernel_total
+				} else {
+					kernel_total
+				};
+				let rps = if vrproofs > 0 { vrproofs } else { rproofs };
+				let rpt = if vrproof_total > 0 {
+					vrproof_total
+				} else {
+					rproof_total
+				};
+				*status = SyncStatus::TxHashsetValidation {
+					kernels: ks,
+					kernel_total: kt,
+					rproofs: rps,
+					rproof_total: rpt,
+				};
+			}
+			_ => {
+				*status = SyncStatus::TxHashsetValidation {
+					kernels: 0,
+					kernel_total: 0,
+					rproofs: 0,
+					rproof_total: 0,
+				}
+			}
+		}
+	}
+
+	fn on_save(&self) {
+		self.update(SyncStatus::TxHashsetSave);
+	}
+
+	fn on_done(&self) {
+		self.update(SyncStatus::BodySync {
+			current_height: 0,
+			highest_height: 0,
+		});
+	}
+}
diff --git a/servers/src/grin/server.rs b/servers/src/grin/server.rs
index f21c30e74..e96c3cb46 100644
--- a/servers/src/grin/server.rs
+++ b/servers/src/grin/server.rs
@@ -26,7 +26,7 @@ use chain;
 use common::adapters::{ChainToPoolAndNetAdapter, NetToChainAdapter, PoolToChainAdapter,
                        PoolToNetAdapter};
 use common::stats::{DiffBlock, DiffStats, PeerStats, ServerStateInfo, ServerStats};
-use common::types::{Error, Seeding, ServerConfig, StratumServerConfig};
+use common::types::{Error, Seeding, ServerConfig, StratumServerConfig, SyncState};
 use core::core::hash::Hashed;
 use core::core::target::Difficulty;
 use core::{consensus, genesis, global, pow};
@@ -49,7 +49,7 @@ pub struct Server {
 	/// in-memory transaction pool
 	tx_pool: Arc<RwLock<pool::TransactionPool<PoolToChainAdapter>>>,
 	/// Whether we're currently syncing
-	currently_syncing: Arc<AtomicBool>,
+	sync_state: Arc<SyncState>,
 	/// To be passed around to collect stats and info
 	state_info: ServerStateInfo,
 	/// Stop flag
@@ -142,11 +142,11 @@ impl Server {
 
 		pool_adapter.set_chain(Arc::downgrade(&shared_chain));
 
-		let currently_syncing = Arc::new(AtomicBool::new(true));
+		let sync_state = Arc::new(SyncState::new());
 		let awaiting_peers = Arc::new(AtomicBool::new(false));
 
 		let net_adapter = Arc::new(NetToChainAdapter::new(
-			currently_syncing.clone(),
+			sync_state.clone(),
 			archive_mode,
 			Arc::downgrade(&shared_chain),
 			tx_pool.clone(),
@@ -205,7 +205,7 @@ impl Server {
 		let syncer = sync::Syncer::new();
 
 		syncer.run_sync(
-			currently_syncing.clone(),
+			sync_state.clone(),
 			awaiting_peers.clone(),
 			p2p_server.peers.clone(),
 			shared_chain.clone(),
@@ -244,7 +244,7 @@ impl Server {
 			p2p: p2p_server,
 			chain: shared_chain,
 			tx_pool: tx_pool,
-			currently_syncing: currently_syncing,
+			sync_state,
 			state_info: ServerStateInfo {
 				awaiting_peers: awaiting_peers,
 				..Default::default()
@@ -268,7 +268,7 @@ impl Server {
 	pub fn start_stratum_server(&self, config: StratumServerConfig) {
 		let cuckoo_size = global::min_sizeshift();
 		let proof_size = global::proofsize();
-		let currently_syncing = self.currently_syncing.clone();
+		let sync_state = self.sync_state.clone();
 
 		let mut stratum_server = stratumserver::StratumServer::new(
 			config.clone(),
@@ -283,7 +283,7 @@ impl Server {
 					stratum_stats,
 					cuckoo_size as u32,
 					proof_size,
-					currently_syncing,
+					sync_state,
 				);
 			});
 	}
@@ -292,7 +292,7 @@ impl Server {
 	/// internal miner, and should only be used for automated testing. Burns
 	/// reward if wallet_listener_url is 'None'
 	pub fn start_test_miner(&self, wallet_listener_url: Option<String>) {
-		let currently_syncing = self.currently_syncing.clone();
+		let sync_state = self.sync_state.clone();
 		let config_wallet_url = match wallet_listener_url.clone() {
 			Some(u) => u,
 			None => String::from("http://127.0.0.1:13415"),
@@ -320,7 +320,7 @@ impl Server {
 				// TODO push this down in the run loop so miner gets paused anytime we
 				// decide to sync again
 				let secs_5 = time::Duration::from_secs(5);
-				while currently_syncing.load(Ordering::Relaxed) {
+				while sync_state.is_syncing() {
 					thread::sleep(secs_5);
 				}
 				miner.run_loop(wallet_listener_url);
@@ -408,7 +408,7 @@ impl Server {
 			peer_count: self.peer_count(),
 			head: self.head(),
 			header_head: self.header_head(),
-			is_syncing: self.currently_syncing.load(Ordering::Relaxed),
+			sync_status: self.sync_state.status(),
 			awaiting_peers: awaiting_peers,
 			stratum_stats: stratum_stats,
 			peer_stats: peer_stats,
diff --git a/servers/src/grin/sync.rs b/servers/src/grin/sync.rs
index 890f7f3ae..33ca14b5d 100644
--- a/servers/src/grin/sync.rs
+++ b/servers/src/grin/sync.rs
@@ -19,7 +19,7 @@ use std::{cmp, thread};
 use time;
 
 use chain;
-use common::types::Error;
+use common::types::{Error, SyncState, SyncStatus};
 use core::core::hash::{Hash, Hashed};
 use core::core::target::Difficulty;
 use core::global;
@@ -36,7 +36,7 @@ impl Syncer {
 
 	pub fn run_sync(
 		&self,
-		currently_syncing: Arc<AtomicBool>,
+		sync_state: Arc<SyncState>,
 		awaiting_peers: Arc<AtomicBool>,
 		peers: Arc<p2p::Peers>,
 		chain: Arc<chain::Chain>,
@@ -45,7 +45,7 @@ impl Syncer {
 		stop: Arc<AtomicBool>,
 	) {
 		sync::run_sync(
-			currently_syncing,
+			sync_state,
 			awaiting_peers,
 			peers,
 			chain,
@@ -58,7 +58,7 @@ impl Syncer {
 
 /// Starts the syncing loop, just spawns two threads that loop forever
 pub fn run_sync(
-	currently_syncing: Arc<AtomicBool>,
+	sync_state: Arc<SyncState>,
 	awaiting_peers: Arc<AtomicBool>,
 	peers: Arc<p2p::Peers>,
 	chain: Arc<chain::Chain>,
@@ -98,7 +98,7 @@ pub fn run_sync(
 
 				// is syncing generally needed when we compare our state with others
 				let (syncing, most_work_height) =
-					needs_syncing(currently_syncing.as_ref(), peers.clone(), chain.clone());
+					needs_syncing(sync_state.as_ref(), peers.clone(), chain.clone());
 
 				if most_work_height > 0 {
 					// we can occasionally get a most work height of 0 if read locks fail
@@ -112,23 +112,26 @@ pub fn run_sync(
 					// run the header sync every 10s
 					if si.header_sync_due(&header_head) {
 						header_sync(peers.clone(), chain.clone());
+						sync_state.update(SyncStatus::HeaderSync{current_height: header_head.height, highest_height: si.highest_height});
 					}
 
 					if fast_sync_enabled {
 						// run fast sync if applicable, every 5 min
 						if header_head.height == si.highest_height && si.fast_sync_due() {
 							fast_sync(peers.clone(), chain.clone(), &header_head);
+							sync_state.update(SyncStatus::TxHashsetDownload);
 						}
 					} else {
 						// run the body_sync every 5s
 						if si.body_sync_due(&head) {
 							body_sync(peers.clone(), chain.clone());
+							sync_state.update(SyncStatus::BodySync{current_height: head.height, highest_height: si.highest_height});
 						}
 					}
+				} else {
+					sync_state.update(SyncStatus::NoSync);
 				}
 
-				currently_syncing.store(syncing, Ordering::Relaxed);
-
 				thread::sleep(Duration::from_secs(1));
 
 				if stop.load(Ordering::Relaxed) {
@@ -275,13 +278,13 @@ fn request_headers(peer: &Peer, chain: Arc<chain::Chain>) {
 /// Whether we're currently syncing the chain or we're fully caught up and
 /// just receiving blocks through gossip.
 fn needs_syncing(
-	currently_syncing: &AtomicBool,
+	sync_state: &SyncState,
 	peers: Arc<Peers>,
 	chain: Arc<chain::Chain>,
 ) -> (bool, u64) {
 	let local_diff = chain.total_difficulty();
 	let peer = peers.most_work_peer();
-	let is_syncing = currently_syncing.load(Ordering::Relaxed);
+	let is_syncing = sync_state.is_syncing();
 	let mut most_work_height = 0;
 
 	// if we're already syncing, we're caught up if no peer has a higher
diff --git a/servers/src/mining/stratumserver.rs b/servers/src/mining/stratumserver.rs
index 98b993af4..33518f36b 100644
--- a/servers/src/mining/stratumserver.rs
+++ b/servers/src/mining/stratumserver.rs
@@ -19,7 +19,6 @@ use serde_json::Value;
 use std::error::Error;
 use std::io::{BufRead, ErrorKind, Write};
 use std::net::{TcpListener, TcpStream};
-use std::sync::atomic::{AtomicBool, Ordering};
 use std::sync::{Arc, Mutex, RwLock};
 use std::time::{Duration, SystemTime};
 use std::{cmp, thread};
@@ -28,7 +27,7 @@ use time;
 use chain;
 use common::adapters::PoolToChainAdapter;
 use common::stats::{StratumStats, WorkerStats};
-use common::types::StratumServerConfig;
+use common::types::{StratumServerConfig, SyncState};
 use core::core::{Block, BlockHeader};
 use core::{pow, global};
 use keychain;
@@ -236,7 +235,7 @@ pub struct StratumServer {
 	minimum_share_difficulty: u64,
 	current_key_id: Option<keychain::Identifier>,
 	workers: Arc<Mutex<Vec<Worker>>>,
-	currently_syncing: Arc<AtomicBool>,
+	sync_state: Arc<SyncState>,
 }
 
 impl StratumServer {
@@ -256,7 +255,7 @@ impl StratumServer {
 			current_difficulty: <u64>::max_value(),
 			current_key_id: None,
 			workers: Arc::new(Mutex::new(Vec::new())),
-			currently_syncing: Arc::new(AtomicBool::new(false)),
+			sync_state: Arc::new(SyncState::new()),
 		}
 	}
 
@@ -320,7 +319,7 @@ impl StratumServer {
 						}
 						"keepalive" => self.handle_keepalive(),
 						"getjobtemplate" => {
-							if self.currently_syncing.load(Ordering::Relaxed) {
+							if self.sync_state.is_syncing() {
 								let e = RpcError {
 									code: -32701,
 									message: "Node is syncing - Please wait".to_string(),
@@ -623,7 +622,7 @@ impl StratumServer {
 		stratum_stats: Arc<RwLock<StratumStats>>,
 		cuckoo_size: u32,
 		proof_size: usize,
-		currently_syncing: Arc<AtomicBool>,
+		sync_state: Arc<SyncState>,
 	) {
 		info!(
 			LOGGER,
@@ -633,7 +632,7 @@ impl StratumServer {
 			proof_size
 		);
 
-		self.currently_syncing = currently_syncing;
+		self.sync_state = sync_state;
 
 		// "globals" for this function
 		let attempt_time_per_block = self.config.attempt_time_per_block;
@@ -673,7 +672,7 @@ impl StratumServer {
 		loop {
 			// If we're fallen into sync mode, (or are just starting up,
 			// tell connected clients to stop what they're doing
-			let mining_stopped = self.currently_syncing.load(Ordering::Relaxed);
+			let mining_stopped = self.sync_state.is_syncing();
 
 			// Remove workers with failed connections
 			num_workers = self.clean_workers(&mut stratum_stats.clone());
diff --git a/src/bin/tui/status.rs b/src/bin/tui/status.rs
index 7d86fad44..c7195e0a1 100644
--- a/src/bin/tui/status.rs
+++ b/src/bin/tui/status.rs
@@ -23,6 +23,7 @@ use cursive::views::{BoxView, LinearLayout, TextView};
 use tui::constants::VIEW_BASIC_STATUS;
 use tui::types::TUIStatusListener;
 
+use servers::common::types::SyncStatus;
 use servers::ServerStats;
 
 pub struct TUIStatusView;
@@ -76,14 +77,63 @@ impl TUIStatusListener for TUIStatusView {
 	fn update(c: &mut Cursive, stats: &ServerStats) {
 		//find and update here as needed
 		let basic_status = {
-			if stats.is_syncing {
-				if stats.awaiting_peers {
-					"Waiting for peers".to_string()
-				} else {
-					format!("Syncing - Latest header: {}", stats.header_head.height).to_string()
-				}
+			if stats.awaiting_peers {
+				"Waiting for peers".to_string()
 			} else {
-				"Running".to_string()
+				match stats.sync_status {
+					SyncStatus::NoSync => "Running".to_string(),
+					SyncStatus::HeaderSync {
+						current_height,
+						highest_height,
+					} => {
+						let percent = if highest_height == 0 {
+							0
+						} else {
+							current_height * 100 / highest_height
+						};
+						format!("Downloading headers: {}%, step 1/4", percent)
+					}
+					SyncStatus::TxHashsetDownload => {
+						"Downloading chain state for fast sync, step 2/4".to_string()
+					}
+					SyncStatus::TxHashsetSetup => {
+						"Preparing chain state for validation, step 3/4".to_string()
+					}
+					SyncStatus::TxHashsetValidation {
+						kernels,
+						kernel_total,
+						rproofs,
+						rproof_total,
+					} => {
+						// 10% of overall progress is attributed to kernel validation
+						// 90% to range proofs (which are much longer)
+						let mut percent = if kernel_total > 0 {
+							kernels * 10 / kernel_total
+						} else {
+							0
+						};
+						percent += if rproof_total > 0 {
+							rproofs * 90 / rproof_total
+						} else {
+							0
+						};
+						format!("Validating chain state: {}%, step 3/4", percent)
+					}
+					SyncStatus::TxHashsetSave => {
+						"Finalizing chain state for fast sync, step 3/4".to_string()
+					}
+					SyncStatus::BodySync {
+						current_height,
+						highest_height,
+					} => {
+						let percent = if highest_height == 0 {
+							0
+						} else {
+							current_height * 100 / highest_height
+						};
+						format!("Downloading blocks: {}%, step 4/4", percent)
+					}
+				}
 			}
 		};
 		/*let basic_mining_config_status = {