Skip to content

Commit

Permalink
proper proofs verification on notarization
Browse files Browse the repository at this point in the history
  • Loading branch information
sstanculeanu committed Oct 23, 2024
1 parent 1158d33 commit 450d335
Show file tree
Hide file tree
Showing 4 changed files with 90 additions and 53 deletions.
7 changes: 3 additions & 4 deletions process/block/baseProcess.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,6 @@ type nonceAndHashInfo struct {

type hdrInfo struct {
usedInBlock bool
hasProof bool
hdr data.HeaderHandler
}

Expand Down Expand Up @@ -620,9 +619,9 @@ func (bp *baseProcessor) sortHeadersForCurrentBlockByNonce(usedInBlock bool) map
hdrsForCurrentBlock := make(map[uint32][]data.HeaderHandler)

bp.hdrsForCurrBlock.mutHdrsForBlock.RLock()
for _, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo {
for hdrHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo {
isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch())
hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof
hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash))
if headerInfo.usedInBlock != usedInBlock || hasMissingProof {
continue
}
Expand All @@ -645,7 +644,7 @@ func (bp *baseProcessor) sortHeaderHashesForCurrentBlockByNonce(usedInBlock bool
bp.hdrsForCurrBlock.mutHdrsForBlock.RLock()
for metaBlockHash, headerInfo := range bp.hdrsForCurrBlock.hdrHashAndInfo {
isFlagEnabledForHeader := bp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch())
hasMissingProof := isFlagEnabledForHeader && !headerInfo.hasProof
hasMissingProof := isFlagEnabledForHeader && !bp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(metaBlockHash))
if headerInfo.usedInBlock != usedInBlock || hasMissingProof {
continue
}
Expand Down
4 changes: 4 additions & 0 deletions process/block/hdrForBlock.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"sync"

"github.com/multiversx/mx-chain-core-go/data"
"github.com/multiversx/mx-chain-core-go/data/block"
)

type hdrForBlock struct {
Expand All @@ -12,19 +13,22 @@ type hdrForBlock struct {
highestHdrNonce map[uint32]uint64
mutHdrsForBlock sync.RWMutex
hdrHashAndInfo map[string]*hdrInfo
missingProofs map[string]*block.HeaderProof
}

func newHdrForBlock() *hdrForBlock {
return &hdrForBlock{
hdrHashAndInfo: make(map[string]*hdrInfo),
highestHdrNonce: make(map[uint32]uint64),
missingProofs: make(map[string]*block.HeaderProof),
}
}

func (hfb *hdrForBlock) initMaps() {
hfb.mutHdrsForBlock.Lock()
hfb.hdrHashAndInfo = make(map[string]*hdrInfo)
hfb.highestHdrNonce = make(map[uint32]uint64)
hfb.missingProofs = make(map[string]*block.HeaderProof)
hfb.mutHdrsForBlock.Unlock()
}

Expand Down
116 changes: 74 additions & 42 deletions process/block/metablock.go
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ func (mp *metaProcessor) ProcessBlock(
)
}

err = mp.waitForBlockHeadersAndProofs(haveTime())
err = mp.waitForBlockHeaders(haveTime())

mp.hdrsForCurrBlock.mutHdrsForBlock.RLock()
missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs
Expand All @@ -341,6 +341,16 @@ func (mp *metaProcessor) ProcessBlock(
}
}

// check proofs for shard data
for _, shardData := range header.ShardInfo {
// TODO: consider the validation of the proof:
// compare the one from proofsPool with what shardData.CurrentSignature and shardData.CurrentPubKeysBitmap hold
// if they are different, verify the proof received on header
if !mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash) {
return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(shardData.HeaderHash))
}
}

defer func() {
go mp.checkAndRequestIfShardHeadersMissing()
}()
Expand Down Expand Up @@ -1098,7 +1108,6 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe(
mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{
hdr: currShardHdr,
usedInBlock: true,
hasProof: true,
}
hdrsAdded++
hdrsAddedForShard[currShardHdr.GetShardID()]++
Expand Down Expand Up @@ -1138,7 +1147,10 @@ func (mp *metaProcessor) createAndProcessCrossMiniBlocksDstMe(
miniBlocks = append(miniBlocks, currMBProcessed...)
txsAdded += currTxsAdded

mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{hdr: currShardHdr, usedInBlock: true, hasProof: true}
mp.hdrsForCurrBlock.hdrHashAndInfo[string(orderedHdrsHashes[i])] = &hdrInfo{
hdr: currShardHdr,
usedInBlock: true,
}
hdrsAdded++
hdrsAddedForShard[currShardHdr.GetShardID()]++

Expand Down Expand Up @@ -1753,7 +1765,10 @@ func (mp *metaProcessor) getLastCrossNotarizedShardHdrs() (map[uint32]data.Heade
log.Debug("lastCrossNotarizedHeader for shard", "shardID", shardID, "hash", hash)
lastCrossNotarizedHeader[shardID] = lastCrossNotarizedHeaderForShard
usedInBlock := mp.isGenesisShardBlockAndFirstMeta(lastCrossNotarizedHeaderForShard.GetNonce())
mp.hdrsForCurrBlock.hdrHashAndInfo[string(hash)] = &hdrInfo{hdr: lastCrossNotarizedHeaderForShard, usedInBlock: usedInBlock}
mp.hdrsForCurrBlock.hdrHashAndInfo[string(hash)] = &hdrInfo{
hdr: lastCrossNotarizedHeaderForShard,
usedInBlock: usedInBlock,
}
}

return lastCrossNotarizedHeader, nil
Expand Down Expand Up @@ -1939,53 +1954,44 @@ func (mp *metaProcessor) receivedShardHeader(headerHandler data.HeaderHandler, s
hdrInfoForHash.hdr = shardHeader
mp.hdrsForCurrBlock.missingHdrs--

// if there is an entry for the missing proof, it means that proofsPool did not have it while scanning shardData
// thus header epoch was not available at that time
incompleteProof, hasMissingProof := mp.hdrsForCurrBlock.missingProofs[string(shardHeaderHash)]
if hasMissingProof {
constructedProof := &block.HeaderProof{
PubKeysBitmap: incompleteProof.PubKeysBitmap,
AggregatedSignature: incompleteProof.AggregatedSignature,
HeaderHash: incompleteProof.HeaderHash,
HeaderEpoch: shardHeader.GetEpoch(),
HeaderNonce: incompleteProof.HeaderNonce,
HeaderShardId: incompleteProof.HeaderShardId,
}
errAddProof := mp.proofsPool.AddProof(constructedProof)
if errAddProof != nil {
log.Trace("could not add the constructed proof after header received", "hash", hex.EncodeToString(incompleteProof.HeaderHash))
}

delete(mp.hdrsForCurrBlock.missingProofs, string(shardHeaderHash))
}

if shardHeader.GetNonce() > mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] {
mp.hdrsForCurrBlock.highestHdrNonce[shardHeader.GetShardID()] = shardHeader.GetNonce()
}
}

if hdrInfoForHash != nil && !hdrInfoForHash.hasProof {
hdrInfoForHash.hasProof = mp.proofsPool.HasProof(shardHeader.GetShardID(), shardHeaderHash)
}

shouldConsiderProofsForNotarization := mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHeader.GetEpoch())
if mp.hdrsForCurrBlock.missingHdrs == 0 {
if !shouldConsiderProofsForNotarization {
mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders()
if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 {
log.Debug("received all missing finality attesting shard headers")
}
if mp.hdrsForCurrBlock.missingHdrs == 0 && !shouldConsiderProofsForNotarization {
mp.hdrsForCurrBlock.missingFinalityAttestingHdrs = mp.requestMissingFinalityAttestingShardHeaders()
if mp.hdrsForCurrBlock.missingFinalityAttestingHdrs == 0 {
log.Debug("received all missing finality attesting shard headers")
}
}

missingShardHdrs := mp.hdrsForCurrBlock.missingHdrs
missingFinalityAttestingShardHdrs := mp.hdrsForCurrBlock.missingFinalityAttestingHdrs
mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock()

hasAllProofs := true
for shardHdrHash, shardHdr := range mp.hdrsForCurrBlock.hdrHashAndInfo {
if check.IfNil(shardHdr.hdr) {
continue
}

if !mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, shardHdr.hdr.GetEpoch()) {
continue
}

if shardHdr.hasProof {
continue
}

hasProof := mp.proofsPool.HasProof(shardHdr.hdr.GetShardID(), []byte(shardHdrHash))
if !hasProof {
hasAllProofs = false
break
}
}
hasMissingProofs := !hasAllProofs && shouldConsiderProofsForNotarization

allMissingShardHeadersReceived := missingShardHdrs == 0 &&
missingFinalityAttestingShardHdrs == 0 && !hasMissingProofs
allMissingShardHeadersReceived := missingShardHdrs == 0 && missingFinalityAttestingShardHdrs == 0
if allMissingShardHeadersReceived {
mp.chRcvAllHdrs <- true
}
Expand Down Expand Up @@ -2058,6 +2064,15 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock
usedInBlock: true,
}

mp.hdrsForCurrBlock.missingProofs[string(shardData.HeaderHash)] = &block.HeaderProof{
PubKeysBitmap: shardData.PubKeysBitmap,
AggregatedSignature: shardData.Signature,
HeaderHash: shardData.HeaderHash,
HeaderEpoch: 0, // will come later with the header
HeaderNonce: shardData.Nonce,
HeaderShardId: shardData.ShardID,
}

go mp.requestHandler.RequestShardHeader(shardData.ShardID, shardData.HeaderHash)
continue
}
Expand All @@ -2076,7 +2091,20 @@ func (mp *metaProcessor) computeExistingAndRequestMissingShardHeaders(metaBlock
notarizedShardHdrsBasedOnProofs++

hasProofForShardHdr := mp.proofsPool.HasProof(shardData.ShardID, shardData.HeaderHash)
mp.hdrsForCurrBlock.hdrHashAndInfo[string(shardData.HeaderHash)].hasProof = hasProofForShardHdr
if !hasProofForShardHdr {
// TODO: consider verifying the proof before adding it into the proofsPool
errAddProof := mp.proofsPool.AddProof(&block.HeaderProof{
PubKeysBitmap: shardData.CurrentPubKeysBitmap,
AggregatedSignature: shardData.CurrentSignature,
HeaderHash: shardData.HeaderHash,
HeaderEpoch: hdr.GetEpoch(),
HeaderNonce: shardData.Nonce,
HeaderShardId: shardData.ShardID,
})
if errAddProof != nil {
log.Trace("could not add proof from shard data for header", "hash", hex.EncodeToString(shardData.HeaderHash))
}
}
}
}

Expand Down Expand Up @@ -2104,7 +2132,7 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) {

isBlockAfterEquivalentMessagesFlag := check.IfNil(headerInfo.hdr) &&
mp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, headerInfo.hdr.GetEpoch())
hasMissingShardHdrProof := isBlockAfterEquivalentMessagesFlag && !headerInfo.hasProof
hasMissingShardHdrProof := isBlockAfterEquivalentMessagesFlag && !mp.proofsPool.HasProof(headerInfo.hdr.GetShardID(), []byte(hdrHash))
if hasMissingShardHdrProof {
return nil, fmt.Errorf("%w for shard header with hash %s", process.ErrMissingHeaderProof, hdrHash)
}
Expand Down Expand Up @@ -2133,6 +2161,7 @@ func (mp *metaProcessor) createShardInfo() ([]data.ShardDataHandler, error) {
return nil, err
}
shardData.CurrentPubKeysBitmap = currentProof.GetPubKeysBitmap()
shardData.CurrentSignature = currentProof.GetAggregatedSignature()
}
shardData.NumPendingMiniBlocks = uint32(len(mp.pendingMiniBlocksHandler.GetPendingMiniBlocks(shardData.ShardID)))
header, _, err := mp.blockTracker.GetLastSelfNotarizedHeader(shardHdr.GetShardID())
Expand Down Expand Up @@ -2371,7 +2400,10 @@ func (mp *metaProcessor) prepareBlockHeaderInternalMapForValidatorProcessor() {
}

mp.hdrsForCurrBlock.mutHdrsForBlock.Lock()
mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{false, false, currentBlockHeader}
mp.hdrsForCurrBlock.hdrHashAndInfo[string(currentBlockHeaderHash)] = &hdrInfo{
usedInBlock: false,
hdr: currentBlockHeader,
}
mp.hdrsForCurrBlock.mutHdrsForBlock.Unlock()
}

Expand All @@ -2398,7 +2430,7 @@ func (mp *metaProcessor) verifyValidatorStatisticsRootHash(header *block.MetaBlo
return nil
}

func (mp *metaProcessor) waitForBlockHeadersAndProofs(waitTime time.Duration) error {
func (mp *metaProcessor) waitForBlockHeaders(waitTime time.Duration) error {
select {
case <-mp.chRcvAllHdrs:
return nil
Expand Down
16 changes: 9 additions & 7 deletions process/block/shardblock.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ func (sp *shardProcessor) ProcessBlock(
)
}

err = sp.waitForMetaHdrHashesAndProofs(haveTime())
err = sp.waitForMetaHdrHashes(haveTime())

sp.hdrsForCurrBlock.mutHdrsForBlock.RLock()
missingMetaHdrs := sp.hdrsForCurrBlock.missingHdrs
Expand Down Expand Up @@ -293,6 +293,13 @@ func (sp *shardProcessor) ProcessBlock(
return process.ErrAccountStateDirty
}

// check proofs for shard data
for _, metaBlockHash := range header.GetMetaBlockHashes() {
if !sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHash) {
return fmt.Errorf("%w for header hash %s", process.ErrMissingHeaderProof, hex.EncodeToString(metaBlockHash))
}
}

defer func() {
go sp.checkAndRequestIfMetaHeadersMissing()
}()
Expand Down Expand Up @@ -1803,9 +1810,6 @@ func (sp *shardProcessor) computeExistingAndRequestMissingMetaHeaders(header dat
shouldConsiderProofsForNotarization := sp.enableEpochsHandler.IsFlagEnabledInEpoch(common.EquivalentMessagesFlag, hdr.Epoch)
if shouldConsiderProofsForNotarization {
notarizedMetaHdrsBasedOnProofs++

hasProofForShardHdr := sp.proofsPool.HasProof(core.MetachainShardId, metaBlockHashes[i])
sp.hdrsForCurrBlock.hdrHashAndInfo[string(metaBlockHashes[i])].hasProof = hasProofForShardHdr
}
}

Expand Down Expand Up @@ -1952,7 +1956,6 @@ func (sp *shardProcessor) createAndProcessMiniBlocksDstMe(haveTime func() bool)
sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{
hdr: createAndProcessInfo.currMetaHdr,
usedInBlock: true,
hasProof: true,
}
createAndProcessInfo.numHdrsAdded++
lastMetaHdr = createAndProcessInfo.currMetaHdr
Expand Down Expand Up @@ -2020,7 +2023,6 @@ func (sp *shardProcessor) createMbsAndProcessCrossShardTransactionsDstMe(
sp.hdrsForCurrBlock.hdrHashAndInfo[string(createAndProcessInfo.currMetaHdrHash)] = &hdrInfo{
hdr: createAndProcessInfo.currMetaHdr,
usedInBlock: true,
hasProof: true,
}
createAndProcessInfo.numHdrsAdded++
createAndProcessInfo.hdrAdded = true
Expand Down Expand Up @@ -2259,7 +2261,7 @@ func (sp *shardProcessor) applyBodyToHeader(
return newBody, nil
}

func (sp *shardProcessor) waitForMetaHdrHashesAndProofs(waitTime time.Duration) error {
func (sp *shardProcessor) waitForMetaHdrHashes(waitTime time.Duration) error {
select {
case <-sp.chRcvAllMetaHdrs:
return nil
Expand Down

0 comments on commit 450d335

Please sign in to comment.