diff --git a/cmd/hack/hack.go b/cmd/hack/hack.go
index b02d78c61678d0df1342d73812b31dbcfa6334b2..e9290e9032d535a75deb3ecc5ebc03ceefa038e9 100644
--- a/cmd/hack/hack.go
+++ b/cmd/hack/hack.go
@@ -1739,22 +1739,16 @@ func extracHeaders(chaindata string, block uint64) error {
 	w := bufio.NewWriter(f)
 	defer w.Flush()
 	var hBuffer [headerdownload.HeaderSerLength]byte
-	var dBuffer [32]byte
 	for {
 		hash := rawdb.ReadCanonicalHash(db, b)
 		if hash == (common.Hash{}) {
 			break
 		}
 		h := rawdb.ReadHeader(db, hash, b)
-		td := rawdb.ReadTd(db, hash, b)
 		headerdownload.SerialiseHeader(h, hBuffer[:])
-		td.FillBytes(dBuffer[:])
 		if _, err := w.Write(hBuffer[:]); err != nil {
 			return err
 		}
-		if _, err := w.Write(dBuffer[:]); err != nil {
-			return err
-		}
 		b += block
 	}
 	fmt.Printf("Last block is %d\n", b)
diff --git a/cmd/headers/check/check.go b/cmd/headers/check/check.go
new file mode 100644
index 0000000000000000000000000000000000000000..c715f6a186471668aa49fd8bdd9a7614a1b31e77
--- /dev/null
+++ b/cmd/headers/check/check.go
@@ -0,0 +1,31 @@
+package check
+
+import (
+	"time"
+
+	"github.com/ledgerwatch/turbo-geth/log"
+	"github.com/ledgerwatch/turbo-geth/turbo/stages/headerdownload"
+)
+
+func Check(filesDir string) error {
+	log.Info("Checking", "directory", filesDir)
+	hd := headerdownload.NewHeaderDownload(
+		filesDir,
+		32*1024, /* bufferLimit */
+		16*1024, /* tipLimit */
+		1024,    /* initPowDepth */
+		nil,
+		nil,
+		3600, /* newAnchor future limit */
+		3600, /* newAnchor past limit */
+	)
+	if recovered, err := hd.RecoverFromFiles(uint64(time.Now().Unix())); err != nil || !recovered {
+		if err != nil {
+			log.Error("Recovery from file failed, will start from scratch", "error", err)
+		} else {
+			log.Info("Nothing recovered")
+		}
+	}
+	log.Info(hd.AnchorState())
+	return nil
+}
diff --git a/cmd/headers/commands/check.go b/cmd/headers/commands/check.go
new file mode 100644
index 0000000000000000000000000000000000000000..d902ba6fd3d1913243f5bd851d5c4a3478666641
--- /dev/null
+++ b/cmd/headers/commands/check.go
@@ -0,0 +1,19 @@
+package commands
+
+import (
+	"github.com/ledgerwatch/turbo-geth/cmd/headers/check"
+	"github.com/spf13/cobra"
+)
+
+func init() {
+	checkCmd.Flags().StringVar(&filesDir, "filesdir", "", "path to directory where files will be stored")
+	rootCmd.AddCommand(checkCmd)
+}
+
+var checkCmd = &cobra.Command{
+	Use:   "check",
+	Short: "Read the header files and print out anchor table",
+	RunE: func(cmd *cobra.Command, args []string) error {
+		return check.Check(filesDir)
+	},
+}
diff --git a/cmd/headers/commands/download.go b/cmd/headers/commands/download.go
index aa20f3becd457c71ba2db4eb16ee553f7dc461bb..2353c21a25f99370678270982a3d4068c3a182b5 100644
--- a/cmd/headers/commands/download.go
+++ b/cmd/headers/commands/download.go
@@ -6,7 +6,6 @@ import (
 )
 
 var (
-	filesDir   string // Directory when the files should be stored
 	bufferSize int    // Size of buffer in MiB
 	natSetting string // NAT setting
 	port       int    // Listening port
@@ -24,6 +23,6 @@ var downloadCmd = &cobra.Command{
 	Use:   "download",
 	Short: "Download headers backwards",
 	RunE: func(cmd *cobra.Command, args []string) error {
-		return download.Download(natSetting, filesDir, port)
+		return download.Download(natSetting, filesDir, bufferSize, port)
 	},
 }
diff --git a/cmd/headers/commands/root.go b/cmd/headers/commands/root.go
index 13df396652822010238117239660b2dac1ada1bc..0b0d6fe64fe8f5d5b3f2df0826e7e5990a619d02 100644
--- a/cmd/headers/commands/root.go
+++ b/cmd/headers/commands/root.go
@@ -13,6 +13,10 @@ import (
 	"github.com/spf13/cobra"
 )
 
+var (
+	filesDir string // Directory when the files should be stored
+)
+
 func init() {
 	utils.CobraFlags(rootCmd, append(debug.Flags, utils.MetricFlags...))
 }
diff --git a/cmd/headers/download/downloader.go b/cmd/headers/download/downloader.go
index 05a9b35b5e989dfd572bcfa9241b1312ded3fa9c..cf42db869fbf803d609df845583f413a17891843 100644
--- a/cmd/headers/download/downloader.go
+++ b/cmd/headers/download/downloader.go
@@ -9,7 +9,6 @@ import (
 	"os"
 	"time"
 
-	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/turbo-geth/common"
 	"github.com/ledgerwatch/turbo-geth/consensus/ethash"
 	"github.com/ledgerwatch/turbo-geth/core/types"
@@ -43,13 +42,18 @@ func processSegment(hd *headerdownload.HeaderDownload, segment *headerdownload.C
 		log.Error(fmt.Sprintf("FindTip penalty %d", penalty))
 		return
 	}
+	currentTime := uint64(time.Now().Unix())
 	var powDepth int
-	if powDepth1, err1 := hd.VerifySeals(segment, foundAnchor, start, end); err1 == nil {
+	if powDepth1, err1 := hd.VerifySeals(segment, foundAnchor, foundTip, start, end, currentTime); err1 == nil {
 		powDepth = powDepth1
 	} else {
 		log.Error("VerifySeals", "error", err1)
+		return
+	}
+	if err1 := hd.FlushBuffer(); err1 != nil {
+		log.Error("Could not flush the buffer, will discard the data", "error", err1)
+		return
 	}
-	currentTime := uint64(time.Now().Unix())
 	// There are 4 cases
 	if foundAnchor {
 		if foundTip {
@@ -57,6 +61,7 @@ func processSegment(hd *headerdownload.HeaderDownload, segment *headerdownload.C
 			if err1 := hd.Connect(segment, start, end, currentTime); err1 != nil {
 				log.Error("Connect failed", "error", err1)
 			} else {
+				hd.AddSegmentToBuffer(segment, start, end)
 				log.Info("Connected", "start", start, "end", end)
 			}
 		} else {
@@ -64,6 +69,7 @@ func processSegment(hd *headerdownload.HeaderDownload, segment *headerdownload.C
 			if err1 := hd.ExtendDown(segment, start, end, powDepth, currentTime); err1 != nil {
 				log.Error("ExtendDown failed", "error", err1)
 			} else {
+				hd.AddSegmentToBuffer(segment, start, end)
 				log.Info("Extended Down", "start", start, "end", end)
 			}
 		}
@@ -75,24 +81,29 @@ func processSegment(hd *headerdownload.HeaderDownload, segment *headerdownload.C
 			if err1 := hd.ExtendUp(segment, start, end, currentTime); err1 != nil {
 				log.Error("ExtendUp failed", "error", err1)
 			} else {
+				hd.AddSegmentToBuffer(segment, start, end)
 				log.Info("Extended Up", "start", start, "end", end)
 			}
 		}
 	} else {
 		// NewAnchor
-		if _, err1 := hd.NewAnchor(segment, start, end, currentTime); err1 != nil {
+		if err1 := hd.NewAnchor(segment, start, end, currentTime); err1 != nil {
 			log.Error("NewAnchor failed", "error", err1)
 		} else {
+			hd.AddSegmentToBuffer(segment, start, end)
 			log.Info("NewAnchor", "start", start, "end", end)
 		}
 	}
-
+	if start == 0 || end > 0 {
+		hd.CheckInitiation(segment, params.MainnetGenesisHash)
+	}
 }
 
 // Downloader needs to be run from a go-routine, and it is in the sole control of the HeaderDownloader object
 func Downloader(
 	ctx context.Context,
 	filesDir string,
+	bufferLimit int,
 	newBlockCh chan NewBlockFromSentry,
 	newBlockHashCh chan NewBlockHashFromSentry,
 	headersCh chan BlockHeadersFromSentry,
@@ -117,50 +128,49 @@ func Downloader(
 	}
 	hd := headerdownload.NewHeaderDownload(
 		filesDir,
-		16*1024, /* tipLimit */
-		1024,    /* initPowDepth */
+		bufferLimit, /* bufferLimit */
+		16*1024,     /* tipLimit */
+		1024,        /* initPowDepth */
 		calcDiffFunc,
 		verifySealFunc,
 		3600, /* newAnchor future limit */
 		3600, /* newAnchor past limit */
 	)
-	//if err := hd.RecoverFromFiles(); err != nil {
-	//	log.Error("Recovery from file failed, downloader not started", "error", err)
-	//}
-	// Insert hard-coded headers if present
-	if _, err := os.Stat("hard-coded-headers.dat"); err == nil {
-		if f, err1 := os.Open("hard-coded-headers.dat"); err1 == nil {
-			var hBuffer [headerdownload.HeaderSerLength]byte
-			var dBuffer [32]byte
-			i := 0
-			for {
-				var h types.Header
-				var d uint256.Int
-				if _, err2 := io.ReadFull(f, hBuffer[:]); err2 == nil {
-					headerdownload.DeserialiseHeader(&h, hBuffer[:])
-				} else if errors.Is(err2, io.EOF) {
-					break
-				} else {
-					log.Error("Failed to read hard coded header", "i", i, "error", err2)
-					break
-				}
-				if _, err2 := io.ReadFull(f, dBuffer[:]); err2 == nil {
-					d.SetBytes(dBuffer[:])
-				} else {
-					log.Error("Failed to read hard coded difficulty", "i", i, "error", err2)
-					break
-				}
-				if err2 := hd.HardCodedHeader(&h, d, uint64(time.Now().Unix())); err2 != nil {
-					log.Error("Failed to insert hard coded header", "i", i, "block", h.Number.Uint64(), "error", err2)
+	hd.InitHardCodedTips("hard-coded-headers.dat")
+	if recovered, err := hd.RecoverFromFiles(uint64(time.Now().Unix())); err != nil || !recovered {
+		if err != nil {
+			log.Error("Recovery from file failed, will start from scratch", "error", err)
+		}
+		// Insert hard-coded headers if present
+		if _, err := os.Stat("hard-coded-headers.dat"); err == nil {
+			if f, err1 := os.Open("hard-coded-headers.dat"); err1 == nil {
+				var hBuffer [headerdownload.HeaderSerLength]byte
+				i := 0
+				for {
+					var h types.Header
+					if _, err2 := io.ReadFull(f, hBuffer[:]); err2 == nil {
+						headerdownload.DeserialiseHeader(&h, hBuffer[:])
+					} else if errors.Is(err2, io.EOF) {
+						break
+					} else {
+						log.Error("Failed to read hard coded header", "i", i, "error", err2)
+						break
+					}
+					if err2 := hd.HardCodedHeader(&h, uint64(time.Now().Unix())); err2 != nil {
+						log.Error("Failed to insert hard coded header", "i", i, "block", h.Number.Uint64(), "error", err2)
+					} else {
+						hd.AddHeaderToBuffer(&h)
+					}
+					i++
 				}
-				i++
 			}
 		}
 	}
+	log.Info(hd.AnchorState())
 	for {
 		select {
 		case newBlockReq := <-newBlockCh:
-			if segments, penalty, err := hd.HandleNewBlockMsg(newBlockReq.Block.Header()); err == nil {
+			if segments, penalty, err := hd.SingleHeaderAsSegment(newBlockReq.Block.Header()); err == nil {
 				if penalty == headerdownload.NoPenalty {
 					processSegment(hd, segments[0]) // There is only one segment in this case
 				} else {
@@ -168,7 +178,7 @@ func Downloader(
 					penaltyCh <- PenaltyMsg{SentryMsg: newBlockReq.SentryMsg, penalty: penalty}
 				}
 			} else {
-				log.Error("HandleNewBlockMsg failed", "error", err)
+				log.Error("SingleHeaderAsSegment failed", "error", err)
 				continue
 			}
 			log.Info(fmt.Sprintf("NewBlockMsg{blockNumber: %d}", newBlockReq.Block.NumberU64()))
@@ -184,7 +194,7 @@ func Downloader(
 				}
 			}
 		case headersReq := <-headersCh:
-			if segments, penalty, err := hd.HandleHeadersMsg(headersReq.headers); err == nil {
+			if segments, penalty, err := hd.SplitIntoSegments(headersReq.headers); err == nil {
 				if penalty == headerdownload.NoPenalty {
 					for _, segment := range segments {
 						processSegment(hd, segment)
@@ -193,7 +203,7 @@ func Downloader(
 					penaltyCh <- PenaltyMsg{SentryMsg: headersReq.SentryMsg, penalty: penalty}
 				}
 			} else {
-				log.Error("HandleHeadersMsg failed", "error", err)
+				log.Error("SingleHeaderAsSegment failed", "error", err)
 			}
 			log.Info("HeadersMsg processed")
 		case <-hd.RequestQueueTimer.C:
diff --git a/cmd/headers/download/sentry.go b/cmd/headers/download/sentry.go
index 04e39b1bfbf4018d88a7f8493cc807759c3b5472..1c32d60db4dcaf6f03e832e0bd6763fb733757fb 100644
--- a/cmd/headers/download/sentry.go
+++ b/cmd/headers/download/sentry.go
@@ -378,7 +378,7 @@ func rootContext() context.Context {
 	return ctx
 }
 
-func Download(natSetting string, filesDir string, port int) error {
+func Download(natSetting string, filesDir string, bufferSize int, port int) error {
 	ctx := rootContext()
 	newBlockCh := make(chan NewBlockFromSentry)
 	newBlockHashCh := make(chan NewBlockHashFromSentry)
@@ -394,7 +394,7 @@ func Download(natSetting string, filesDir string, port int) error {
 	if err = server.Start(); err != nil {
 		return fmt.Errorf("could not start server: %w", err)
 	}
-	go Downloader(ctx, filesDir, newBlockCh, newBlockHashCh, headersCh, penaltyCh, reqHeadersCh)
+	go Downloader(ctx, filesDir, bufferSize*1024*1024, newBlockCh, newBlockHashCh, headersCh, penaltyCh, reqHeadersCh)
 
 	go func() {
 		for {
diff --git a/turbo/stages/headerdownload/header_algos.go b/turbo/stages/headerdownload/header_algos.go
index 6a2116032d0112572938bf61ae687429e3f88318..999995ada39b5ef253210baac139c7423cbd9e3a 100644
--- a/turbo/stages/headerdownload/header_algos.go
+++ b/turbo/stages/headerdownload/header_algos.go
@@ -2,13 +2,16 @@ package headerdownload
 
 import (
 	"bufio"
+	"bytes"
 	"container/heap"
+	"encoding/binary"
 	"errors"
 	"fmt"
 	"io"
 	"io/ioutil"
 	"math/big"
 	"os"
+	"path"
 	"sort"
 	"strings"
 	"time"
@@ -16,6 +19,7 @@ import (
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/turbo-geth/common"
 	"github.com/ledgerwatch/turbo-geth/core/types"
+	"github.com/ledgerwatch/turbo-geth/log"
 )
 
 // Implements sort.Interface so we can sort the incoming header in the message by block height
@@ -34,8 +38,8 @@ func (h HeadersByBlockHeight) Swap(i, j int) {
 	h[i], h[j] = h[j], h[i]
 }
 
-// HandleHeadersMsg converts message containing headers into a collection of chain segments
-func (hd *HeaderDownload) HandleHeadersMsg(msg []*types.Header) ([]*ChainSegment, Penalty, error) {
+// SplitIntoSegments converts message containing headers into a collection of chain segments
+func (hd *HeaderDownload) SplitIntoSegments(msg []*types.Header) ([]*ChainSegment, Penalty, error) {
 	sort.Sort(HeadersByBlockHeight(msg))
 	// Now all headers are order from the highest block height to the lowest
 	var segments []*ChainSegment                         // Segments being built
@@ -88,8 +92,8 @@ func (hd *HeaderDownload) childParentValid(child, parent *types.Header) (bool, P
 	return true, NoPenalty
 }
 
-// HandleNewBlockMsg converts message containing 1 header into one singleton chain segment
-func (hd *HeaderDownload) HandleNewBlockMsg(header *types.Header) ([]*ChainSegment, Penalty, error) {
+// SingleHeaderAsSegment converts message containing 1 header into one singleton chain segment
+func (hd *HeaderDownload) SingleHeaderAsSegment(header *types.Header) ([]*ChainSegment, Penalty, error) {
 	headerHash := header.Hash()
 	if _, bad := hd.badHeaders[headerHash]; bad {
 		return nil, BadBlockPenalty, nil
@@ -101,16 +105,15 @@ func (hd *HeaderDownload) HandleNewBlockMsg(header *types.Header) ([]*ChainSegme
 func (hd *HeaderDownload) FindAnchors(segment *ChainSegment) (found bool, start int, anchorParent common.Hash, invalidAnchors []int) {
 	// Walk the segment from children towards parents
 	for i, header := range segment.Headers {
-		headerHash := header.Hash()
 		// Check if the header can be attached to an anchor of a working tree
-		if anchors, attaching := hd.anchors[headerHash]; attaching {
+		if anchors, attaching := hd.anchors[header.Hash()]; attaching {
 			var invalidAnchors []int
 			for anchorIdx, anchor := range anchors {
 				if valid := hd.anchorParentValid(anchor, header); !valid {
 					invalidAnchors = append(invalidAnchors, anchorIdx)
 				}
 			}
-			return true, i, headerHash, invalidAnchors
+			return true, i, header.Hash(), invalidAnchors
 		}
 	}
 	return false, 0, common.Hash{}, nil
@@ -125,8 +128,10 @@ func (hd *HeaderDownload) InvalidateAnchors(anchorParent common.Hash, invalidAnc
 			for k, anchor := range anchors {
 				if j < len(invalidAnchors) && invalidAnchors[j] == k {
 					// Invalidate the entire tree that is rooted at this anchor anchor
-					for _, tipHash := range anchor.tips {
-						hd.tips.Remove(tipHash)
+					hd.anchorTree.Delete(anchor)
+					for _, anchorTipItem := range *anchor.tipQueue {
+						delete(hd.tips, anchorTipItem.hash)
+						hd.tipCount--
 					}
 					tombstones = append(tombstones, anchor.hash)
 					j++
@@ -149,13 +154,13 @@ func (hd *HeaderDownload) InvalidateAnchors(anchorParent common.Hash, invalidAnc
 // FindTip attempts to find tip of a tree that given chain segment can be attached to
 // the given chain segment may be found invalid relative to a working tree, in this case penalty for peer is returned
 func (hd *HeaderDownload) FindTip(segment *ChainSegment, start int) (found bool, end int, penalty Penalty) {
-	if _, duplicate := hd.getTip(segment.Headers[start].Hash(), false); duplicate {
+	if _, duplicate := hd.getTip(segment.Headers[start].Hash()); duplicate {
 		return true, 0, NoPenalty
 	}
 	// Walk the segment from children towards parents
 	for i, header := range segment.Headers[start:] {
 		// Check if the header can be attached to any tips
-		if tip, attaching := hd.getTip(header.ParentHash, false); attaching {
+		if tip, attaching := hd.getTip(header.ParentHash); attaching {
 			// Before attaching, we must check the parent-child relationship
 			if valid, penalty := hd.childTipValid(header, header.ParentHash, tip); !valid {
 				return true, start + i + 1, penalty
@@ -169,7 +174,17 @@ func (hd *HeaderDownload) FindTip(segment *ChainSegment, start int) (found bool,
 // VerifySeals verifies Proof Of Work for the part of the given chain segment
 // It reports first verification error, or returns the powDepth that the anchor of this
 // chain segment should have, if created
-func (hd *HeaderDownload) VerifySeals(segment *ChainSegment, anchorFound bool, start, end int) (powDepth int, err error) {
+func (hd *HeaderDownload) VerifySeals(segment *ChainSegment, anchorFound, tipFound bool, start, end int, currentTime uint64) (powDepth int, err error) {
+	if !anchorFound && !tipFound {
+		anchorHeader := segment.Headers[end-1]
+		if anchorHeader.Time > currentTime+hd.newAnchorFutureLimit {
+			return 0, fmt.Errorf("detached segment too far in the future")
+		}
+		if anchorHeader.Time+hd.newAnchorPastLimit < currentTime {
+			return 0, fmt.Errorf("detached segment too far in the past")
+		}
+	}
+
 	var powDepthSet bool
 	if anchorFound {
 		if anchors, ok := hd.anchors[segment.Headers[start].Hash()]; ok {
@@ -200,10 +215,7 @@ func (hd *HeaderDownload) VerifySeals(segment *ChainSegment, anchorFound bool, s
 func (hd *HeaderDownload) ExtendUp(segment *ChainSegment, start, end int, currentTime uint64) error {
 	// Find attachment tip again
 	tipHeader := segment.Headers[end-1]
-	if attachmentTip, attaching := hd.getTip(tipHeader.ParentHash, true); attaching {
-		if attachmentTip.noPrepend {
-			return fmt.Errorf("extendUp attachment tip had noPrepend flag on for %x", tipHeader.ParentHash)
-		}
+	if attachmentTip, attaching := hd.getTip(tipHeader.ParentHash); attaching {
 		newAnchor := attachmentTip.anchor
 		cumulativeDifficulty := attachmentTip.cumulativeDifficulty
 		// Iterate over headers backwards (from parents towards children), to be able calculate cumulative difficulty along the way
@@ -241,12 +253,39 @@ func (hd *HeaderDownload) ExtendDown(segment *ChainSegment, start, end int, powD
 			difficulty:  *diff,
 			hash:        newAnchorHeader.Hash(),
 			blockHeight: newAnchorHeader.Number.Uint64(),
+			tipQueue:    &AnchorTipQueue{},
+			anchorID:    hd.nextAnchorID,
 		}
+		hd.nextAnchorID++
+		heap.Init(newAnchor.tipQueue)
 		hd.anchors[newAnchorHeader.ParentHash] = append(hd.anchors[newAnchorHeader.ParentHash], newAnchor)
-		heap.Push(hd.requestQueue, RequestQueueItem{anchorParent: newAnchorHeader.ParentHash, waitUntil: currentTime})
-
+		// Iterate headers in the segment to compute difficulty difference along the way
+		var difficultyDifference uint256.Int
+		for _, header := range segment.Headers[start:end] {
+			diff, overflow := uint256.FromBig(header.Difficulty)
+			if overflow {
+				return fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
+			}
+			difficultyDifference.Add(&difficultyDifference, diff)
+		}
+		// Go over tips of the anchors we are replacing, bump their cumulative difficulty, and add them to the new anchor
+		for _, anchor := range anchors {
+			hd.anchorTree.Delete(anchor)
+			for _, tipQueueItem := range *anchor.tipQueue {
+				if tip, ok := hd.getTip(tipQueueItem.hash); ok {
+					tip.cumulativeDifficulty.Add(&tip.cumulativeDifficulty, &difficultyDifference)
+					tip.anchor = newAnchor
+					heap.Push(newAnchor.tipQueue, tipQueueItem)
+					if tip.blockHeight > newAnchor.maxTipHeight {
+						newAnchor.maxTipHeight = tip.blockHeight
+					}
+				}
+			}
+		}
+		delete(hd.anchors, anchorHeader.Hash())
+		hd.anchorTree.ReplaceOrInsert(newAnchor)
 		// Add all headers in the segments as tips to this anchor
-		// Iterate in reverse order to be able to compute cumulative difficulty along the way
+		// Recalculate cumulative difficulty for each header
 		var cumulativeDifficulty uint256.Int
 		for i := end - 1; i >= start; i-- {
 			header := segment.Headers[i]
@@ -259,17 +298,7 @@ func (hd *HeaderDownload) ExtendDown(segment *ChainSegment, start, end int, powD
 				return fmt.Errorf("extendUp addHeaderAsTip for %x: %v", header.Hash(), err)
 			}
 		}
-		// Go over tips of the anchors we are replacing, bump their cumulative difficulty, and add them to the new anchor
-		for _, anchor := range anchors {
-			for _, tipHash := range anchor.tips {
-				if tip, ok := hd.getTip(tipHash, false); ok {
-					tip.cumulativeDifficulty.Add(&tip.cumulativeDifficulty, &cumulativeDifficulty)
-					tip.anchor = newAnchor
-					newAnchor.tips = append(newAnchor.tips, tipHash)
-				}
-			}
-		}
-		delete(hd.anchors, anchorHeader.Hash())
+		hd.requestQueue.PushFront(RequestQueueItem{anchorParent: newAnchorHeader.ParentHash, waitUntil: currentTime})
 	} else {
 		return fmt.Errorf("extendDown attachment anchors not found for %x", anchorHeader.Hash())
 	}
@@ -282,7 +311,7 @@ func (hd *HeaderDownload) Connect(segment *ChainSegment, start, end int, current
 	tipHeader := segment.Headers[end-1]
 	// Find attachement anchors again
 	anchorHeader := segment.Headers[start]
-	attachmentTip, ok1 := hd.getTip(tipHeader.ParentHash, true)
+	attachmentTip, ok1 := hd.getTip(tipHeader.ParentHash)
 	if !ok1 {
 		return fmt.Errorf("connect attachment tip not found for %x", tipHeader.ParentHash)
 	}
@@ -291,7 +320,33 @@ func (hd *HeaderDownload) Connect(segment *ChainSegment, start, end int, current
 		return fmt.Errorf("connect attachment anchors not found for %x", anchorHeader.Hash())
 	}
 	newAnchor := attachmentTip.anchor
+	// Iterate headers in the segment to compute difficulty difference along the way
+	difficultyDifference := attachmentTip.cumulativeDifficulty
+	for _, header := range segment.Headers[start:end] {
+		diff, overflow := uint256.FromBig(header.Difficulty)
+		if overflow {
+			return fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
+		}
+		difficultyDifference.Add(&difficultyDifference, diff)
+	}
+	hd.anchorTree.Delete(newAnchor)
+	// Go over tips of the anchors we are replacing, bump their cumulative difficulty, and add them to the new anchor
+	for _, anchor := range anchors {
+		hd.anchorTree.Delete(anchor)
+		for _, tipQueueItem := range *anchor.tipQueue {
+			if tip, ok := hd.getTip(tipQueueItem.hash); ok {
+				tip.cumulativeDifficulty.Add(&tip.cumulativeDifficulty, &difficultyDifference)
+				tip.anchor = newAnchor
+				heap.Push(newAnchor.tipQueue, tipQueueItem)
+				if tip.blockHeight > newAnchor.maxTipHeight {
+					newAnchor.maxTipHeight = tip.blockHeight
+				}
+			}
+		}
+	}
 	cumulativeDifficulty := attachmentTip.cumulativeDifficulty
+	delete(hd.anchors, anchorHeader.Hash())
+	hd.anchorTree.ReplaceOrInsert(newAnchor)
 	// Iterate over headers backwards (from parents towards children), to be able calculate cumulative difficulty along the way
 	for i := end - 1; i >= start; i-- {
 		header := segment.Headers[i]
@@ -304,35 +359,15 @@ func (hd *HeaderDownload) Connect(segment *ChainSegment, start, end int, current
 			return fmt.Errorf("extendUp addHeaderAsTip for %x: %v", header.Hash(), err)
 		}
 	}
-	// Go over tips of the anchors we are replacing, bump their cumulative difficulty, and add them to the new anchor
-	for _, anchor := range anchors {
-		for _, tipHash := range anchor.tips {
-			if tip, ok := hd.getTip(tipHash, false); ok {
-				tip.cumulativeDifficulty.Add(&tip.cumulativeDifficulty, &cumulativeDifficulty)
-				tip.anchor = newAnchor
-				newAnchor.tips = append(newAnchor.tips, tipHash)
-			}
-		}
-	}
-	delete(hd.anchors, anchorHeader.Hash())
 	return nil
 }
 
-func (hd *HeaderDownload) NewAnchor(segment *ChainSegment, start, end int, currentTime uint64) (Penalty, error) {
+func (hd *HeaderDownload) NewAnchor(segment *ChainSegment, start, end int, currentTime uint64) error {
 	anchorHeader := segment.Headers[end-1]
-	if anchorHeader.Time > currentTime+hd.newAnchorFutureLimit {
-		return TooFarFuturePenalty, nil
-	}
-	if anchorHeader.Time+hd.newAnchorPastLimit < currentTime {
-		return TooFarPastPenalty, nil
-	}
 	var anchor *Anchor
 	var err error
-	if anchor, err = hd.addHeaderAsAnchor(anchorHeader, hd.initPowDepth, uint256.Int{}); err != nil {
-		return NoPenalty, err
-	}
-	if anchorHeader.ParentHash != (common.Hash{}) {
-		heap.Push(hd.requestQueue, RequestQueueItem{anchorParent: anchorHeader.ParentHash, waitUntil: currentTime})
+	if anchor, err = hd.addHeaderAsAnchor(anchorHeader, hd.initPowDepth); err != nil {
+		return err
 	}
 	cumulativeDifficulty := uint256.Int{}
 	// Iterate over headers backwards (from parents towards children), to be able calculate cumulative difficulty along the way
@@ -340,36 +375,45 @@ func (hd *HeaderDownload) NewAnchor(segment *ChainSegment, start, end int, curre
 		header := segment.Headers[i]
 		diff, overflow := uint256.FromBig(header.Difficulty)
 		if overflow {
-			return NoPenalty, fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
+			return fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
 		}
 		cumulativeDifficulty.Add(&cumulativeDifficulty, diff)
 		if err = hd.addHeaderAsTip(header, anchor, cumulativeDifficulty, currentTime); err != nil {
-			return NoPenalty, fmt.Errorf("newAnchor addHeaderAsTip for %x: %v", header.Hash(), err)
+			return fmt.Errorf("newAnchor addHeaderAsTip for %x: %v", header.Hash(), err)
 		}
 	}
-	return NoPenalty, nil
+	if anchorHeader.ParentHash != (common.Hash{}) {
+		hd.requestQueue.PushFront(RequestQueueItem{anchorParent: anchorHeader.ParentHash, waitUntil: currentTime})
+	}
+	return nil
 }
 
-func (hd *HeaderDownload) HardCodedHeader(header *types.Header, totalDifficulty uint256.Int, currentTime uint64) error {
-	if anchor, err := hd.addHeaderAsAnchor(header, 0 /* powDepth */, totalDifficulty); err == nil {
+func (hd *HeaderDownload) HardCodedHeader(header *types.Header, currentTime uint64) error {
+	if anchor, err := hd.addHeaderAsAnchor(header, 0 /* powDepth */); err == nil {
 		diff, overflow := uint256.FromBig(header.Difficulty)
 		if overflow {
 			return fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
 		}
 		tip := &Tip{
 			anchor:               anchor,
-			cumulativeDifficulty: totalDifficulty,
+			cumulativeDifficulty: *diff,
 			timestamp:            header.Time,
 			blockHeight:          header.Number.Uint64(),
 			uncleHash:            header.UncleHash,
 			difficulty:           *diff,
-			noPrepend:            true,
 		}
 		tipHash := header.Hash()
-		hd.hardTips[tipHash] = tip
-		anchor.tips = append(anchor.tips, tipHash)
+		hd.tips[tipHash] = tip
+		_, hard := hd.hardTips[tipHash]
+		hd.tips[tipHash] = tip
+		heap.Push(anchor.tipQueue, AnchorTipItem{hash: tipHash, height: tip.blockHeight, hard: hard})
+		hd.tipCount++
+		if tip.blockHeight > anchor.maxTipHeight {
+			anchor.maxTipHeight = tip.blockHeight
+		}
+		hd.anchorTree.ReplaceOrInsert(anchor)
 		if header.ParentHash != (common.Hash{}) {
-			heap.Push(hd.requestQueue, RequestQueueItem{anchorParent: header.ParentHash, waitUntil: currentTime})
+			hd.requestQueue.PushFront(RequestQueueItem{anchorParent: header.ParentHash, waitUntil: currentTime})
 		}
 	} else {
 		return err
@@ -377,7 +421,11 @@ func (hd *HeaderDownload) HardCodedHeader(header *types.Header, totalDifficulty
 	return nil
 }
 
-func (hd *HeaderDownload) AddToBuffer(segment *ChainSegment, start, end int) {
+// AddSegmentToBuffer adds another segment to the buffer and return true if the buffer is now full
+func (hd *HeaderDownload) AddSegmentToBuffer(segment *ChainSegment, start, end int) {
+	if end > start {
+		fmt.Printf("Adding segment [%d-%d] to the buffer\n", segment.Headers[end-1].Number.Uint64(), segment.Headers[start].Number.Uint64())
+	}
 	var serBuffer [HeaderSerLength]byte
 	for _, header := range segment.Headers[start:end] {
 		SerialiseHeader(header, serBuffer[:])
@@ -385,6 +433,13 @@ func (hd *HeaderDownload) AddToBuffer(segment *ChainSegment, start, end int) {
 	}
 }
 
+func (hd *HeaderDownload) AddHeaderToBuffer(header *types.Header) {
+	fmt.Printf("Adding header %d to the buffer\n", header.Number.Uint64())
+	var serBuffer [HeaderSerLength]byte
+	SerialiseHeader(header, serBuffer[:])
+	hd.buffer = append(hd.buffer, serBuffer[:]...)
+}
+
 func (hd *HeaderDownload) AnchorState() string {
 	var ss = make([]string, len(hd.anchors))
 	var j int
@@ -394,22 +449,50 @@ func (hd *HeaderDownload) AnchorState() string {
 			if i > 0 {
 				sb.WriteString("; ")
 			}
-			sb.WriteString(fmt.Sprintf("{%8d-", anchor.blockHeight))
-			var end uint64
-			var count int
-			for _, tipHash := range anchor.tips {
-				if tip, ok := hd.getTip(tipHash, false); ok {
-					if tip.blockHeight > end {
-						end = tip.blockHeight
+			sb.WriteString(fmt.Sprintf("{%8d", anchor.blockHeight))
+			end := anchor.maxTipHeight
+			var sbb strings.Builder
+			var bs []int
+			for _, tipQueueItem := range *anchor.tipQueue {
+				bs = append(bs, int(tipQueueItem.height))
+			}
+			sort.Ints(bs)
+			for j, b := range bs {
+				if j == 0 {
+					sbb.WriteString(fmt.Sprintf("%d", b))
+				} else if j == len(bs)-1 {
+					if bs[j-1]+1 == b {
+						// Close interval
+						sbb.WriteString(fmt.Sprintf("-%d", b))
+					} else {
+						// Standalone
+						sbb.WriteString(fmt.Sprintf(" %d", b))
+					}
+				} else {
+					if bs[j-1] == b {
+						// Skip
+					} else if bs[j-1]+1 == b {
+						if b+1 == bs[j+1] {
+							// Skip
+						} else {
+							// Close interval
+							sbb.WriteString(fmt.Sprintf("-%d", b))
+						}
+					} else {
+						// Open interval or standalone
+						sbb.WriteString(fmt.Sprintf(" %d", b))
 					}
-					count++
 				}
 			}
-			sb.WriteString(fmt.Sprintf("%d (%d) tips=%d}", end, end-anchor.blockHeight, count))
+			if end == 0 {
+				sb.WriteString(fmt.Sprintf(" HardCoded tips=%d tipStretch=%d (%s)}", anchor.tipQueue.Len(), anchor.tipStretch(), sbb.String()))
+			} else {
+				sb.WriteString(fmt.Sprintf("-%d (%d) tips=%d tipStretch=%d (%s)}", end, end-anchor.blockHeight, anchor.tipQueue.Len(), anchor.tipStretch(), sbb.String()))
+			}
 		}
 		sb.WriteString(fmt.Sprintf(" => %x", anchorParent))
-		j++
 		ss[j] = sb.String()
+		j++
 	}
 	sort.Strings(ss)
 	return strings.Join(ss, "\n")
@@ -420,6 +503,7 @@ type HeapElem struct {
 	file        *os.File
 	reader      io.Reader
 	blockHeight uint64
+	hash        common.Hash
 	header      *types.Header
 }
 
@@ -430,6 +514,9 @@ func (h Heap) Len() int {
 }
 
 func (h Heap) Less(i, j int) bool {
+	if h[i].blockHeight == h[j].blockHeight {
+		return bytes.Compare(h[i].hash[:], h[j].hash[:]) < 0
+	}
 	return h[i].blockHeight < h[j].blockHeight
 }
 
@@ -451,20 +538,130 @@ func (h *Heap) Pop() interface{} {
 	return x
 }
 
-func (hd *HeaderDownload) RecoverFromFiles(currentTime uint64) error {
+const AnchorSerLen = 32 /* ParentHash */ + 8 /* powDepth */ + 8 /* maxTipHeight */
+
+func (hd *HeaderDownload) CheckFiles() error {
 	fileInfos, err := ioutil.ReadDir(hd.filesDir)
 	if err != nil {
 		return err
 	}
+	var buffer [HeaderSerLength]byte
+	var anchorBuf [AnchorSerLen]byte
+	for _, fileInfo := range fileInfos {
+		f, err1 := os.Open(path.Join(hd.filesDir, fileInfo.Name()))
+		if err1 != nil {
+			return fmt.Errorf("open file %s: %v", fileInfo.Name(), err1)
+		}
+		r := bufio.NewReader(f)
+		if _, err = io.ReadFull(r, anchorBuf[:8]); err != nil {
+			fmt.Printf("reading anchor sequence and count from file: %v\n", err)
+			continue
+		}
+		anchorCount := int(binary.BigEndian.Uint32((anchorBuf[4:])))
+		for i := 0; i < anchorCount; i++ {
+			if _, err = io.ReadFull(r, anchorBuf[:]); err != nil {
+				fmt.Printf("reading anchor %x from file: %v\n", i, err)
+			}
+		}
+		for {
+			var header types.Header
+			if _, err = io.ReadFull(r, buffer[:]); err != nil {
+				if !errors.Is(err, io.EOF) {
+					fmt.Printf("reading header from file: %v\n", err)
+				}
+				break
+			}
+			DeserialiseHeader(&header, buffer[:])
+			fmt.Printf("Read header %d from file %s\n", header.Number.Uint64(), fileInfo.Name())
+		}
+	}
+	return nil
+}
+
+func (hd *HeaderDownload) InitHardCodedTips(filename string) {
+	// Insert hard-coded headers if present
+	if _, err := os.Stat(filename); err == nil {
+		if f, err1 := os.Open(filename); err1 == nil {
+			var hBuffer [HeaderSerLength]byte
+			for {
+				var h types.Header
+				if _, err2 := io.ReadFull(f, hBuffer[:]); err2 == nil {
+					DeserialiseHeader(&h, hBuffer[:])
+				} else if errors.Is(err2, io.EOF) {
+					break
+				} else {
+					log.Error("Failed to read hard coded header", "error", err2)
+					break
+				}
+				hd.hardTips[h.Hash()] = struct{}{}
+			}
+		} else {
+			log.Error("Failed to open hard-coded headers", "file", filename, "error", err1)
+		}
+	} else {
+		log.Error("Failed to stat hard-coded headers", "file", filename, "error", err)
+	}
+}
+
+func (hd *HeaderDownload) RecoverFromFiles(currentTime uint64) (bool, error) {
+	fileInfos, err := ioutil.ReadDir(hd.filesDir)
+	if err != nil {
+		return false, err
+	}
 	h := &Heap{}
 	heap.Init(h)
 	var buffer [HeaderSerLength]byte
+	var anchorBuf [AnchorSerLen]byte
+	//nolint:prealloc
+	var fs []*os.File
+	//nolint:prealloc
+	var rs []io.Reader
+	// Open all files and only read anchor sequences to decide which one has the latest information about the anchors
+	hd.anchorSequence = 0
+	var lastAnchors = make(map[common.Hash]*Anchor)
 	for _, fileInfo := range fileInfos {
-		f, err1 := os.Open(fileInfo.Name())
+		f, err1 := os.Open(path.Join(hd.filesDir, fileInfo.Name()))
 		if err1 != nil {
-			return fmt.Errorf("open file %s: %v", fileInfo.Name(), err1)
+			return false, fmt.Errorf("open file %s: %v", fileInfo.Name(), err1)
 		}
 		r := bufio.NewReader(f)
+		if _, err = io.ReadFull(r, anchorBuf[:8]); err != nil {
+			fmt.Printf("reading anchor sequence and count from file: %v\n", err)
+			continue
+		}
+		anchorSequence := binary.BigEndian.Uint32(anchorBuf[:])
+		anchorCount := int(binary.BigEndian.Uint32((anchorBuf[4:])))
+		var anchors = make(map[common.Hash]*Anchor)
+		if anchorSequence >= hd.anchorSequence {
+			fmt.Printf("Reading anchor sequence %d, anchor count: %d\n", anchorSequence, anchorCount)
+		}
+		for i := 0; i < anchorCount; i++ {
+			if _, err = io.ReadFull(r, anchorBuf[:]); err != nil {
+				fmt.Printf("reading anchor %x from file: %v\n", i, err)
+			}
+			if anchorSequence >= hd.anchorSequence { // Don't bother with parsing if we are not going to use this info
+				anchor := &Anchor{tipQueue: &AnchorTipQueue{}, anchorID: hd.nextAnchorID}
+				hd.nextAnchorID++
+				heap.Init(anchor.tipQueue)
+				pos := 0
+				copy(anchor.hash[:], anchorBuf[pos:])
+				pos += 32
+				anchor.powDepth = int(binary.BigEndian.Uint64(anchorBuf[pos:]))
+				pos += 8
+				anchor.maxTipHeight = binary.BigEndian.Uint64(anchorBuf[pos:])
+				anchors[anchor.hash] = anchor
+				fmt.Printf("anchor: %x, powDepth: %d, maxTipHeight %d\n", anchor.hash, anchor.powDepth, anchor.maxTipHeight)
+			}
+		}
+		if anchorSequence >= hd.anchorSequence {
+			hd.anchorSequence = anchorSequence + 1
+			lastAnchors = anchors
+		}
+		fs = append(fs, f)
+		rs = append(rs, r)
+	}
+	for i, f := range fs {
+		r := rs[i]
 		var header types.Header
 		if _, err = io.ReadFull(r, buffer[:]); err != nil {
 			if !errors.Is(err, io.EOF) {
@@ -473,7 +670,7 @@ func (hd *HeaderDownload) RecoverFromFiles(currentTime uint64) error {
 			continue
 		}
 		DeserialiseHeader(&header, buffer[:])
-		he := HeapElem{file: f, reader: r, blockHeight: header.Number.Uint64(), header: &header}
+		he := HeapElem{file: f, reader: r, blockHeight: header.Number.Uint64(), hash: header.Hash(), header: &header}
 		heap.Push(h, he)
 	}
 	var prevHeight uint64
@@ -481,46 +678,74 @@ func (hd *HeaderDownload) RecoverFromFiles(currentTime uint64) error {
 	var parentDiffs = make(map[common.Hash]*uint256.Int)
 	var childAnchors = make(map[common.Hash]*Anchor)
 	var childDiffs = make(map[common.Hash]*uint256.Int)
+	var prevHash common.Hash // Hash of previously seen header - to filter out potential duplicates
 	for h.Len() > 0 {
 		he := (heap.Pop(h)).(HeapElem)
-		if he.blockHeight > prevHeight {
-			// Clear out parent map and move childMap to its place
-			childAnchors = make(map[common.Hash]*Anchor)
-			childDiffs = make(map[common.Hash]*uint256.Int)
-			if he.blockHeight == prevHeight+1 {
+		hash := he.header.Hash()
+		if hash != prevHash {
+			if he.blockHeight > prevHeight {
+				// Clear out parent map and move childMap to its place
 				parentAnchors = childAnchors
 				parentDiffs = childDiffs
-			} else {
-				// Skipping the level, so no connection between grand-parents and grand-children
-				parentAnchors = make(map[common.Hash]*Anchor)
-				parentDiffs = make(map[common.Hash]*uint256.Int)
+				childAnchors = make(map[common.Hash]*Anchor)
+				childDiffs = make(map[common.Hash]*uint256.Int)
+				if he.blockHeight != prevHeight+1 {
+					// Skipping the level, so no connection between grand-parents and grand-children
+					parentAnchors = make(map[common.Hash]*Anchor)
+					parentDiffs = make(map[common.Hash]*uint256.Int)
+				}
+				prevHeight = he.blockHeight
 			}
-			prevHeight = he.blockHeight
-		}
-		// Since this header has already been processed, we do not expect overflow
-		headerDiff, overflow := uint256.FromBig(he.header.Difficulty)
-		if overflow {
-			return fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", he.header.Difficulty)
-		}
-		if parentAnchor, found := parentAnchors[he.header.ParentHash]; found {
-			parentDiff := parentDiffs[he.header.ParentHash]
-			cumulativeDiff := headerDiff.Add(headerDiff, parentDiff)
-			if err = hd.addHeaderAsTip(he.header, parentAnchor, *cumulativeDiff, currentTime); err != nil {
-				return fmt.Errorf("add header as tip: %v", err)
+			// Since this header has already been processed, we do not expect overflow
+			cumulativeDiff, overflow := uint256.FromBig(he.header.Difficulty)
+			if overflow {
+				return false, fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", he.header.Difficulty)
 			}
-		} else {
-			// Add header as anchor
-			//TODO - persist powDepth and totalDifficulty
-			if parentAnchor, err = hd.addHeaderAsAnchor(he.header, hd.initPowDepth, uint256.Int{}); err != nil {
-				return fmt.Errorf("add header as anchor: %v", err)
+			parentHash := he.header.ParentHash
+			if parentAnchor, found := parentAnchors[parentHash]; found {
+				parentDiff := parentDiffs[parentHash]
+				cumulativeDiff.Add(cumulativeDiff, parentDiff)
+				if err = hd.addHeaderAsTip(he.header, parentAnchor, *cumulativeDiff, currentTime); err != nil {
+					return false, fmt.Errorf("add header as tip: %v", err)
+				}
+				childAnchors[hash] = parentAnchor
+				childDiffs[hash] = cumulativeDiff
+			} else {
+				anchor, anchorExisted := lastAnchors[hash]
+				if !anchorExisted {
+					anchor = &Anchor{powDepth: hd.initPowDepth, hash: hash, tipQueue: &AnchorTipQueue{}, anchorID: hd.nextAnchorID}
+					hd.nextAnchorID++
+					heap.Init(anchor.tipQueue)
+					fmt.Printf("Undeclared anchor for hash %x, inserting as empty\n", hash)
+				}
+				diff, overflow := uint256.FromBig(he.header.Difficulty)
+				if overflow {
+					return false, fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", he.header.Difficulty)
+				}
+				anchor.difficulty = *diff
+				anchor.timestamp = he.header.Time
+				anchor.blockHeight = he.header.Number.Uint64()
+				if err = hd.addHeaderAsTip(he.header, anchor, *cumulativeDiff, currentTime); err != nil {
+					return false, fmt.Errorf("add header as tip: %v", err)
+				}
+				if len(hd.anchors[parentHash]) == 0 {
+					if parentHash != (common.Hash{}) {
+						hd.requestQueue.PushFront(RequestQueueItem{anchorParent: parentHash, waitUntil: currentTime})
+					}
+				}
+				hd.anchors[parentHash] = append(hd.anchors[parentHash], anchor)
+				childAnchors[hash] = anchor
+				childDiffs[hash] = cumulativeDiff
 			}
-			childAnchors[he.header.Hash()] = parentAnchor
-			childDiffs[he.header.Hash()] = new(uint256.Int)
+			prevHash = hash
+		} else {
+			fmt.Printf("Duplicate header: %d %x\n", he.header.Number.Uint64(), hash)
 		}
 		var header types.Header
 		if _, err = io.ReadFull(he.reader, buffer[:]); err == nil {
 			DeserialiseHeader(&header, buffer[:])
 			he.blockHeight = header.Number.Uint64()
+			he.hash = header.Hash()
 			he.header = &header
 			heap.Push(h, he)
 		} else {
@@ -532,30 +757,22 @@ func (hd *HeaderDownload) RecoverFromFiles(currentTime uint64) error {
 			}
 		}
 	}
-	return nil
+	return hd.anchorSequence > 0, nil
 }
 
 func (hd *HeaderDownload) RequestMoreHeaders(currentTime, timeout uint64) []*HeaderRequest {
 	if hd.requestQueue.Len() == 0 {
 		return nil
 	}
-	var prevTopTime uint64
-	if hd.requestQueue.Len() > 0 {
-		prevTopTime = (*hd.requestQueue)[0].waitUntil
-	}
+	var prevTopTime uint64 = hd.requestQueue.Front().Value.(RequestQueueItem).waitUntil
 	var requests []*HeaderRequest
-	peek := (*hd.requestQueue)[0]
-	for hd.requestQueue.Len() > 0 && peek.waitUntil <= currentTime {
-		pop := heap.Pop(hd.requestQueue).(RequestQueueItem)
-		if anchors, present := hd.anchors[pop.anchorParent]; present {
+	for peek := hd.requestQueue.Front(); peek != nil && peek.Value.(RequestQueueItem).waitUntil <= currentTime; peek = hd.requestQueue.Front() {
+		hd.requestQueue.Remove(peek)
+		item := peek.Value.(RequestQueueItem)
+		if anchors, present := hd.anchors[item.anchorParent]; present {
 			// Anchor still exists after the timeout
-			//TODO: Figure out correct request length
-			requests = append(requests, &HeaderRequest{Hash: pop.anchorParent, Number: anchors[0].blockHeight - 1, Length: 192})
-			pop.waitUntil = currentTime + timeout
-			heap.Push(hd.requestQueue, pop)
-		}
-		if hd.requestQueue.Len() > 0 {
-			peek = (*hd.requestQueue)[0]
+			requests = append(requests, &HeaderRequest{Hash: item.anchorParent, Number: anchors[0].blockHeight - 1, Length: 192})
+			hd.requestQueue.PushBack(RequestQueueItem{anchorParent: item.anchorParent, waitUntil: currentTime + timeout})
 		}
 	}
 	hd.resetRequestQueueTimer(prevTopTime, currentTime)
@@ -565,7 +782,7 @@ func (hd *HeaderDownload) RequestMoreHeaders(currentTime, timeout uint64) []*Hea
 func (hd *HeaderDownload) resetRequestQueueTimer(prevTopTime, currentTime uint64) {
 	var nextTopTime uint64
 	if hd.requestQueue.Len() > 0 {
-		nextTopTime = (*hd.requestQueue)[0].waitUntil
+		nextTopTime = hd.requestQueue.Front().Value.(RequestQueueItem).waitUntil
 	}
 	if nextTopTime == prevTopTime {
 		return // Nothing changed
@@ -579,9 +796,39 @@ func (hd *HeaderDownload) resetRequestQueueTimer(prevTopTime, currentTime uint64
 }
 
 func (hd *HeaderDownload) FlushBuffer() error {
+	if len(hd.buffer) < hd.bufferLimit {
+		// Not flushing the buffer unless it is full
+		return nil
+	}
 	// Sort the buffer first
 	sort.Sort(BufferSorter(hd.buffer))
 	if bufferFile, err := ioutil.TempFile(hd.filesDir, "headers-buf"); err == nil {
+		// First write the anchors
+		var buf [AnchorSerLen]byte
+		binary.BigEndian.PutUint32(buf[:], hd.anchorSequence)
+		anchorCount := 0
+		for _, anchors := range hd.anchors {
+			anchorCount += len(anchors)
+		}
+		binary.BigEndian.PutUint32(buf[4:], uint32(anchorCount))
+		if _, err = bufferFile.Write(buf[:8]); err != nil {
+			bufferFile.Close()
+			return err
+		}
+		for _, anchors := range hd.anchors {
+			for _, anchor := range anchors {
+				pos := 0
+				copy(buf[pos:], anchor.hash[:])
+				pos += 32
+				binary.BigEndian.PutUint64(buf[pos:], uint64(anchor.powDepth))
+				pos += 8
+				binary.BigEndian.PutUint64(buf[pos:], anchor.maxTipHeight)
+				if _, err = bufferFile.Write(buf[:]); err != nil {
+					bufferFile.Close()
+					return err
+				}
+			}
+		}
 		if _, err = bufferFile.Write(hd.buffer); err != nil {
 			bufferFile.Close()
 			return err
@@ -590,9 +837,11 @@ func (hd *HeaderDownload) FlushBuffer() error {
 			return err
 		}
 		hd.buffer = hd.buffer[:0]
+		hd.anchorSequence++
 	} else {
 		return err
 	}
+	fmt.Printf("Successfully flushed the buffer\n")
 	return nil
 }
 
@@ -600,14 +849,15 @@ func (hd *HeaderDownload) FlushBuffer() error {
 // that it has been added as a tip, checks whether the anchor parent hash
 // associated with this tip equals to pre-set value (0x00..00 for genesis)
 func (hd *HeaderDownload) CheckInitiation(segment *ChainSegment, initialHash common.Hash) bool {
-	// Find attachment tip again
-	tip, exists := hd.getTip(segment.Headers[0].Hash(), false)
+	tipHash := segment.Headers[0].Hash()
+	tip, exists := hd.getTip(tipHash)
 	if !exists {
 		return false
 	}
 	if tip.anchor.hash != initialHash {
 		return false
 	}
+	fmt.Printf("Tip %d %x has total difficulty %d, highest %d\n", tip.blockHeight, tipHash, tip.cumulativeDifficulty.ToBig(), hd.highestTotalDifficulty.ToBig())
 	if tip.cumulativeDifficulty.Gt(&hd.highestTotalDifficulty) {
 		hd.highestTotalDifficulty.Set(&tip.cumulativeDifficulty)
 		return true
@@ -629,44 +879,19 @@ func (hd *HeaderDownload) childTipValid(child *types.Header, tipHash common.Hash
 }
 
 func (hd *HeaderDownload) HasTip(tipHash common.Hash) bool {
-	if _, ok := hd.getTip(tipHash, false); ok {
+	if _, ok := hd.getTip(tipHash); ok {
 		return true
 	}
 	return false
 }
 
-func (hd *HeaderDownload) getTip(tipHash common.Hash, touch bool) (*Tip, bool) {
-	if hardTip, ok := hd.hardTips[tipHash]; ok {
-		return hardTip, true
-	}
-	if touch {
-		if tipRaw, ok := hd.tips.Get(tipHash); ok {
-			tip := tipRaw.(*Tip)
-			return tip, true
-		}
-	} else {
-		if tipRaw, ok := hd.tips.Peek(tipHash); ok {
-			tip := tipRaw.(*Tip)
-			return tip, true
-		}
+func (hd *HeaderDownload) getTip(tipHash common.Hash) (*Tip, bool) {
+	if tip, ok := hd.tips[tipHash]; ok {
+		return tip, true
 	}
 	return nil, false
 }
 
-func (hd *HeaderDownload) tipEvicted(key interface{}, value interface{}) {
-	tipHash := key.(common.Hash)
-	tip := value.(*Tip)
-	anchor := tip.anchor
-	var newTips []common.Hash
-	for _, anchorTipHash := range anchor.tips {
-		if anchorTipHash != tipHash {
-			newTips = append(newTips, anchorTipHash)
-		}
-	}
-	//fmt.Printf("Tip %d [%x] evicted\n", tip.blockHeight, tipHash)
-	anchor.tips = newTips
-}
-
 // addHeaderAsTip adds given header as a tip belonging to a given anchorParent
 func (hd *HeaderDownload) addHeaderAsTip(header *types.Header, anchor *Anchor, cumulativeDifficulty uint256.Int, currentTime uint64) error {
 	diff, overflow := uint256.FromBig(header.Difficulty)
@@ -681,26 +906,17 @@ func (hd *HeaderDownload) addHeaderAsTip(header *types.Header, anchor *Anchor, c
 		difficulty:           *diff,
 		blockHeight:          header.Number.Uint64(),
 		uncleHash:            header.UncleHash,
-		noPrepend:            false,
-	}
-	// Move expired items from protected map to the LRU cache
-	for hd.tipQueue.Len() > 0 {
-		if peek := (*hd.tipQueue)[0]; peek.tip.timestamp+hd.newAnchorPastLimit < currentTime {
-			p := heap.Pop(hd.tipQueue).(TipQueueItem)
-			delete(hd.hardTips, p.tipHash)
-			hd.tips.Add(p.tipHash, p.tip)
-			//fmt.Printf("Moved tip %d [%x] from hard to soft %d+%d < %d\n", p.tip.blockHeight, tipHash, p.tip.timestamp, hd.newAnchorPastLimit, currentTime)
-		} else {
-			break
-		}
 	}
-	if tip.timestamp+hd.newAnchorPastLimit >= currentTime {
-		heap.Push(hd.tipQueue, TipQueueItem{tip: tip, tipHash: tipHash})
-		hd.hardTips[tipHash] = tip
-	} else {
-		hd.tips.Add(tipHash, tip)
+	_, hard := hd.hardTips[tipHash]
+	hd.anchorTree.Delete(anchor)
+	hd.tips[tipHash] = tip
+	heap.Push(anchor.tipQueue, AnchorTipItem{hash: tipHash, height: tip.blockHeight, hard: hard})
+	hd.tipCount++
+	if tip.blockHeight > anchor.maxTipHeight {
+		anchor.maxTipHeight = tip.blockHeight
 	}
-	anchor.tips = append(anchor.tips, tipHash)
+	hd.anchorTree.ReplaceOrInsert(anchor)
+	hd.limitTips()
 	return nil
 }
 
@@ -711,29 +927,45 @@ func (hd *HeaderDownload) addHardCodedTip(blockHeight uint64, timestamp uint64,
 		cumulativeDifficulty: cumulativeDifficulty,
 		timestamp:            timestamp,
 		blockHeight:          blockHeight,
-		noPrepend:            true,
 	}
-	hd.hardTips[hash] = tip
-	anchor.tips = append(anchor.tips, hash)
+	hd.tips[hash] = tip
 }
 
-func (hd *HeaderDownload) addHeaderAsAnchor(header *types.Header, powDepth int, totalDifficulty uint256.Int) (*Anchor, error) {
+func (hd *HeaderDownload) addHeaderAsAnchor(header *types.Header, powDepth int) (*Anchor, error) {
 	diff, overflow := uint256.FromBig(header.Difficulty)
 	if overflow {
 		return nil, fmt.Errorf("overflow when converting header.Difficulty to uint256: %s", header.Difficulty)
 	}
 	anchor := &Anchor{
-		powDepth:        powDepth,
-		totalDifficulty: totalDifficulty,
-		difficulty:      *diff,
-		timestamp:       header.Time,
-		hash:            header.Hash(),
-		blockHeight:     header.Number.Uint64(),
-	}
+		powDepth:    powDepth,
+		difficulty:  *diff,
+		timestamp:   header.Time,
+		hash:        header.Hash(),
+		blockHeight: header.Number.Uint64(),
+		tipQueue:    &AnchorTipQueue{},
+		anchorID:    hd.nextAnchorID,
+	}
+	hd.nextAnchorID++
+	heap.Init(anchor.tipQueue)
 	hd.anchors[header.ParentHash] = append(hd.anchors[header.ParentHash], anchor)
 	return anchor, nil
 }
 
+// reserveTip makes sure there is a space for at least one more tip
+func (hd *HeaderDownload) limitTips() {
+	for hd.tipCount > hd.tipLimit {
+		//fmt.Printf("limitTips tips %d >= %d\n", hd.tipCount, hd.tipLimit)
+		// Pick the anchor with the largest (maxTipHeight - minTipHeight) difference
+		anchor := hd.anchorTree.DeleteMin().(*Anchor)
+		//fmt.Printf("Chose anchor %d with maxTipHeight %d, tipStetch: %d\n", anchor.blockHeight, anchor.maxTipHeight, anchor.tipStretch())
+		//hd.anchorTree.Delete(&AnchorItem{ID: anchor.anchorID, tipStretch: anchor.tipStretch()})
+		tipItem := heap.Pop(anchor.tipQueue).(AnchorTipItem)
+		hd.anchorTree.ReplaceOrInsert(anchor)
+		delete(hd.tips, tipItem.hash)
+		hd.tipCount--
+	}
+}
+
 // anchorParentValid checks whether child-parent relationship between an anchor and
 // its extension (parent) is correct
 // (excluding Proof Of Work validity)
diff --git a/turbo/stages/headerdownload/header_data_struct.go b/turbo/stages/headerdownload/header_data_struct.go
index 1acce71756b556c642f4768877b2ce972606c9db..e7b91d3dc75d839ff273320099cc5eb26cf7ce55 100644
--- a/turbo/stages/headerdownload/header_data_struct.go
+++ b/turbo/stages/headerdownload/header_data_struct.go
@@ -1,26 +1,82 @@
 package headerdownload
 
 import (
-	"container/heap"
+	"container/list"
 	"encoding/binary"
 	"fmt"
 	"math/big"
 	"time"
 
-	lru "github.com/hashicorp/golang-lru"
 	"github.com/holiman/uint256"
 	"github.com/ledgerwatch/turbo-geth/common"
 	"github.com/ledgerwatch/turbo-geth/core/types"
+	"github.com/petar/GoLLRB/llrb"
 )
 
+// AnchorTipItem is element of the priority queue of tips belonging to an anchor
+// This queue is prioritised by block heights, lowest block height being first out
+type AnchorTipItem struct {
+	hash   common.Hash
+	height uint64
+	hard   bool // Whether the tip is hard coded
+}
+
+type AnchorTipQueue []AnchorTipItem
+
+func (atq AnchorTipQueue) Len() int {
+	return len(atq)
+}
+
+func (atq AnchorTipQueue) Less(i, j int) bool {
+	if atq[i].hard == atq[j].hard {
+		return atq[i].height < atq[j].height
+	}
+	return !atq[i].hard
+}
+
+func (atq AnchorTipQueue) Swap(i, j int) {
+	atq[i], atq[j] = atq[j], atq[i]
+}
+
+func (atq *AnchorTipQueue) Push(x interface{}) {
+	// Push and Pop use pointer receivers because they modify the slice's length,
+	// not just its contents.
+	*atq = append(*atq, x.(AnchorTipItem))
+}
+
+func (atq *AnchorTipQueue) Pop() interface{} {
+	old := *atq
+	n := len(old)
+	x := old[n-1]
+	*atq = old[0 : n-1]
+	return x
+}
+
 type Anchor struct {
-	powDepth        int
-	totalDifficulty uint256.Int
-	tips            []common.Hash
-	difficulty      uint256.Int
-	hash            common.Hash
-	blockHeight     uint64
-	timestamp       uint64
+	powDepth     int
+	tipQueue     *AnchorTipQueue
+	difficulty   uint256.Int
+	hash         common.Hash
+	blockHeight  uint64
+	timestamp    uint64
+	maxTipHeight uint64 // Maximum value of `blockHeight` of all tips associated with this anchor
+	anchorID     int    // Unique ID of this anchor to be able to find it in the balanced tree
+}
+
+// For placing anchors into the sorting tree
+func (a *Anchor) Less(bi llrb.Item) bool {
+	b := bi.(*Anchor)
+	if a.tipStretch() == b.tipStretch() {
+		return a.anchorID < b.anchorID
+	}
+	return a.tipStretch() > b.tipStretch()
+}
+
+func (a *Anchor) tipStretch() uint64 {
+	if a.tipQueue.Len() == 0 {
+		return 0
+	}
+	return a.maxTipHeight - (*a.tipQueue)[0].height
 }
 
 type Tip struct {
@@ -30,7 +86,6 @@ type Tip struct {
 	difficulty           uint256.Int
 	blockHeight          uint64
 	uncleHash            common.Hash
-	noPrepend            bool
 }
 
 // First item in ChainSegment is the anchor
@@ -72,20 +127,23 @@ type VerifySealFunc func(header *types.Header) error
 type CalcDifficultyFunc func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int
 
 type HeaderDownload struct {
-	buffer   []byte
-	filesDir string
-	//currentFile            *os.File
-	//currentFileWriter      io.Writer
+	buffer                 []byte
+	bufferLimit            int
+	filesDir               string
+	anchorSequence         uint32 // Sequence number to be used for recording anchors next time the buffer is flushed
 	badHeaders             map[common.Hash]struct{}
 	anchors                map[common.Hash][]*Anchor // Mapping from parentHash to collection of anchors
-	tips                   *lru.Cache
-	hardTips               map[common.Hash]*Tip // Hard-coded tips
-	tipQueue               *TipQueue            // Tips that are within the newAnchorPastLimit from current time
-	initPowDepth           int                  // powDepth assigned to the newly inserted anchor
-	newAnchorFutureLimit   uint64               // How far in the future (relative to current time) the new anchors are allowed to be
-	newAnchorPastLimit     uint64               // How far in the past (relative to current time) the new anchors are allowed to be
+	anchorTree             *llrb.LLRB                // Balanced tree of anchors sorted by tip stretch (longest stretch first)
+	nextAnchorID           int
+	hardTips               map[common.Hash]struct{} // Set of hashes for hard-coded tips
+	tips                   map[common.Hash]*Tip     // Tips by tip hash
+	tipCount               int                      // Total number of tips associated to all anchors
+	tipLimit               int                      // Maximum allowed number of tips
+	initPowDepth           int                      // powDepth assigned to the newly inserted anchor
+	newAnchorFutureLimit   uint64                   // How far in the future (relative to current time) the new anchors are allowed to be
+	newAnchorPastLimit     uint64                   // How far in the past (relative to current time) the new anchors are allowed to be
 	highestTotalDifficulty uint256.Int
-	requestQueue           *RequestQueue
+	requestQueue           *list.List
 	calcDifficultyFunc     CalcDifficultyFunc
 	verifySealFunc         VerifySealFunc
 	RequestQueueTimer      *time.Timer
@@ -96,34 +154,6 @@ type TipQueueItem struct {
 	tipHash common.Hash
 }
 
-type TipQueue []TipQueueItem
-
-func (tq TipQueue) Len() int {
-	return len(tq)
-}
-
-func (tq TipQueue) Less(i, j int) bool {
-	return tq[i].tip.timestamp < tq[j].tip.timestamp
-}
-
-func (tq TipQueue) Swap(i, j int) {
-	tq[i], tq[j] = tq[j], tq[i]
-}
-
-func (tq *TipQueue) Push(x interface{}) {
-	// Push and Pop use pointer receivers because they modify the slice's length,
-	// not just its contents.
-	*tq = append(*tq, x.(TipQueueItem))
-}
-
-func (tq *TipQueue) Pop() interface{} {
-	old := *tq
-	n := len(old)
-	x := old[n-1]
-	*tq = old[0 : n-1]
-	return x
-}
-
 type RequestQueueItem struct {
 	anchorParent common.Hash
 	waitUntil    uint64
@@ -158,27 +188,27 @@ func (rq *RequestQueue) Pop() interface{} {
 }
 
 func NewHeaderDownload(filesDir string,
-	tipLimit, initPowDepth int,
+	bufferLimit, tipLimit, initPowDepth int,
 	calcDifficultyFunc CalcDifficultyFunc,
 	verifySealFunc VerifySealFunc,
 	newAnchorFutureLimit, newAnchorPastLimit uint64,
 ) *HeaderDownload {
 	hd := &HeaderDownload{
 		filesDir:             filesDir,
+		bufferLimit:          bufferLimit,
 		badHeaders:           make(map[common.Hash]struct{}),
 		anchors:              make(map[common.Hash][]*Anchor),
+		tipLimit:             tipLimit,
 		initPowDepth:         initPowDepth,
-		requestQueue:         &RequestQueue{},
-		tipQueue:             &TipQueue{},
+		requestQueue:         list.New(),
+		anchorTree:           llrb.New(),
 		calcDifficultyFunc:   calcDifficultyFunc,
 		verifySealFunc:       verifySealFunc,
 		newAnchorFutureLimit: newAnchorFutureLimit,
 		newAnchorPastLimit:   newAnchorPastLimit,
+		hardTips:             make(map[common.Hash]struct{}),
+		tips:                 make(map[common.Hash]*Tip),
 	}
-	hd.tips, _ = lru.NewWithEvict(tipLimit, hd.tipEvicted)
-	hd.hardTips = make(map[common.Hash]*Tip)
-	heap.Init(hd.requestQueue)
-	heap.Init(hd.tipQueue)
 	hd.RequestQueueTimer = time.NewTimer(time.Hour)
 	return hd
 }
diff --git a/turbo/stages/headerdownload/header_test.go b/turbo/stages/headerdownload/header_test.go
index 9f9336efabdb266cfc26b0802f965000711081d2..270e94165d852e4cc36dad85f64e1670cb7fda10 100644
--- a/turbo/stages/headerdownload/header_test.go
+++ b/turbo/stages/headerdownload/header_test.go
@@ -11,14 +11,18 @@ import (
 	"github.com/ledgerwatch/turbo-geth/core/types"
 )
 
-func TestHandleHeadersMsg(t *testing.T) {
-	hd := NewHeaderDownload("", 10, 16, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
+const TestBufferLimit = 32 * 1024
+const TestTipLimit = 10
+const TestInitPowDepth = 16
+
+func TestSplitIntoSegments(t *testing.T) {
+	hd := NewHeaderDownload("", TestBufferLimit, TestTipLimit, TestInitPowDepth, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
 		// To get child difficulty, we just add 1000 to the parent difficulty
 		return big.NewInt(0).Add(parentDifficulty, big.NewInt(1000))
 	}, nil, 60, 60)
 
 	// Empty message
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -32,7 +36,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	// Single header
 	var h types.Header
 	h.Number = big.NewInt(5)
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -44,7 +48,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	}
 
 	// Same header repeated twice
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h, &h}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h, &h}); err == nil {
 		if penalty != DuplicateHeaderPenalty {
 			t.Errorf("expected DuplicateHeader penalty, got %s", penalty)
 		}
@@ -57,7 +61,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 
 	// Single header with a bad hash
 	hd.badHeaders[h.Hash()] = struct{}{}
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h}); err == nil {
 		if penalty != BadBlockPenalty {
 			t.Errorf("expected BadBlock penalty, got %s", penalty)
 		}
@@ -75,7 +79,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	h2.Number = big.NewInt(2)
 	h2.Difficulty = big.NewInt(1010)
 	h2.ParentHash = h1.Hash()
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h1, &h2}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h1, &h2}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -94,7 +98,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 
 	// Two connected headers with wrong numbers
 	h2.Number = big.NewInt(3) // Child number 3, parent number 1
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h1, &h2}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h1, &h2}); err == nil {
 		if penalty != WrongChildBlockHeightPenalty {
 			t.Errorf("expected WrongChildBlockHeight penalty, got %s", penalty)
 		}
@@ -108,7 +112,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	// Two connected headers with wrong difficulty
 	h2.Number = big.NewInt(2)        // Child number 2, parent number 1
 	h2.Difficulty = big.NewInt(2000) // Expected difficulty 10 + 1000 = 1010
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h1, &h2}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h1, &h2}); err == nil {
 		if penalty != WrongChildDifficultyPenalty {
 			t.Errorf("expected WrongChildDifficulty penalty, got %s", penalty)
 		}
@@ -126,7 +130,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	h3.Difficulty = big.NewInt(1010)
 	h3.ParentHash = h1.Hash()
 	h3.Extra = []byte("I'm different") // To make sure the hash of h3 is different from the hash of h2
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h1, &h2, &h3}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h1, &h2, &h3}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -144,7 +148,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	}
 
 	// Same three headers, but in a reverse order
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h3, &h2, &h1}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h3, &h2, &h1}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -162,7 +166,7 @@ func TestHandleHeadersMsg(t *testing.T) {
 	}
 
 	// Two headers not connected to each other
-	if chainSegments, penalty, err := hd.HandleHeadersMsg([]*types.Header{&h3, &h2}); err == nil {
+	if chainSegments, penalty, err := hd.SplitIntoSegments([]*types.Header{&h3, &h2}); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -174,14 +178,14 @@ func TestHandleHeadersMsg(t *testing.T) {
 	}
 }
 
-func TestHandleNewBlockMsg(t *testing.T) {
-	hd := NewHeaderDownload("", 10, 16, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
+func TestSingleHeaderAsSegment(t *testing.T) {
+	hd := NewHeaderDownload("", TestBufferLimit, TestTipLimit, TestInitPowDepth, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
 		// To get child difficulty, we just add 1000 to the parent difficulty
 		return big.NewInt(0).Add(parentDifficulty, big.NewInt(1000))
 	}, nil, 60, 60)
 	var h types.Header
 	h.Number = big.NewInt(5)
-	if chainSegments, penalty, err := hd.HandleNewBlockMsg(&h); err == nil {
+	if chainSegments, penalty, err := hd.SingleHeaderAsSegment(&h); err == nil {
 		if penalty != NoPenalty {
 			t.Errorf("unexpected penalty: %s", penalty)
 		}
@@ -200,7 +204,7 @@ func TestHandleNewBlockMsg(t *testing.T) {
 
 	// Same header with a bad hash
 	hd.badHeaders[h.Hash()] = struct{}{}
-	if chainSegments, penalty, err := hd.HandleNewBlockMsg(&h); err == nil {
+	if chainSegments, penalty, err := hd.SingleHeaderAsSegment(&h); err == nil {
 		if penalty != BadBlockPenalty {
 			t.Errorf("expected BadBlock penalty, got %s", penalty)
 		}
@@ -284,7 +288,7 @@ func TestFindTip(t *testing.T) {
 }
 
 func TestExtendUp(t *testing.T) {
-	hd := NewHeaderDownload("", 10, 16, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
+	hd := NewHeaderDownload("", TestBufferLimit, TestTipLimit, TestInitPowDepth, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
 		// To get child difficulty, we just add 1000 to the parent difficulty
 		return big.NewInt(0).Add(parentDifficulty, big.NewInt(1000))
 	}, func(header *types.Header) error {
@@ -306,7 +310,7 @@ func TestExtendUp(t *testing.T) {
 	h2.Number = big.NewInt(2)
 	h2.Difficulty = big.NewInt(1010)
 	h2.ParentHash = h1.Hash()
-	if anchor, err := hd.addHeaderAsAnchor(&h1, 256, uint256.Int{}); err == nil {
+	if anchor, err := hd.addHeaderAsAnchor(&h1, 256); err == nil {
 		if err1 := hd.addHeaderAsTip(&h1, anchor, *new(uint256.Int).SetUint64(2000), currentTime); err1 != nil {
 			t.Fatalf("setting up h1 (tip): %v", err1)
 		}
@@ -314,8 +318,8 @@ func TestExtendUp(t *testing.T) {
 		t.Errorf("setting up h1 (anchor): %v", err)
 	}
 	if err := hd.ExtendUp(&ChainSegment{Headers: []*types.Header{&h2}}, 0, 1, currentTime); err == nil {
-		if hd.tips.Len() != 2 {
-			t.Errorf("expected 2 tips, got %d", hd.tips.Len())
+		if len(hd.tips) != 2 {
+			t.Errorf("expected 2 tips, got %d", len(hd.tips))
 		}
 	} else {
 		t.Errorf("extendUp: %v", err)
@@ -330,10 +334,10 @@ func TestExtendUp(t *testing.T) {
 	h4.Difficulty = big.NewInt(3010)
 	h4.ParentHash = h3.Hash()
 	if err := hd.ExtendUp(&ChainSegment{Headers: []*types.Header{&h4, &h3}}, 0, 2, currentTime); err == nil {
-		if hd.tips.Len() != 4 {
-			t.Errorf("expected 4 tips, got %d", hd.tips.Len())
+		if len(hd.tips) != 4 {
+			t.Errorf("expected 4 tips, got %d", len(hd.tips))
 		}
-		tip, ok := hd.getTip(h4.Hash(), false)
+		tip, ok := hd.getTip(h4.Hash())
 		if !ok {
 			t.Errorf("did not find h4 in the tips")
 		}
@@ -351,10 +355,10 @@ func TestExtendUp(t *testing.T) {
 	h41.Extra = []byte("Extra")
 	h41.ParentHash = h3.Hash()
 	if err := hd.ExtendUp(&ChainSegment{Headers: []*types.Header{&h41}}, 0, 1, currentTime); err == nil {
-		if hd.tips.Len() != 5 {
-			t.Errorf("expected 5 tips, got %d", hd.tips.Len())
+		if len(hd.tips) != 5 {
+			t.Errorf("expected 5 tips, got %d", len(hd.tips))
 		}
-		tip, ok := hd.getTip(h41.Hash(), false)
+		tip, ok := hd.getTip(h41.Hash())
 		if !ok {
 			t.Errorf("did not find h41 in the tips")
 		}
@@ -382,7 +386,7 @@ func TestExtendUp(t *testing.T) {
 	}
 
 	// Introduce h5 as a tip and prepend h6
-	if anchor, err := hd.addHeaderAsAnchor(&h5, 256, uint256.Int{}); err == nil {
+	if anchor, err := hd.addHeaderAsAnchor(&h5, 256); err == nil {
 		if err1 := hd.addHeaderAsTip(&h5, anchor, *new(uint256.Int).SetUint64(10000), currentTime); err1 != nil {
 			t.Fatalf("setting up h5 (tip): %v", err1)
 		}
@@ -390,10 +394,10 @@ func TestExtendUp(t *testing.T) {
 		t.Errorf("setting up h5 (anchor): %v", err)
 	}
 	if err := hd.ExtendUp(&ChainSegment{Headers: []*types.Header{&h6}}, 0, 1, currentTime); err == nil {
-		if hd.tips.Len() != 7 {
-			t.Errorf("expected 7 tips, got %d", hd.tips.Len())
+		if len(hd.tips) != 7 {
+			t.Errorf("expected 7 tips, got %d", len(hd.tips))
 		}
-		tip, ok := hd.getTip(h6.Hash(), false)
+		tip, ok := hd.getTip(h6.Hash())
 		if !ok {
 			t.Errorf("did not find h6 in the tips")
 		}
@@ -412,24 +416,15 @@ func TestExtendUp(t *testing.T) {
 	h7.Difficulty = big.NewInt(6010)
 	h7.ParentHash = common.HexToHash("0x4354543543959438594359348990345893408")
 	// Introduce hard-coded tip
-	if anchor, err := hd.addHeaderAsAnchor(&h7, 256, uint256.Int{}); err == nil {
+	if anchor, err := hd.addHeaderAsAnchor(&h7, 256); err == nil {
 		hd.addHardCodedTip(10, 5555, h7.Hash(), anchor, *new(uint256.Int).SetUint64(2000))
 	} else {
 		t.Fatalf("settings up h7 (anchor): %v", err)
 	}
-
-	// Try to prepend to the hard-coded tip
-	var h8 types.Header
-	h8.Number = big.NewInt(8)
-	h8.Difficulty = big.NewInt(7010)
-	h8.ParentHash = h7.Hash()
-	if err := hd.ExtendUp(&ChainSegment{Headers: []*types.Header{&h8}}, 0, 1, currentTime); err == nil {
-		t.Errorf("extendUp to hard-coded tip - expected error")
-	}
 }
 
 func TestExtendDown(t *testing.T) {
-	hd := NewHeaderDownload("", 10, 16, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
+	hd := NewHeaderDownload("", TestBufferLimit, TestTipLimit, TestInitPowDepth, func(childTimestamp uint64, parentTime uint64, parentDifficulty, parentNumber *big.Int, parentHash, parentUncleHash common.Hash) *big.Int {
 		// To get child difficulty, we just add 1000 to the parent difficulty
 		return big.NewInt(0).Add(parentDifficulty, big.NewInt(1000))
 	}, func(header *types.Header) error {