diff --git a/core/blockchain.go b/core/blockchain.go
index c52be68354295c66ffe6c9aa4e88038530a21b6b..bc1db49f376102b3b5d0bb87064ad425b8bfa1b3 100644
--- a/core/blockchain.go
+++ b/core/blockchain.go
@@ -2438,12 +2438,8 @@ func (bc *BlockChain) InsertHeaderChain(chain []*types.Header, checkFreq int) (i
 
 	bc.wg.Add(1)
 	defer bc.wg.Done()
-
-	whFunc := func(header *types.Header) error {
-		_, err := bc.hc.WriteHeader(header)
-		return err
-	}
-	return bc.hc.InsertHeaderChain(chain, whFunc, start)
+	_, err := bc.hc.InsertHeaderChain(chain, start)
+	return 0, err
 }
 
 // CurrentHeader retrieves the current head header of the canonical chain. The
diff --git a/core/headerchain.go b/core/headerchain.go
index 7c1aff99d8e75965a7a6a2ec1642ff3ca1b3dd1b..dc354549c083c4fd1f5b7cd168403f35eef5209f 100644
--- a/core/headerchain.go
+++ b/core/headerchain.go
@@ -129,118 +129,192 @@ func (hc *HeaderChain) GetBlockNumber(hash common.Hash) *uint64 {
 	return number
 }
 
-// WriteHeader writes a header into the local chain, given that its parent is
-// already known. If the total difficulty of the newly inserted header becomes
-// greater than the current known TD, the canonical chain is re-routed.
+type headerWriteResult struct {
+	status     WriteStatus
+	ignored    int
+	imported   int
+	lastHash   common.Hash
+	lastHeader *types.Header
+}
+
+// WriteHeaders writes a chain of headers into the local chain, given that the parents
+// are already known. If the total difficulty of the newly inserted chain becomes
+// greater than the current known TD, the canonical chain is reorged.
 //
 // Note: This method is not concurrent-safe with inserting blocks simultaneously
 // into the chain, as side effects caused by reorganisations cannot be emulated
 // without the real blocks. Hence, writing headers directly should only be done
 // in two scenarios: pure-header mode of operation (light clients), or properly
 // separated header/block phases (non-archive clients).
-func (hc *HeaderChain) WriteHeader(header *types.Header) (status WriteStatus, err error) {
-	// Cache some values to prevent constant recalculation
+func (hc *HeaderChain) writeHeaders(headers []*types.Header) (result *headerWriteResult, err error) {
+	if len(headers) == 0 {
+		return &headerWriteResult{}, nil
+	}
+	ptd := hc.GetTd(headers[0].ParentHash, headers[0].Number.Uint64()-1)
+	if ptd == nil {
+		return &headerWriteResult{}, consensus.ErrUnknownAncestor
+	}
 	var (
-		hash   = header.Hash()
-		number = header.Number.Uint64()
+		lastNumber = headers[0].Number.Uint64() - 1 // Last successfully imported number
+		lastHash   = headers[0].ParentHash          // Last imported header hash
+		newTD      = new(big.Int).Set(ptd)          // Total difficulty of inserted chain
+
+		lastHeader    *types.Header
+		inserted      []numberHash // Ephemeral lookup of number/hash for the chain
+		firstInserted = -1         // Index of the first non-ignored header
 	)
-	// Calculate the total difficulty of the header
-	ptd := hc.GetTd(header.ParentHash, number-1)
-	if ptd == nil {
-		return NonStatTy, consensus.ErrUnknownAncestor
-	}
-	head := hc.CurrentHeader().Number.Uint64()
-	localTd := hc.GetTd(hc.currentHeaderHash, head)
-	externTd := new(big.Int).Add(header.Difficulty, ptd)
-
-	// Irrelevant of the canonical status, write the td and header to the database
-	//
-	// Note all the components of header(td, hash->number index and header) should
-	// be written atomically.
-	headerBatch := hc.chainDb.NewBatch()
-	rawdb.WriteTd(headerBatch, hash, number, externTd)
-	rawdb.WriteHeader(headerBatch, header)
-	if err := headerBatch.Write(); err != nil {
-		log.Crit("Failed to write header into disk", "err", err)
+
+	batch := hc.chainDb.NewBatch()
+	for i, header := range headers {
+		var hash common.Hash
+		// The headers have already been validated at this point, so we already
+		// know that it's a contiguous chain, where
+		// headers[i].Hash() == headers[i+1].ParentHash
+		if i < len(headers)-1 {
+			hash = headers[i+1].ParentHash
+		} else {
+			hash = header.Hash()
+		}
+		number := header.Number.Uint64()
+		newTD.Add(newTD, header.Difficulty)
+
+		// If the header is already known, skip it, otherwise store
+		if !hc.HasHeader(hash, number) {
+			// Irrelevant of the canonical status, write the TD and header to the database.
+			rawdb.WriteTd(batch, hash, number, newTD)
+			hc.tdCache.Add(hash, new(big.Int).Set(newTD))
+
+			rawdb.WriteHeader(batch, header)
+			inserted = append(inserted, numberHash{number, hash})
+			hc.headerCache.Add(hash, header)
+			hc.numberCache.Add(hash, number)
+			if firstInserted < 0 {
+				firstInserted = i
+			}
+		}
+		lastHeader, lastHash, lastNumber = header, hash, number
+	}
+
+	// Skip the slow disk write of all headers if interrupted.
+	if hc.procInterrupt() {
+		log.Debug("Premature abort during headers import")
+		return &headerWriteResult{}, errors.New("aborted")
 	}
+	// Commit to disk!
+	if err := batch.Write(); err != nil {
+		log.Crit("Failed to write headers", "error", err)
+	}
+	batch.Reset()
+
+	var (
+		head    = hc.CurrentHeader().Number.Uint64()
+		localTD = hc.GetTd(hc.currentHeaderHash, head)
+		status  = SideStatTy
+	)
 	// If the total difficulty is higher than our known, add it to the canonical chain
 	// Second clause in the if statement reduces the vulnerability to selfish mining.
 	// Please refer to http://www.cs.cornell.edu/~ie53/publications/btcProcFC.pdf
-	reorg := externTd.Cmp(localTd) > 0
-	if !reorg && externTd.Cmp(localTd) == 0 {
-		if header.Number.Uint64() < head {
+	reorg := newTD.Cmp(localTD) > 0
+	if !reorg && newTD.Cmp(localTD) == 0 {
+		if lastNumber < head {
 			reorg = true
-		} else if header.Number.Uint64() == head {
+		} else if lastNumber == head {
 			reorg = mrand.Float64() < 0.5
 		}
 	}
+	// If the parent of the (first) block is already the canon header,
+	// we don't have to go backwards to delete canon blocks, but
+	// simply pile them onto the existing chain
+	chainAlreadyCanon := headers[0].ParentHash == hc.currentHeaderHash
 	if reorg {
 		// If the header can be added into canonical chain, adjust the
 		// header chain markers(canonical indexes and head header flag).
 		//
 		// Note all markers should be written atomically.
-
-		// Delete any canonical number assignments above the new head
-		markerBatch := hc.chainDb.NewBatch()
-		for i := number + 1; ; i++ {
-			hash := rawdb.ReadCanonicalHash(hc.chainDb, i)
-			if hash == (common.Hash{}) {
-				break
+		markerBatch := batch // we can reuse the batch to keep allocs down
+		if !chainAlreadyCanon {
+			// Delete any canonical number assignments above the new head
+			for i := lastNumber + 1; ; i++ {
+				hash := rawdb.ReadCanonicalHash(hc.chainDb, i)
+				if hash == (common.Hash{}) {
+					break
+				}
+				rawdb.DeleteCanonicalHash(markerBatch, i)
+			}
+			// Overwrite any stale canonical number assignments, going
+			// backwards from the first header in this import
+			var (
+				headHash   = headers[0].ParentHash          // inserted[0].parent?
+				headNumber = headers[0].Number.Uint64() - 1 // inserted[0].num-1 ?
+				headHeader = hc.GetHeader(headHash, headNumber)
+			)
+			for rawdb.ReadCanonicalHash(hc.chainDb, headNumber) != headHash {
+				rawdb.WriteCanonicalHash(markerBatch, headHash, headNumber)
+				headHash = headHeader.ParentHash
+				headNumber = headHeader.Number.Uint64() - 1
+				headHeader = hc.GetHeader(headHash, headNumber)
+			}
+			// If some of the older headers were already known, but obtained canon-status
+			// during this import batch, then we need to write that now
+			// Further down, we continue writing the staus for the ones that
+			// were not already known
+			for i := 0; i < firstInserted; i++ {
+				hash := headers[i].Hash()
+				num := headers[i].Number.Uint64()
+				rawdb.WriteCanonicalHash(markerBatch, hash, num)
+				rawdb.WriteHeadHeaderHash(markerBatch, hash)
 			}
-			rawdb.DeleteCanonicalHash(markerBatch, i)
 		}
-
-		// Overwrite any stale canonical number assignments
-		var (
-			headHash   = header.ParentHash
-			headNumber = header.Number.Uint64() - 1
-			headHeader = hc.GetHeader(headHash, headNumber)
-		)
-		for rawdb.ReadCanonicalHash(hc.chainDb, headNumber) != headHash {
-			rawdb.WriteCanonicalHash(markerBatch, headHash, headNumber)
-
-			headHash = headHeader.ParentHash
-			headNumber = headHeader.Number.Uint64() - 1
-			headHeader = hc.GetHeader(headHash, headNumber)
+		// Extend the canonical chain with the new headers
+		for _, hn := range inserted {
+			rawdb.WriteCanonicalHash(markerBatch, hn.hash, hn.number)
+			rawdb.WriteHeadHeaderHash(markerBatch, hn.hash)
 		}
-		// Extend the canonical chain with the new header
-		rawdb.WriteCanonicalHash(markerBatch, hash, number)
-		rawdb.WriteHeadHeaderHash(markerBatch, hash)
 		if err := markerBatch.Write(); err != nil {
 			log.Crit("Failed to write header markers into disk", "err", err)
 		}
+		markerBatch.Reset()
 		// Last step update all in-memory head header markers
-		hc.currentHeaderHash = hash
-		hc.currentHeader.Store(types.CopyHeader(header))
-		headHeaderGauge.Update(header.Number.Int64())
+		hc.currentHeaderHash = lastHash
+		hc.currentHeader.Store(types.CopyHeader(lastHeader))
+		headHeaderGauge.Update(lastHeader.Number.Int64())
 
+		// Chain status is canonical since this insert was a reorg.
+		// Note that all inserts which have higher TD than existing are 'reorg'.
 		status = CanonStatTy
-	} else {
-		status = SideStatTy
 	}
-	hc.tdCache.Add(hash, externTd)
-	hc.headerCache.Add(hash, header)
-	hc.numberCache.Add(hash, number)
-	return
-}
 
-// WhCallback is a callback function for inserting individual headers.
-// A callback is used for two reasons: first, in a LightChain, status should be
-// processed and light chain events sent, while in a BlockChain this is not
-// necessary since chain events are sent after inserting blocks. Second, the
-// header writes should be protected by the parent chain mutex individually.
-type WhCallback func(*types.Header) error
+	if len(inserted) == 0 {
+		status = NonStatTy
+	}
+	return &headerWriteResult{
+		status:     status,
+		ignored:    len(headers) - len(inserted),
+		imported:   len(inserted),
+		lastHash:   lastHash,
+		lastHeader: lastHeader,
+	}, nil
+}
 
 func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int) (int, error) {
 	// Do a sanity check that the provided chain is actually ordered and linked
 	for i := 1; i < len(chain); i++ {
-		if chain[i].Number.Uint64() != chain[i-1].Number.Uint64()+1 || chain[i].ParentHash != chain[i-1].Hash() {
+		parentHash := chain[i-1].Hash()
+		if chain[i].Number.Uint64() != chain[i-1].Number.Uint64()+1 || chain[i].ParentHash != parentHash {
 			// Chain broke ancestry, log a message (programming error) and skip insertion
 			log.Error("Non contiguous header insert", "number", chain[i].Number, "hash", chain[i].Hash(),
-				"parent", chain[i].ParentHash, "prevnumber", chain[i-1].Number, "prevhash", chain[i-1].Hash())
+				"parent", chain[i].ParentHash, "prevnumber", chain[i-1].Number, "prevhash", parentHash)
 
 			return 0, fmt.Errorf("non contiguous insert: item %d is #%d [%x…], item %d is #%d [%x…] (parent [%x…])", i-1, chain[i-1].Number,
-				chain[i-1].Hash().Bytes()[:4], i, chain[i].Number, chain[i].Hash().Bytes()[:4], chain[i].ParentHash[:4])
+				parentHash.Bytes()[:4], i, chain[i].Number, chain[i].Hash().Bytes()[:4], chain[i].ParentHash[:4])
+		}
+		// If the header is a banned one, straight out abort
+		if BadHashes[parentHash] {
+			return i - 1, ErrBlacklistedHash
+		}
+		// If it's the last header in the cunk, we need to check it too
+		if i == len(chain)-1 && BadHashes[chain[i].Hash()] {
+			return i, ErrBlacklistedHash
 		}
 	}
 
@@ -263,16 +337,12 @@ func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int)
 	defer close(abort)
 
 	// Iterate over the headers and ensure they all check out
-	for i, header := range chain {
+	for i := range chain {
 		// If the chain is terminating, stop processing blocks
 		if hc.procInterrupt() {
 			log.Debug("Premature abort during headers verification")
 			return 0, errors.New("aborted")
 		}
-		// If the header is a banned one, straight out abort
-		if BadHashes[header.Hash()] {
-			return i, ErrBlacklistedHash
-		}
 		// Otherwise wait for headers checks and ensure they pass
 		if err := <-results; err != nil {
 			return i, err
@@ -282,55 +352,41 @@ func (hc *HeaderChain) ValidateHeaderChain(chain []*types.Header, checkFreq int)
 	return 0, nil
 }
 
-// InsertHeaderChain attempts to insert the given header chain in to the local
-// chain, possibly creating a reorg. If an error is returned, it will return the
-// index number of the failing header as well an error describing what went wrong.
+// InsertHeaderChain inserts the given headers.
 //
-// The verify parameter can be used to fine tune whether nonce verification
-// should be done or not. The reason behind the optional check is because some
-// of the header retrieval mechanisms already need to verfy nonces, as well as
-// because nonces can be verified sparsely, not needing to check each.
-func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, writeHeader WhCallback, start time.Time) (int, error) {
-	// Collect some import statistics to report on
-	stats := struct{ processed, ignored int }{}
-	// All headers passed verification, import them into the database
-	for i, header := range chain {
-		// Short circuit insertion if shutting down
-		if hc.procInterrupt() {
-			log.Debug("Premature abort during headers import")
-			return i, errors.New("aborted")
-		}
-		// If the header's already known, skip it, otherwise store
-		hash := header.Hash()
-		if hc.HasHeader(hash, header.Number.Uint64()) {
-			externTd := hc.GetTd(hash, header.Number.Uint64())
-			localTd := hc.GetTd(hc.currentHeaderHash, hc.CurrentHeader().Number.Uint64())
-			if externTd == nil || externTd.Cmp(localTd) <= 0 {
-				stats.ignored++
-				continue
-			}
-		}
-		if err := writeHeader(header); err != nil {
-			return i, err
-		}
-		stats.processed++
+// The validity of the headers is NOT CHECKED by this method, i.e. they need to be
+// validated by ValidateHeaderChain before calling InsertHeaderChain.
+//
+// This insert is all-or-nothing. If this returns an error, no headers were written,
+// otherwise they were all processed successfully.
+//
+// The returned 'write status' says if the inserted headers are part of the canonical chain
+// or a side chain.
+func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, start time.Time) (WriteStatus, error) {
+	if hc.procInterrupt() {
+		return 0, errors.New("aborted")
 	}
-	// Report some public statistics so the user has a clue what's going on
-	last := chain[len(chain)-1]
+	res, err := hc.writeHeaders(chain)
 
+	// Report some public statistics so the user has a clue what's going on
 	context := []interface{}{
-		"count", stats.processed, "elapsed", common.PrettyDuration(time.Since(start)),
-		"number", last.Number, "hash", last.Hash(),
+		"count", res.imported,
+		"elapsed", common.PrettyDuration(time.Since(start)),
+	}
+	if err != nil {
+		context = append(context, "err", err)
 	}
-	if timestamp := time.Unix(int64(last.Time), 0); time.Since(timestamp) > time.Minute {
-		context = append(context, []interface{}{"age", common.PrettyAge(timestamp)}...)
+	if last := res.lastHeader; last != nil {
+		context = append(context, "number", last.Number, "hash", res.lastHash)
+		if timestamp := time.Unix(int64(last.Time), 0); time.Since(timestamp) > time.Minute {
+			context = append(context, []interface{}{"age", common.PrettyAge(timestamp)}...)
+		}
 	}
-	if stats.ignored > 0 {
-		context = append(context, []interface{}{"ignored", stats.ignored}...)
+	if res.ignored > 0 {
+		context = append(context, []interface{}{"ignored", res.ignored}...)
 	}
 	log.Info("Imported new block headers", context...)
-
-	return 0, nil
+	return res.status, err
 }
 
 // GetBlockHashesFromHash retrieves a number of block hashes starting at a given
diff --git a/core/headerchain_test.go b/core/headerchain_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..0aa25efd1f232b08dd99e19c0870598a8aadf218
--- /dev/null
+++ b/core/headerchain_test.go
@@ -0,0 +1,115 @@
+// Copyright 2020 The go-ethereum Authors
+// This file is part of the go-ethereum library.
+//
+// The go-ethereum library is free software: you can redistribute it and/or modify
+// it under the terms of the GNU Lesser General Public License as published by
+// the Free Software Foundation, either version 3 of the License, or
+// (at your option) any later version.
+//
+// The go-ethereum library is distributed in the hope that it will be useful,
+// but WITHOUT ANY WARRANTY; without even the implied warranty of
+// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+// GNU Lesser General Public License for more details.
+//
+// You should have received a copy of the GNU Lesser General Public License
+// along with the go-ethereum library. If not, see <http://www.gnu.org/licenses/>.
+
+package core
+
+import (
+	"errors"
+	"fmt"
+	"testing"
+	"time"
+
+	"github.com/ethereum/go-ethereum/consensus"
+	"github.com/ethereum/go-ethereum/consensus/ethash"
+	"github.com/ethereum/go-ethereum/core/rawdb"
+	"github.com/ethereum/go-ethereum/core/types"
+	"github.com/ethereum/go-ethereum/log"
+	"github.com/ethereum/go-ethereum/params"
+)
+
+func verifyUnbrokenCanonchain(hc *HeaderChain) error {
+	h := hc.CurrentHeader()
+	for {
+		canonHash := rawdb.ReadCanonicalHash(hc.chainDb, h.Number.Uint64())
+		if exp := h.Hash(); canonHash != exp {
+			return fmt.Errorf("Canon hash chain broken, block %d got %x, expected %x",
+				h.Number, canonHash[:8], exp[:8])
+		}
+		// Verify that we have the TD
+		if td := rawdb.ReadTd(hc.chainDb, canonHash, h.Number.Uint64()); td == nil {
+			return fmt.Errorf("Canon TD missing at block %d", h.Number)
+		}
+		if h.Number.Uint64() == 0 {
+			break
+		}
+		h = hc.GetHeader(h.ParentHash, h.Number.Uint64()-1)
+	}
+	return nil
+}
+
+func testInsert(t *testing.T, hc *HeaderChain, chain []*types.Header, wantStatus WriteStatus, wantErr error) {
+	t.Helper()
+
+	status, err := hc.InsertHeaderChain(chain, time.Now())
+	if status != wantStatus {
+		t.Errorf("wrong write status from InsertHeaderChain: got %v, want %v", status, wantStatus)
+	}
+	// Always verify that the header chain is unbroken
+	if err := verifyUnbrokenCanonchain(hc); err != nil {
+		t.Fatal(err)
+	}
+	if !errors.Is(err, wantErr) {
+		t.Fatalf("unexpected error from InsertHeaderChain: %v", err)
+	}
+}
+
+// This test checks status reporting of InsertHeaderChain.
+func TestHeaderInsertion(t *testing.T) {
+	var (
+		db      = rawdb.NewMemoryDatabase()
+		genesis = new(Genesis).MustCommit(db)
+	)
+
+	hc, err := NewHeaderChain(db, params.AllEthashProtocolChanges, ethash.NewFaker(), func() bool { return false })
+	if err != nil {
+		t.Fatal(err)
+	}
+	// chain A: G->A1->A2...A128
+	chainA := makeHeaderChain(genesis.Header(), 128, ethash.NewFaker(), db, 10)
+	// chain B: G->A1->B2...B128
+	chainB := makeHeaderChain(chainA[0], 128, ethash.NewFaker(), db, 10)
+	log.Root().SetHandler(log.StdoutHandler)
+
+	// Inserting 64 headers on an empty chain, expecting
+	// 1 callbacks, 1 canon-status, 0 sidestatus,
+	testInsert(t, hc, chainA[:64], CanonStatTy, nil)
+
+	// Inserting 64 identical headers, expecting
+	// 0 callbacks, 0 canon-status, 0 sidestatus,
+	testInsert(t, hc, chainA[:64], NonStatTy, nil)
+
+	// Inserting the same some old, some new headers
+	// 1 callbacks, 1 canon, 0 side
+	testInsert(t, hc, chainA[32:96], CanonStatTy, nil)
+
+	// Inserting side blocks, but not overtaking the canon chain
+	testInsert(t, hc, chainB[0:32], SideStatTy, nil)
+
+	// Inserting more side blocks, but we don't have the parent
+	testInsert(t, hc, chainB[34:36], NonStatTy, consensus.ErrUnknownAncestor)
+
+	// Inserting more sideblocks, overtaking the canon chain
+	testInsert(t, hc, chainB[32:97], CanonStatTy, nil)
+
+	// Inserting more A-headers, taking back the canonicality
+	testInsert(t, hc, chainA[90:100], CanonStatTy, nil)
+
+	// And B becomes canon again
+	testInsert(t, hc, chainB[97:107], CanonStatTy, nil)
+
+	// And B becomes even longer
+	testInsert(t, hc, chainB[107:128], CanonStatTy, nil)
+}
diff --git a/light/lightchain.go b/light/lightchain.go
index 6fc321ae0b5717d182dc6a79a0a71e5ab06481b5..ca6fbfac49de92d5348114c46a30fbe4cd260d68 100644
--- a/light/lightchain.go
+++ b/light/lightchain.go
@@ -396,24 +396,26 @@ func (lc *LightChain) InsertHeaderChain(chain []*types.Header, checkFreq int) (i
 	lc.wg.Add(1)
 	defer lc.wg.Done()
 
-	var events []interface{}
-	whFunc := func(header *types.Header) error {
-		status, err := lc.hc.WriteHeader(header)
-
-		switch status {
-		case core.CanonStatTy:
-			log.Debug("Inserted new header", "number", header.Number, "hash", header.Hash())
-			events = append(events, core.ChainEvent{Block: types.NewBlockWithHeader(header), Hash: header.Hash()})
-
-		case core.SideStatTy:
-			log.Debug("Inserted forked header", "number", header.Number, "hash", header.Hash())
-			events = append(events, core.ChainSideEvent{Block: types.NewBlockWithHeader(header)})
-		}
-		return err
+	status, err := lc.hc.InsertHeaderChain(chain, start)
+	if err != nil || len(chain) == 0 {
+		return 0, err
+	}
+
+	// Create chain event for the new head block of this insertion.
+	var (
+		events     = make([]interface{}, 0, 1)
+		lastHeader = chain[len(chain)-1]
+		block      = types.NewBlockWithHeader(lastHeader)
+	)
+	switch status {
+	case core.CanonStatTy:
+		events = append(events, core.ChainEvent{Block: block, Hash: block.Hash()})
+	case core.SideStatTy:
+		events = append(events, core.ChainSideEvent{Block: block})
 	}
-	i, err := lc.hc.InsertHeaderChain(chain, whFunc, start)
 	lc.postChainEvents(events)
-	return i, err
+
+	return 0, err
 }
 
 // CurrentHeader retrieves the current head header of the canonical chain. The