diff --git a/common/bidutil/bidutil.go b/common/bidutil/bidutil.go
index d2735808c6..11ac39537e 100644
--- a/common/bidutil/bidutil.go
+++ b/common/bidutil/bidutil.go
@@ -17,7 +17,7 @@ func BidBetterBefore(parentHeader *types.Header, blockPeriod uint64, delayLeftOv
 // BidMustBefore returns the time when the next bid must be received,
 // only considering the consensus delay but not bid simulation duration.
 func BidMustBefore(parentHeader *types.Header, blockPeriod uint64, delayLeftOver time.Duration) time.Time {
-	nextHeaderTime := time.Unix(int64(parentHeader.Time+blockPeriod), 0)
+	nextHeaderTime := time.UnixMilli(int64(parentHeader.TimeInMilliseconds() + blockPeriod))
 	nextHeaderTime = nextHeaderTime.Add(-delayLeftOver)
 	return nextHeaderTime
 }
diff --git a/consensus/beacon/consensus.go b/consensus/beacon/consensus.go
index 994b744df3..bbc1808bdd 100644
--- a/consensus/beacon/consensus.go
+++ b/consensus/beacon/consensus.go
@@ -368,6 +368,10 @@ func (beacon *Beacon) Delay(_ consensus.ChainReader, _ *types.Header, _ *time.Du
 	return nil
 }
 
+func (beacon *Beacon) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) {
+	panic("not supported")
+}
+
 // Finalize implements consensus.Engine, setting the final state on the header
 func (beacon *Beacon) Finalize(chain consensus.ChainHeaderReader, header *types.Header, state vm.StateDB, txs *[]*types.Transaction, uncles []*types.Header, withdrawals []*types.Withdrawal, _ *[]*types.Receipt, _ *[]*types.Transaction, _ *uint64, tracer *tracing.Hooks) error {
 	// Finalize is different with Prepare, it can be used in both block verification.
diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go
index 4591642305..3eebeec0a9 100644
--- a/consensus/clique/clique.go
+++ b/consensus/clique/clique.go
@@ -628,6 +628,10 @@ func (c *Clique) Delay(chain consensus.ChainReader, header *types.Header, leftOv
 	return nil
 }
 
+func (c *Clique) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) {
+	return c.config.Period * 1000, nil
+}
+
 // Seal implements consensus.Engine, attempting to create a sealed block using
 // the local signing credentials.
 func (c *Clique) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error {
diff --git a/consensus/consensus.go b/consensus/consensus.go
index a58287f5e8..b24bbbe9cd 100644
--- a/consensus/consensus.go
+++ b/consensus/consensus.go
@@ -147,6 +147,9 @@ type Engine interface {
 	// Delay returns the max duration the miner can commit txs
 	Delay(chain ChainReader, header *types.Header, leftOver *time.Duration) *time.Duration
 
+	// BlockInterval returns the block interval in milliseconds after given header applied
+	BlockInterval(chain ChainHeaderReader, header *types.Header) (uint64, error)
+
 	// Close terminates any background threads maintained by the consensus engine.
 	Close() error
 }
@@ -162,6 +165,5 @@ type PoSA interface {
 	GetFinalizedHeader(chain ChainHeaderReader, header *types.Header) *types.Header
 	VerifyVote(chain ChainHeaderReader, vote *types.VoteEnvelope) error
 	IsActiveValidatorAt(chain ChainHeaderReader, header *types.Header, checkVoteKeyFn func(bLSPublicKey *types.BLSPublicKey) bool) bool
-	BlockInterval() uint64
 	NextProposalBlock(chain ChainHeaderReader, header *types.Header, proposer common.Address) (uint64, uint64, error)
 }
diff --git a/consensus/ethash/ethash.go b/consensus/ethash/ethash.go
index f37ec26056..fcf3afd2b4 100644
--- a/consensus/ethash/ethash.go
+++ b/consensus/ethash/ethash.go
@@ -83,3 +83,7 @@ func (ethash *Ethash) APIs(chain consensus.ChainHeaderReader) []rpc.API {
 func (ethash *Ethash) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error {
 	panic("ethash (pow) sealing not supported any more")
 }
+
+func (ethash *Ethash) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) {
+	panic("not supported")
+}
diff --git a/consensus/misc/eip1559/eip1559_test.go b/consensus/misc/eip1559/eip1559_test.go
index 54f91046e3..387ec7a65c 100644
--- a/consensus/misc/eip1559/eip1559_test.go
+++ b/consensus/misc/eip1559/eip1559_test.go
@@ -53,8 +53,7 @@ func config() *params.ChainConfig {
 	config := copyConfig(params.TestChainConfig)
 	config.Ethash = nil
 	config.Parlia = &params.ParliaConfig{
-		Period: 3,
-		Epoch:  200,
+		Epoch: 200,
 	}
 	config.LondonBlock = big.NewInt(5)
 	return config
diff --git a/consensus/parlia/parlia.go b/consensus/parlia/parlia.go
index bb4f13e2f6..14871bc18f 100644
--- a/consensus/parlia/parlia.go
+++ b/consensus/parlia/parlia.go
@@ -53,9 +53,11 @@ const (
 	inMemorySignatures = 4096  // Number of recent block signatures to keep in memory
 	inMemoryHeaders    = 86400 // Number of recent headers to keep in memory for double sign detection,
 
-	checkpointInterval = 1024        // Number of blocks after which to save the snapshot to the database
-	defaultEpochLength = uint64(200) // Default number of blocks of checkpoint to update validatorSet from contract
-	defaultTurnLength  = uint8(1)    // Default consecutive number of blocks a validator receives priority for block production
+	checkpointInterval   = 1024         // Number of blocks after which to save the snapshot to the database
+	defaultEpochLength   = uint64(200)  // Default number of blocks of checkpoint to update validatorSet from contract
+	defaultBlockInterval = uint16(3000) // Default block interval in milliseconds
+	lorentzBlockInterval = uint16(500)  // Block interval starting from the Lorentz hard fork
+	defaultTurnLength    = uint8(1)     // Default consecutive number of blocks a validator receives priority for block production
 
 	extraVanity      = 32 // Fixed number of extra-data prefix bytes reserved for signer vanity
 	extraSeal        = 65 // Fixed number of extra-data suffix bytes reserved for signer seal
@@ -66,8 +68,8 @@ const (
 	validatorBytesLength            = common.AddressLength + types.BLSPublicKeyLength
 	validatorNumberSize             = 1 // Fixed number of extra prefix bytes reserved for validator number after Luban
 
-	wiggleTime         = uint64(1) // second, Random delay (per signer) to allow concurrent signers
-	initialBackOffTime = uint64(1) // second
+	wiggleTime                = uint64(1000) // milliseconds, Random delay (per signer) to allow concurrent signers
+	defaultInitialBackOffTime = uint64(1000) // milliseconds, Default backoff time for the second validator permitted to produce blocks
 
 	systemRewardPercent = 4 // it means 1/2^4 = 1/16 percentage of gas fee incoming will be distributed to system
 
@@ -316,10 +318,6 @@ func New(
 	return c
 }
 
-func (p *Parlia) Period() uint64 {
-	return p.config.Period
-}
-
 func (p *Parlia) IsSystemTransaction(tx *types.Transaction, header *types.Header) (bool, error) {
 	// deploy a contract
 	if tx.To() == nil {
@@ -640,8 +638,18 @@ func (p *Parlia) verifyHeader(chain consensus.ChainHeaderReader, header *types.H
 			return fmt.Errorf("invalid parentBeaconRoot, have %#x, expected nil", header.ParentBeaconRoot)
 		}
 	} else {
-		if header.ParentBeaconRoot == nil || *header.ParentBeaconRoot != (common.Hash{}) {
-			return fmt.Errorf("invalid parentBeaconRoot, have %#x, expected zero hash", header.ParentBeaconRoot)
+		if header.ParentBeaconRoot == nil {
+			return fmt.Errorf("nil parentBeaconRoot after bohr hard fork")
+		}
+		lorentz := chain.Config().IsLorentz(header.Number, header.Time)
+		if !lorentz {
+			if *header.ParentBeaconRoot != (common.Hash{}) { // remove it once bsc mainnet passed lorentz hard fork for simplicity
+				return fmt.Errorf("invalid parentBeaconRoot, have %#x, expected zero hash", header.ParentBeaconRoot)
+			}
+		} else {
+			if header.Milliseconds() >= 1000 {
+				return fmt.Errorf("invalid parentBeaconRoot, have %#x, expected the last two bytes to represent milliseconds", header.ParentBeaconRoot)
+			}
 		}
 	}
 
@@ -779,6 +787,10 @@ func (p *Parlia) snapshot(chain consensus.ChainHeaderReader, number uint64, hash
 				// new snapshot
 				snap = newSnapshot(p.config, p.signatures, number, blockHash, validators, voteAddrs, p.ethAPI)
 
+				if p.chainConfig.IsLorentz(checkpoint.Number, checkpoint.Time) {
+					snap.BlockInterval = lorentzBlockInterval
+				}
+
 				// get turnLength from headers and use that for new turnLength
 				turnLength, err := parseTurnLength(checkpoint, p.chainConfig, p.config)
 				if err != nil {
@@ -1094,9 +1106,13 @@ func (p *Parlia) Prepare(chain consensus.ChainHeaderReader, header *types.Header
 	if parent == nil {
 		return consensus.ErrUnknownAncestor
 	}
-	header.Time = p.blockTimeForRamanujanFork(snap, header, parent)
-	if header.Time < uint64(time.Now().Unix()) {
-		header.Time = uint64(time.Now().Unix())
+	blockTime := p.blockTimeForRamanujanFork(snap, header, parent)
+	if now := uint64(time.Now().UnixMilli()); blockTime < now {
+		blockTime = now
+	}
+	header.Time = blockTime / 1000 // get seconds
+	if p.chainConfig.IsLorentz(header.Number, header.Time) {
+		header.SetMilliseconds(blockTime % 1000)
 	}
 
 	header.Extra = header.Extra[:extraVanity-nextForkHashSize]
@@ -1539,9 +1555,9 @@ func (p *Parlia) Delay(chain consensus.ChainReader, header *types.Header, leftOv
 	}
 	delay := p.delayForRamanujanFork(snap, header)
 
-	if *leftOver >= time.Duration(p.config.Period)*time.Second {
+	if *leftOver >= time.Duration(snap.BlockInterval)*time.Millisecond {
 		// ignore invalid leftOver
-		log.Error("Delay invalid argument", "leftOver", leftOver.String(), "Period", p.config.Period)
+		log.Error("Delay invalid argument", "leftOver", leftOver.String(), "Period", snap.BlockInterval)
 	} else if *leftOver >= delay {
 		delay = time.Duration(0)
 		return &delay
@@ -1550,9 +1566,9 @@ func (p *Parlia) Delay(chain consensus.ChainReader, header *types.Header, leftOv
 	}
 
 	// The blocking time should be no more than half of period when snap.TurnLength == 1
-	timeForMining := time.Duration(p.config.Period) * time.Second / 2
+	timeForMining := time.Duration(snap.BlockInterval) * time.Millisecond / 2
 	if !snap.lastBlockInOneTurn(header.Number.Uint64()) {
-		timeForMining = time.Duration(p.config.Period) * time.Second * 2 / 3
+		timeForMining = time.Duration(snap.BlockInterval) * time.Millisecond * 2 / 3
 	}
 	if delay > timeForMining {
 		delay = timeForMining
@@ -1570,11 +1586,6 @@ func (p *Parlia) Seal(chain consensus.ChainHeaderReader, block *types.Block, res
 	if number == 0 {
 		return errUnknownBlock
 	}
-	// For 0-period chains, refuse to seal empty blocks (no reward but would spin sealing)
-	if p.config.Period == 0 && len(block.Transactions()) == 0 {
-		log.Info("Sealing paused, waiting for transactions")
-		return nil
-	}
 	// Don't hold the val fields for the entire sealing procedure
 	p.lock.RLock()
 	val, signFn := p.val, p.signFn
@@ -2089,7 +2100,10 @@ func (p *Parlia) backOffTime(snap *Snapshot, header *types.Header, val common.Ad
 		log.Debug("backOffTime", "blockNumber", header.Number, "in turn validator", val)
 		return 0
 	} else {
-		delay := initialBackOffTime
+		delay := defaultInitialBackOffTime
+		if p.chainConfig.IsLorentz(header.Number, header.Time) && snap.lastBlockInOneTurn(header.Number.Uint64()) {
+			delay = uint64(snap.BlockInterval) / 2
+		}
 		validators := snap.validators()
 		if p.chainConfig.IsPlanck(header.Number) {
 			counts := snap.countRecents()
@@ -2159,8 +2173,15 @@ func (p *Parlia) backOffTime(snap *Snapshot, header *types.Header, val common.Ad
 	}
 }
 
-func (p *Parlia) BlockInterval() uint64 {
-	return p.config.Period
+func (p *Parlia) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) {
+	if header == nil {
+		return uint64(defaultBlockInterval), errUnknownBlock
+	}
+	snap, err := p.snapshot(chain, header.Number.Uint64(), header.Hash(), nil)
+	if err != nil {
+		return uint64(defaultBlockInterval), err
+	}
+	return uint64(snap.BlockInterval), nil
 }
 
 func (p *Parlia) NextProposalBlock(chain consensus.ChainHeaderReader, header *types.Header, proposer common.Address) (uint64, uint64, error) {
diff --git a/consensus/parlia/parlia_test.go b/consensus/parlia/parlia_test.go
index d32cdda892..ceb8fe2931 100644
--- a/consensus/parlia/parlia_test.go
+++ b/consensus/parlia/parlia_test.go
@@ -178,7 +178,7 @@ func producerBlockDelay(candidates map[int]bool, height, numOfValidators int) (i
 			minCandidate = c
 		}
 	}
-	delay := initialBackOffTime + uint64(minDelay)*wiggleTime
+	delay := defaultInitialBackOffTime + uint64(minDelay)*wiggleTime
 	return minCandidate, delay
 }
 
diff --git a/consensus/parlia/ramanujanfork.go b/consensus/parlia/ramanujanfork.go
index ce9089debc..ad23324bb1 100644
--- a/consensus/parlia/ramanujanfork.go
+++ b/consensus/parlia/ramanujanfork.go
@@ -27,7 +27,7 @@ func (p *Parlia) delayForRamanujanFork(snap *Snapshot, header *types.Header) tim
 }
 
 func (p *Parlia) blockTimeForRamanujanFork(snap *Snapshot, header, parent *types.Header) uint64 {
-	blockTime := parent.Time + p.config.Period
+	blockTime := parent.TimeInMilliseconds() + uint64(snap.BlockInterval)
 	if p.chainConfig.IsRamanujan(header.Number) {
 		blockTime = blockTime + p.backOffTime(snap, header, p.val)
 	}
@@ -36,7 +36,7 @@ func (p *Parlia) blockTimeForRamanujanFork(snap *Snapshot, header, parent *types
 
 func (p *Parlia) blockTimeVerifyForRamanujanFork(snap *Snapshot, header, parent *types.Header) error {
 	if p.chainConfig.IsRamanujan(header.Number) {
-		if header.Time < parent.Time+p.config.Period+p.backOffTime(snap, header, header.Coinbase) {
+		if header.TimeInMilliseconds() < parent.TimeInMilliseconds()+uint64(snap.BlockInterval)+p.backOffTime(snap, header, header.Coinbase) {
 			return consensus.ErrFutureBlock
 		}
 	}
diff --git a/consensus/parlia/snapshot.go b/consensus/parlia/snapshot.go
index d474c06041..7d5fd6acc4 100644
--- a/consensus/parlia/snapshot.go
+++ b/consensus/parlia/snapshot.go
@@ -44,6 +44,7 @@ type Snapshot struct {
 
 	Number           uint64                            `json:"number"`                // Block number where the snapshot was created
 	Hash             common.Hash                       `json:"hash"`                  // Block hash where the snapshot was created
+	BlockInterval    uint16                            `json:"block_interval"`        // Block Interval in milliseconds
 	TurnLength       uint8                             `json:"turn_length"`           // Length of `turn`, meaning the consecutive number of blocks a validator receives priority for block production
 	Validators       map[common.Address]*ValidatorInfo `json:"validators"`            // Set of authorized validators at this moment
 	Recents          map[uint64]common.Address         `json:"recents"`               // Set of recent validators for spam protections
@@ -74,6 +75,7 @@ func newSnapshot(
 		sigCache:         sigCache,
 		Number:           number,
 		Hash:             hash,
+		BlockInterval:    defaultBlockInterval,
 		TurnLength:       defaultTurnLength,
 		Recents:          make(map[uint64]common.Address),
 		RecentForkHashes: make(map[uint64]string),
@@ -117,6 +119,9 @@ func loadSnapshot(config *params.ParliaConfig, sigCache *lru.ARCCache, db ethdb.
 	if err := json.Unmarshal(blob, snap); err != nil {
 		return nil, err
 	}
+	if snap.BlockInterval == 0 { // no BlockInterval field in old snapshots
+		snap.BlockInterval = defaultBlockInterval
+	}
 	if snap.TurnLength == 0 { // no TurnLength field in old snapshots
 		snap.TurnLength = defaultTurnLength
 	}
@@ -145,6 +150,7 @@ func (s *Snapshot) copy() *Snapshot {
 		sigCache:         s.sigCache,
 		Number:           s.Number,
 		Hash:             s.Hash,
+		BlockInterval:    s.BlockInterval,
 		TurnLength:       s.TurnLength,
 		Validators:       make(map[common.Address]*ValidatorInfo),
 		Recents:          make(map[uint64]common.Address),
@@ -308,6 +314,10 @@ func (s *Snapshot) apply(headers []*types.Header, chain consensus.ChainHeaderRea
 				}
 			}
 		}
+		// It's better to set it based on IsOnLorentz, but in practice, the effect is the same as using IsLorentz.
+		if chainConfig.IsLorentz(header.Number, header.Time) {
+			snap.BlockInterval = lorentzBlockInterval
+		}
 		snap.Recents[number] = validator
 		snap.RecentForkHashes[number] = hex.EncodeToString(header.Extra[extraVanity-nextForkHashSize : extraVanity])
 		snap.updateAttestation(header, chainConfig, s.config)
diff --git a/core/chain_makers.go b/core/chain_makers.go
index 9e6ff779f1..7b470a5f05 100644
--- a/core/chain_makers.go
+++ b/core/chain_makers.go
@@ -614,7 +614,7 @@ func (cm *chainMaker) makeHeader(parent *types.Block, state *state.StateDB, engi
 			header.ParentBeaconRoot = new(common.Hash)
 		} else {
 			header.WithdrawalsHash = &types.EmptyWithdrawalsHash
-			if cm.config.IsBohr(header.Number, header.Time) {
+			if cm.config.IsBohr(header.Number, header.Time) && !cm.config.IsLorentz(header.Number, header.Time) {
 				header.ParentBeaconRoot = new(common.Hash)
 			}
 			if cm.config.IsPrague(header.Number, header.Time) {
diff --git a/core/genesis.go b/core/genesis.go
index 488e7fd421..880d058a8d 100644
--- a/core/genesis.go
+++ b/core/genesis.go
@@ -497,7 +497,7 @@ func (g *Genesis) toBlockWithRoot(root common.Hash) *types.Block {
 			// EIP-4788: The parentBeaconBlockRoot of the genesis block is always
 			// the zero hash. This is because the genesis block does not have a parent
 			// by definition.
-			if conf.Parlia == nil || conf.IsBohr(num, g.Timestamp) {
+			if conf.Parlia == nil || conf.IsBohr(num, g.Timestamp) && !conf.IsLorentz(num, g.Timestamp) {
 				head.ParentBeaconRoot = new(common.Hash)
 			}
 
diff --git a/core/types/block.go b/core/types/block.go
index a23601eecd..94159607eb 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -30,6 +30,8 @@ import (
 
 	"golang.org/x/crypto/sha3"
 
+	"github.com/holiman/uint256"
+
 	"github.com/ethereum/go-ethereum/common"
 	"github.com/ethereum/go-ethereum/common/hexutil"
 	"github.com/ethereum/go-ethereum/rlp"
@@ -156,6 +158,22 @@ func (h *Header) Hash() common.Hash {
 	return rlpHash(h)
 }
 
+func (h *Header) SetMilliseconds(milliseconds uint64) {
+	tmp := common.Hash(uint256.NewInt(milliseconds % 1000).Bytes32())
+	h.ParentBeaconRoot = &tmp
+}
+
+func (h *Header) Milliseconds() uint64 {
+	if h.ParentBeaconRoot == nil || *h.ParentBeaconRoot == (common.Hash{}) {
+		return 0
+	}
+	return uint256.NewInt(0).SetBytes2(h.ParentBeaconRoot[30:]).Uint64()
+}
+
+func (h *Header) TimeInMilliseconds() uint64 {
+	return h.Time*1000 + h.Milliseconds()
+}
+
 var headerSize = common.StorageSize(reflect.TypeOf(Header{}).Size())
 
 // Size returns the approximate memory used by all internal contents. It is used
diff --git a/core/vote/vote_manager.go b/core/vote/vote_manager.go
index aed7d29b41..5a48745b86 100644
--- a/core/vote/vote_manager.go
+++ b/core/vote/vote_manager.go
@@ -8,7 +8,6 @@ import (
 
 	"github.com/ethereum/go-ethereum/common"
 	"github.com/ethereum/go-ethereum/consensus"
-	"github.com/ethereum/go-ethereum/consensus/parlia"
 	"github.com/ethereum/go-ethereum/core"
 	"github.com/ethereum/go-ethereum/core/types"
 	"github.com/ethereum/go-ethereum/eth/downloader"
@@ -147,13 +146,13 @@ func (voteManager *VoteManager) loop() {
 			}
 
 			curHead := cHead.Header
-			if p, ok := voteManager.engine.(*parlia.Parlia); ok {
-				nextBlockMinedTime := time.Unix(int64((curHead.Time + p.Period())), 0)
-				timeForBroadcast := 50 * time.Millisecond // enough to broadcast a vote
-				if time.Now().Add(timeForBroadcast).After(nextBlockMinedTime) {
-					log.Warn("too late to vote", "Head.Time(Second)", curHead.Time, "Now(Millisecond)", time.Now().UnixMilli())
-					continue
-				}
+			parentHeader := voteManager.chain.GetHeaderByHash(curHead.ParentHash)
+			blockInterval, _ := voteManager.engine.BlockInterval(voteManager.chain, parentHeader)
+			nextBlockMinedTime := time.UnixMilli(int64((curHead.TimeInMilliseconds() + blockInterval)))
+			timeForBroadcast := 50 * time.Millisecond // enough to broadcast a vote
+			if time.Now().Add(timeForBroadcast).After(nextBlockMinedTime) {
+				log.Warn("too late to vote", "Head.Time(Second)", curHead.Time, "Now(Millisecond)", time.Now().UnixMilli())
+				continue
 			}
 
 			// Check if cur validator is within the validatorSet at curHead
diff --git a/internal/ethapi/api.go b/internal/ethapi/api.go
index 852f3a3615..3b9a3bc2c6 100644
--- a/internal/ethapi/api.go
+++ b/internal/ethapi/api.go
@@ -1434,6 +1434,7 @@ func RPCMarshalBlock(block *types.Block, inclTx bool, fullTx bool, config *param
 func (api *BlockChainAPI) rpcMarshalHeader(ctx context.Context, header *types.Header) map[string]interface{} {
 	fields := RPCMarshalHeader(header)
 	fields["totalDifficulty"] = (*hexutil.Big)(api.b.GetTd(ctx, header.Hash()))
+	// fields["timestampInMilliseconds"] = hexutil.Uint64(header.TimeInMilliseconds())
 	return fields
 }
 
@@ -1443,6 +1444,7 @@ func (api *BlockChainAPI) rpcMarshalBlock(ctx context.Context, b *types.Block, i
 	fields := RPCMarshalBlock(b, inclTx, fullTx, api.b.ChainConfig())
 	if inclTx {
 		fields["totalDifficulty"] = (*hexutil.Big)(api.b.GetTd(ctx, b.Hash()))
+		// fields["timestampInMilliseconds"] = hexutil.Uint64(b.Header().TimeInMilliseconds())
 	}
 	return fields, nil
 }
diff --git a/miner/bid_simulator.go b/miner/bid_simulator.go
index afeaeeebda..b31463871f 100644
--- a/miner/bid_simulator.go
+++ b/miner/bid_simulator.go
@@ -398,7 +398,8 @@ func (b *bidSimulator) newBidLoop() {
 
 func (b *bidSimulator) bidBetterBefore(parentHash common.Hash) time.Time {
 	parentHeader := b.chain.GetHeaderByHash(parentHash)
-	return bidutil.BidBetterBefore(parentHeader, b.chainConfig.Parlia.Period, b.delayLeftOver, b.config.BidSimulationLeftOver)
+	blockInterval, _ := b.engine.BlockInterval(b.chain, parentHeader)
+	return bidutil.BidBetterBefore(parentHeader, blockInterval, b.delayLeftOver, b.config.BidSimulationLeftOver)
 }
 
 func (b *bidSimulator) clearLoop() {
diff --git a/miner/worker.go b/miner/worker.go
index 3e9aa2e979..fa95e46e3e 100644
--- a/miner/worker.go
+++ b/miner/worker.go
@@ -461,7 +461,7 @@ func (w *worker) newWorkLoop(recommit time.Duration) {
 			// If sealing is running resubmit a new work cycle periodically to pull in
 			// higher priced transactions. Disable this overhead for pending blocks.
 			if w.isRunning() && ((w.chainConfig.Clique != nil &&
-				w.chainConfig.Clique.Period > 0) || (w.chainConfig.Parlia != nil && w.chainConfig.Parlia.Period > 0)) {
+				w.chainConfig.Clique.Period > 0) || (w.chainConfig.Parlia != nil)) {
 				// Short circuit if no new transaction arrives.
 				commit(commitInterruptResubmit)
 			}
@@ -1032,7 +1032,7 @@ func (w *worker) prepareWork(genParams *generateParams, witness bool) (*environm
 			header.ParentBeaconRoot = genParams.beaconRoot
 		} else {
 			header.WithdrawalsHash = &types.EmptyWithdrawalsHash
-			if w.chainConfig.IsBohr(header.Number, header.Time) {
+			if w.chainConfig.IsBohr(header.Number, header.Time) && !w.chainConfig.IsLorentz(header.Number, header.Time) {
 				header.ParentBeaconRoot = new(common.Hash)
 			}
 			if w.chainConfig.IsPrague(header.Number, header.Time) {
@@ -1529,13 +1529,14 @@ func (w *worker) tryWaitProposalDoneWhenStopping() {
 		log.Warn("next proposal end block has passed, ignore")
 		return
 	}
-	if startBlock > currentBlock && (startBlock-currentBlock)*posa.BlockInterval() > w.config.MaxWaitProposalInSecs {
+	blockInterval, _ := w.engine.BlockInterval(w.chain, currentHeader)
+	if startBlock > currentBlock && uint64(time.Duration((startBlock-currentBlock)*blockInterval*uint64(time.Millisecond)).Seconds()) > w.config.MaxWaitProposalInSecs {
 		log.Warn("the next proposal start block is too far, just skip waiting")
 		return
 	}
 
 	// wait one more block for safety
-	waitSecs := (endBlock - currentBlock + 1) * posa.BlockInterval()
+	waitSecs := time.Duration((endBlock - currentBlock + 1) * blockInterval * uint64(time.Millisecond)).Seconds()
 	log.Info("The miner will propose in later, waiting for the proposal to be done",
 		"currentBlock", currentBlock, "nextProposalStart", startBlock, "nextProposalEnd", endBlock, "waitTime", waitSecs)
 	time.Sleep(time.Duration(waitSecs) * time.Second)
diff --git a/params/config.go b/params/config.go
index 6e540dc240..6d00777027 100644
--- a/params/config.go
+++ b/params/config.go
@@ -179,8 +179,7 @@ var (
 		LorentzTime: nil,
 
 		Parlia: &ParliaConfig{
-			Period: 3,
-			Epoch:  200,
+			Epoch: 200,
 		},
 	}
 
@@ -224,8 +223,7 @@ var (
 		LorentzTime: nil,
 
 		Parlia: &ParliaConfig{
-			Period: 3,
-			Epoch:  200,
+			Epoch: 200,
 		},
 	}
 
@@ -270,8 +268,7 @@ var (
 		LorentzTime: nil,
 
 		Parlia: &ParliaConfig{
-			Period: 3,
-			Epoch:  200,
+			Epoch: 200,
 		},
 	}
 
@@ -308,8 +305,7 @@ var (
 		CancunTime:          newUint64(0),
 
 		Parlia: &ParliaConfig{
-			Period: 3,
-			Epoch:  200,
+			Epoch: 200,
 		},
 	}
 
@@ -600,8 +596,7 @@ func (c CliqueConfig) String() string {
 
 // ParliaConfig is the consensus engine configs for proof-of-staked-authority based sealing.
 type ParliaConfig struct {
-	Period uint64 `json:"period"` // Number of seconds between blocks to enforce
-	Epoch  uint64 `json:"epoch"`  // Epoch length to update validatorSet
+	Epoch uint64 `json:"epoch"` // Epoch length to update validatorSet
 }
 
 // String implements the stringer interface, returning the consensus engine details.