From 68b727b871d2543ac0e49e5f6f5218e0770d0929 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 08:01:53 -0500 Subject: [PATCH 1/9] wip --- api.go | 15 +- encoding.go | 113 -- encoding_test.go | 106 +- epoch.canoto_test.go | 215 +++ epoch.go | 115 +- epoch_multinode_test.go | 14 +- epoch_test.go | 42 +- global.go | 19 - go.mod | 3 +- go.sum | 10 +- metadata.go | 124 -- metadata_test.go | 35 +- msg.go | 105 +- record/consts.go | 11 - types.canoto.go | 2786 +++++++++++++++++++++++++++++++++++++++ types.go | 150 +++ 16 files changed, 3361 insertions(+), 502 deletions(-) delete mode 100644 encoding.go create mode 100644 epoch.canoto_test.go delete mode 100644 global.go delete mode 100644 metadata.go delete mode 100644 record/consts.go create mode 100644 types.canoto.go create mode 100644 types.go diff --git a/api.go b/api.go index 388dbac9..e56b3e3e 100644 --- a/api.go +++ b/api.go @@ -95,12 +95,15 @@ type BlockDeserializer interface { DeserializeBlock(bytes []byte) (Block, error) } -// Signature encodes a signature and the node that signed it, without the message it was signed on. -type Signature struct { - // Signer is the NodeID of the creator of the signature. - Signer NodeID - // Value is the byte representation of the signature. - Value []byte +// QuorumCertificate is equivalent to a collection of signatures from a quorum of nodes, +type QuorumCertificate interface { + // Signers returns who participated in creating this QuorumCertificate. + Signers() []NodeID + // Verify checks whether the nodes participated in creating this QuorumCertificate, + // signed the given message. + Verify(msg []byte) error + // Bytes returns a raw representation of the given QuorumCertificate. + Bytes() []byte } // QCDeserializer deserializes QuorumCertificates according to formatting diff --git a/encoding.go b/encoding.go deleted file mode 100644 index 8986780b..00000000 --- a/encoding.go +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package simplex - -import ( - "encoding/asn1" - "encoding/binary" - "errors" - "fmt" - "simplex/record" -) - -type QuorumRecord struct { - QC []byte - Vote []byte -} - -func finalizationFromRecord(payload []byte) ([]byte, ToBeSignedFinalization, error) { - var nr QuorumRecord - _, err := asn1.Unmarshal(payload, &nr) - if err != nil { - return nil, ToBeSignedFinalization{}, err - } - - var finalization ToBeSignedFinalization - if err := finalization.FromBytes(nr.Vote); err != nil { - return nil, ToBeSignedFinalization{}, err - } - - return nr.QC, finalization, nil -} - -func quorumRecord(qc []byte, rawVote []byte, recordType uint16) []byte { - var qr QuorumRecord - qr.QC = qc - qr.Vote = rawVote - - payload, err := asn1.Marshal(qr) - if err != nil { - panic(err) - } - - buff := make([]byte, len(payload)+2) - binary.BigEndian.PutUint16(buff, recordType) - copy(buff[2:], payload) - - return buff -} - -func NotarizationFromRecord(record []byte) ([]byte, ToBeSignedVote, error) { - record = record[2:] - var nr QuorumRecord - _, err := asn1.Unmarshal(record, &nr) - if err != nil { - return nil, ToBeSignedVote{}, err - } - - var vote ToBeSignedVote - if err := vote.FromBytes(nr.Vote); err != nil { - return nil, ToBeSignedVote{}, err - } - - return nr.QC, vote, nil -} - -func blockRecord(bh BlockHeader, blockData []byte) []byte { - mdBytes := bh.Bytes() - - mdSizeBuff := make([]byte, 4) - binary.BigEndian.PutUint32(mdSizeBuff, uint32(len(mdBytes))) - - blockDataSizeBuff := make([]byte, 4) - binary.BigEndian.PutUint32(blockDataSizeBuff, uint32(len(blockData))) - - buff := make([]byte, len(mdBytes)+len(blockData)+len(mdSizeBuff)+len(blockDataSizeBuff)+2) - binary.BigEndian.PutUint16(buff, record.BlockRecordType) - copy(buff[2:], mdSizeBuff) - copy(buff[6:], blockDataSizeBuff) - copy(buff[10:], mdBytes) - copy(buff[10+len(mdBytes):], blockData) - - return buff -} - -func blockFromRecord(buff []byte) (BlockHeader, []byte, error) { - buff = buff[2:] - if len(buff) < 8 { - return BlockHeader{}, nil, errors.New("buffer too small, expected 8 bytes") - } - - mdSizeBuff := binary.BigEndian.Uint32(buff) - blockDataSizeBuff := binary.BigEndian.Uint32(buff[4:]) - - buff = buff[8:] - - expectedBuffSize := int(mdSizeBuff + blockDataSizeBuff) - - if len(buff) < expectedBuffSize { - return BlockHeader{}, nil, fmt.Errorf("buffer too small, expected %d bytes", expectedBuffSize) - } - - mdBuff := buff[:mdSizeBuff] - - var bh BlockHeader - if err := bh.FromBytes(mdBuff); err != nil { - return BlockHeader{}, nil, fmt.Errorf("failed to deserialize block metadata: %w", err) - } - - payload := buff[mdSizeBuff:] - - return bh, payload, nil -} diff --git a/encoding_test.go b/encoding_test.go index f9d8d98a..2d8e3c60 100644 --- a/encoding_test.go +++ b/encoding_test.go @@ -6,7 +6,6 @@ package simplex import ( "crypto/rand" "crypto/sha256" - "simplex/record" "testing" "github.com/stretchr/testify/require" @@ -15,10 +14,9 @@ import ( func TestBlockRecord(t *testing.T) { bh := BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: 1, - Round: 2, - Seq: 3, - Epoch: 4, + Round: 2, + Seq: 3, + Epoch: 4, }, } @@ -30,13 +28,18 @@ func TestBlockRecord(t *testing.T) { payload := []byte{11, 12, 13, 14, 15, 16} - record := blockRecord(bh, payload) - - md2, payload2, err := blockFromRecord(record) - require.NoError(t, err) + record := Record{ + Block: &BlockRecord{ + Header: bh, + Payload: payload, + }, + } + recordBytes := record.MarshalCanoto() - require.Equal(t, bh, md2) - require.Equal(t, payload, payload2) + var parsedRecord Record + require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) + require.True(t, bh.Equals(&parsedRecord.Block.Header)) + require.Equal(t, payload, parsedRecord.Block.Payload) } func FuzzBlockRecord(f *testing.F) { @@ -45,22 +48,26 @@ func FuzzBlockRecord(f *testing.F) { digest := sha256.Sum256(digestPreimage) bh := BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: version, - Round: round, - Seq: seq, - Epoch: epoch, - Prev: prev, + Round: round, + Seq: seq, + Epoch: epoch, + Prev: prev, }, Digest: digest, } - record := blockRecord(bh, payload) - - md2, payload2, err := blockFromRecord(record) - require.NoError(t, err) + record := Record{ + Block: &BlockRecord{ + Header: bh, + Payload: payload, + }, + } + recordBytes := record.MarshalCanoto() - require.Equal(t, bh, md2) - require.Equal(t, payload, payload2) + var parsedRecord Record + require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) + require.True(t, bh.Equals(&parsedRecord.Block.Header)) + require.Equal(t, payload, parsedRecord.Block.Payload) }) } @@ -72,10 +79,9 @@ func TestNotarizationRecord(t *testing.T) { vote := ToBeSignedVote{ BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: 1, - Round: 2, - Seq: 3, - Epoch: 4, + Round: 2, + Seq: 3, + Epoch: 4, }, }, } @@ -83,43 +89,51 @@ func TestNotarizationRecord(t *testing.T) { _, err = rand.Read(vote.Prev[:]) require.NoError(t, err) - _, err = rand.Read(vote.Prev[:]) + _, err = rand.Read(vote.Digest[:]) require.NoError(t, err) - record := quorumRecord([]byte{1, 2, 3}, vote.Bytes(), record.NotarizationRecordType) - qc, vote2, err := NotarizationFromRecord(record) - require.NoError(t, err) - require.Equal(t, []byte{1, 2, 3}, qc) - require.Equal(t, vote, vote2) + record := Record{ + Notarization: &QuorumRecord{ + Header: vote.BlockHeader, + QC: []byte{1, 2, 3}, + }, + } + recordBytes := record.MarshalCanoto() + + var parsedRecord Record + require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) + require.True(t, vote.BlockHeader.Equals(&parsedRecord.Notarization.Header)) + require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) } func FuzzNotarizationRecord(f *testing.F) { - f.Fuzz(func(t *testing.T, version uint8, round uint64, seq uint64, epoch uint64, prevPreimage, digestPreimage []byte, sig []byte, signer1, signer2 []byte) { + f.Fuzz(func(t *testing.T, round uint64, seq uint64, epoch uint64, prevPreimage, digestPreimage []byte, sig []byte, signer1, signer2 []byte) { prev := sha256.Sum256(prevPreimage) digest := sha256.Sum256(digestPreimage) vote := ToBeSignedVote{ BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: version, - Round: round, - Seq: seq, - Epoch: epoch, - Prev: prev, + Round: round, + Seq: seq, + Epoch: epoch, + Prev: prev, }, Digest: digest, }, } - var signers []NodeID - for _, signer := range [][]byte{signer1, signer2} { - signers = append(signers, signer) + record := Record{ + Notarization: &QuorumRecord{ + Header: vote.BlockHeader, + QC: []byte{1, 2, 3}, + }, } + recordBytes := record.MarshalCanoto() - record := quorumRecord([]byte{1, 2, 3}, vote.Bytes(), record.NotarizationRecordType) - qc, vote2, err := NotarizationFromRecord(record) - require.NoError(t, err) - require.Equal(t, []byte{1, 2, 3}, qc) - require.Equal(t, vote, vote2) + var parsedRecord Record + require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) + require.True(t, vote.BlockHeader.Equals(&parsedRecord.Notarization.Header)) + require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) }) } diff --git a/epoch.canoto_test.go b/epoch.canoto_test.go new file mode 100644 index 00000000..0dd120d8 --- /dev/null +++ b/epoch.canoto_test.go @@ -0,0 +1,215 @@ +// Code generated by canoto. DO NOT EDIT. +// versions: +// canoto v0.10.0 +// source: epoch_test.go + +package simplex_test + +import ( + "io" + "sync/atomic" + "unicode/utf8" + + "github.com/StephenButtolph/canoto" +) + +// Ensure that unused imports do not error +var ( + _ atomic.Int64 + + _ = io.ErrUnexpectedEOF + _ = utf8.ValidString +) + +const ( + canoto__testQC__qc__tag = "\x0a" // canoto.Tag(1, canoto.Len) +) + +type canotoData_testQC struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*testQC) MakeCanoto() *testQC { + return new(testQC) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *testQC) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *testQC) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + + remainingBytes := r.B + count, err := canoto.CountBytes(remainingBytes, canoto__testQC__qc__tag) + if err != nil { + return err + } + + c.qc = canoto.MakeSlice(c.qc, 1+count) + if len(msgBytes) != 0 { + r.B = msgBytes + err = (&c.qc[0]).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + } + + for i := range count { + r.B = r.B[len(canoto__testQC__qc__tag):] + r.Unsafe = true + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + + if len(msgBytes) != 0 { + remainingBytes := r.B + r.B = msgBytes + err = (&c.qc[1+i]).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + } + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *testQC) ValidCanoto() bool { + if c == nil { + return true + } + for i := range c.qc { + if !(&c.qc[i]).ValidCanoto() { + return false + } + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *testQC) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + for i := range c.qc { + (&c.qc[i]).CalculateCanotoCache() + fieldSize := (&c.qc[i]).CachedCanotoSize() + c.canotoData.size += len(canoto__testQC__qc__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *testQC) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *testQC) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *testQC) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + for i := range c.qc { + canoto.Append(&w, canoto__testQC__qc__tag) + canoto.AppendInt(&w, int64((&c.qc[i]).CachedCanotoSize())) + w = (&c.qc[i]).MarshalCanotoInto(w) + } + return w +} diff --git a/epoch.go b/epoch.go index c85ec375..a81a8d50 100644 --- a/epoch.go +++ b/epoch.go @@ -8,7 +8,6 @@ import ( "context" "errors" "fmt" - "simplex/record" "time" "go.uber.org/zap" @@ -96,15 +95,15 @@ func (e *Epoch) HandleMessage(msg *Message, from NodeID) error { } switch { - case msg.BlockMessage != nil: + case msg.Proposal != nil: return e.handleBlockMessage(msg, from) - case msg.VoteMessage != nil: + case msg.Vote != nil: return e.handleVoteMessage(msg, from) case msg.Notarization != nil: return e.handleNotarizationMessage(msg, from) - case msg.Finalization != nil: + case msg.Finalize != nil: return e.handleFinalizationMessage(msg, from) - case msg.FinalizationCertificate != nil: + case msg.Finalization != nil: return e.handleFinalizationCertificateMessage(msg, from) default: return fmt.Errorf("invalid message type: %v", msg) @@ -174,7 +173,7 @@ func (e *Epoch) Stop() { } func (e *Epoch) handleFinalizationCertificateMessage(message *Message, from NodeID) error { - fCert := message.FinalizationCertificate + fCert := message.Finalization round, exists := e.rounds[fCert.Finalization.Round] if !exists { e.Logger.Debug("Received finalization certificate for a non existent round", zap.Int("round", int(fCert.Finalization.Round))) @@ -241,7 +240,7 @@ func (e *Epoch) validateFinalizationQC(fCert *FinalizationCertificate) (bool, er } func (e *Epoch) handleFinalizationMessage(message *Message, from NodeID) error { - msg := message.Finalization + msg := message.Finalize finalization := msg.Finalization // Only process a point to point finalization @@ -272,7 +271,7 @@ func (e *Epoch) handleFinalizationMessage(message *Message, from NodeID) error { } func (e *Epoch) handleVoteMessage(message *Message, _ NodeID) error { - msg := message.VoteMessage + msg := message.Vote vote := msg.Vote // TODO: what if we've received a vote for a round we didn't instantiate yet? @@ -345,7 +344,7 @@ func (e *Epoch) assembleFinalizationCertificate(round *Round) error { finalizationsByMD := make(map[string][]*Finalization) for _, vote := range round.finalizations { - key := string(vote.Finalization.Bytes()) + key := string(vote.Finalization.MarshalCanoto()) finalizationsByMD[key] = append(finalizationsByMD[key], vote) } @@ -411,7 +410,13 @@ func (e *Epoch) persistFinalizationCertificate(fCert FinalizationCertificate) er delete(messagesFromNode, fCert.Finalization.Round) } } else { - recordBytes := quorumRecord(fCert.QC.Bytes(), fCert.Finalization.Bytes(), record.FinalizationRecordType) + record := Record{ + Finalization: &QuorumRecord{ + Header: fCert.Finalization.BlockHeader, + QC: fCert.QC.Bytes(), + }, + } + recordBytes := record.MarshalCanoto() if err := e.WAL.Append(recordBytes); err != nil { e.Logger.Error("Failed to append finalization certificate record to WAL", zap.Error(err)) return err @@ -423,7 +428,7 @@ func (e *Epoch) persistFinalizationCertificate(fCert FinalizationCertificate) er zap.Stringer("digest", fCert.Finalization.BlockHeader.Digest)) } - finalizationCertificate := &Message{FinalizationCertificate: &fCert} + finalizationCertificate := &Message{Finalization: &fCert} e.Comm.Broadcast(finalizationCertificate) e.Logger.Debug("Broadcast finalization certificate", @@ -464,7 +469,7 @@ func (e *Epoch) maybeCollectNotarization() error { return e.assembleNotarization(votesForCurrentRound, digestWeExpect) } -func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, digest [metadataDigestLen]byte) error { +func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, digest Digest) error { vote := ToBeSignedVote{ BlockHeader{ ProtocolMetadata: ProtocolMetadata{ @@ -501,19 +506,24 @@ func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, dige } func (e *Epoch) persistNotarization(notarization Notarization, vote ToBeSignedVote) error { - notarizationMessage := &Message{Notarization: ¬arization} - record := quorumRecord(notarization.QC.Bytes(), vote.Bytes(), record.NotarizationRecordType) - - if err := e.WAL.Append(record); err != nil { + record := Record{ + Notarization: &QuorumRecord{ + Header: vote.BlockHeader, + QC: notarization.QC.Bytes(), + }, + } + recordBytes := record.MarshalCanoto() + if err := e.WAL.Append(recordBytes); err != nil { e.Logger.Error("Failed to append notarization record to WAL", zap.Error(err)) return err } e.Logger.Debug("Persisted notarization to WAL", - zap.Int("size", len(record)), + zap.Int("size", len(recordBytes)), zap.Uint64("round", notarization.Vote.Round), zap.Stringer("digest", notarization.Vote.BlockHeader.Digest)) + notarizationMessage := &Message{Notarization: ¬arization} e.Comm.Broadcast(notarizationMessage) e.Logger.Debug("Broadcast notarization", @@ -587,13 +597,13 @@ func (e *Epoch) hasSomeNodeSignedTwice(nodeIDs []NodeID) bool { } func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { - block := message.BlockMessage.Block + block := message.Proposal.Block if block == nil { e.Logger.Debug("Got empty block in a BlockMessage") return nil } - vote := message.BlockMessage.Vote + vote := message.Proposal.Vote from := vote.Signature.Signer md := block.BlockHeader() @@ -668,8 +678,15 @@ func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { e.Logger.Debug("Failed verifying block", zap.Error(err)) return nil } - record := blockRecord(md, block.Bytes()) - if err := e.WAL.Append(record); err != nil { + + record := Record{ + Block: &BlockRecord{ + Header: md, + Payload: block.Bytes(), + }, + } + recordBytes := record.MarshalCanoto() + if err := e.WAL.Append(recordBytes); err != nil { e.Logger.Error("Failed appending block to WAL", zap.Error(err)) return err } @@ -681,7 +698,7 @@ func (e *Epoch) wasBlockAlreadyVerified(from NodeID, md BlockHeader) bool { var alreadyVerified bool msgsForRound, exists := e.futureMessages[string(from)][md.Round] if exists && msgsForRound.proposal != nil { - bh := msgsForRound.proposal.BlockMessage.Block.BlockHeader() + bh := msgsForRound.proposal.Proposal.Block.BlockHeader() alreadyVerified = bh.Equals(&md) } return alreadyVerified @@ -690,18 +707,13 @@ func (e *Epoch) wasBlockAlreadyVerified(from NodeID, md BlockHeader) bool { func (e *Epoch) verifyProposalIsPartOfOurChain(block Block) bool { bh := block.BlockHeader() - if bh.Version != 0 { - e.Logger.Debug("Got block message with wrong version number, expected 0", zap.Uint8("version", bh.Version)) - return false - } - if e.Epoch != bh.Epoch { e.Logger.Debug("Got block message but the epoch mismatches our epoch", zap.Uint64("our epoch", e.Epoch), zap.Uint64("block epoch", bh.Epoch)) } var expectedSeq uint64 - var expectedPrevDigest [metadataDigestLen]byte + var expectedPrevDigest Digest // Else, either it's not the first block, or we haven't committed the first block, and it is the first block. // If it's the latter we have nothing else to do. @@ -735,11 +747,10 @@ func (e *Epoch) verifyProposalIsPartOfOurChain(block Block) bool { expectedBH := BlockHeader{ Digest: digest, ProtocolMetadata: ProtocolMetadata{ - Round: e.round, - Seq: expectedSeq, - Epoch: e.Epoch, - Prev: expectedPrevDigest, - Version: 0, + Round: e.round, + Seq: expectedSeq, + Epoch: e.Epoch, + Prev: expectedPrevDigest, }, } @@ -798,15 +809,20 @@ func (e *Epoch) proposeBlock() error { // Write record to WAL before broadcasting it, so that // if we crash during broadcasting, we know what we sent. - rawBlock := block.Bytes() - record := blockRecord(block.BlockHeader(), rawBlock) - if err := e.WAL.Append(record); err != nil { + record := Record{ + Block: &BlockRecord{ + Header: block.BlockHeader(), + Payload: block.Bytes(), + }, + } + recordBytes := record.MarshalCanoto() + if err := e.WAL.Append(recordBytes); err != nil { e.Logger.Error("Failed appending block to WAL", zap.Error(err)) return err } e.Logger.Debug("Wrote block to WAL", zap.Uint64("round", md.Round), - zap.Int("size", len(rawBlock)), + zap.Int("size", len(record.Block.Payload)), zap.Stringer("digest", md.Digest)) vote, err := e.voteOnBlock(block) @@ -815,7 +831,7 @@ func (e *Epoch) proposeBlock() error { } proposal := &Message{ - BlockMessage: &BlockMessage{ + Proposal: &BlockMessage{ Block: block, Vote: vote, }, @@ -828,14 +844,14 @@ func (e *Epoch) proposeBlock() error { e.Comm.Broadcast(proposal) e.Logger.Debug("Proposal broadcast", zap.Uint64("round", md.Round), - zap.Int("size", len(rawBlock)), + zap.Int("size", len(record.Block.Payload)), zap.Stringer("digest", md.Digest)) - return e.handleVoteMessage(&Message{VoteMessage: &vote}, e.ID) + return e.handleVoteMessage(&Message{Vote: &vote}, e.ID) } func (e *Epoch) Metadata() ProtocolMetadata { - var prev [metadataDigestLen]byte + var prev Digest seq := e.Storage.Height() if e.lastBlock != nil { // Build on top of the latest block @@ -845,11 +861,10 @@ func (e *Epoch) Metadata() ProtocolMetadata { } md := ProtocolMetadata{ - Round: e.round, - Seq: seq, - Epoch: e.Epoch, - Prev: prev, - Version: 0, + Round: e.round, + Seq: seq, + Epoch: e.Epoch, + Prev: prev, } return md } @@ -863,7 +878,7 @@ func (e *Epoch) startRound() error { // If we're not the leader, check if we have received a proposal earlier for this round msgsForRound, exists := e.futureMessages[string(leaderForCurrentRound)][e.round] - if !exists || msgsForRound.proposal == nil || msgsForRound.proposal.BlockMessage == nil { + if !exists || msgsForRound.proposal == nil || msgsForRound.proposal.Proposal == nil { return nil } @@ -881,7 +896,7 @@ func (e *Epoch) doProposed(block Block, voteFromLeader Vote) error { // We do not write the vote to the WAL as we have written the block itself to the WAL // and we can always restore the block and sign it again if needed. voteMsg := &Message{ - VoteMessage: &vote, + Vote: &vote, } e.Logger.Debug("Broadcasting vote", @@ -894,7 +909,7 @@ func (e *Epoch) doProposed(block Block, voteFromLeader Vote) error { return err } - return e.handleVoteMessage(&Message{VoteMessage: &voteFromLeader}, e.ID) + return e.handleVoteMessage(&Message{Vote: &voteFromLeader}, e.ID) } func (e *Epoch) voteOnBlock(block Block) (Vote, error) { @@ -942,7 +957,7 @@ func (e *Epoch) doNotarized() error { } finalizationMsg := &Message{ - Finalization: &sf, + Finalize: &sf, } e.Comm.Broadcast(finalizationMsg) diff --git a/epoch_multinode_test.go b/epoch_multinode_test.go index d1a1a8c9..35a9af33 100644 --- a/epoch_multinode_test.go +++ b/epoch_multinode_test.go @@ -6,9 +6,7 @@ package simplex_test import ( "bytes" "context" - "encoding/binary" . "simplex" - "simplex/record" "simplex/wal" "sync" "testing" @@ -116,19 +114,15 @@ func (t *testInstance) assertNotarization(round uint64) { require.NoError(t.t, err) for _, rawRecord := range rawRecords { - if binary.BigEndian.Uint16(rawRecord[:2]) == record.NotarizationRecordType { - _, vote, err := NotarizationFromRecord(rawRecord) - require.NoError(t.t, err) - - if vote.Round == round { - return - } + var record Record + require.NoError(t.t, record.UnmarshalCanoto(rawRecord)) + if record.Notarization != nil && record.Notarization.Header.Round == round { + return } } t.wal.signal.Wait() } - } func (t *testInstance) handleMessages() { diff --git a/epoch_test.go b/epoch_test.go index f038fca8..e1160e99 100644 --- a/epoch_test.go +++ b/epoch_test.go @@ -1,6 +1,8 @@ // Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. +//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false $GOFILE + package simplex_test import ( @@ -8,7 +10,6 @@ import ( "context" "crypto/rand" "crypto/sha256" - "encoding/asn1" "encoding/binary" "fmt" . "simplex" @@ -63,7 +64,7 @@ func TestEpochSimpleFlow(t *testing.T) { // send node a message from the leader vote := newVote(block, leader) e.HandleMessage(&Message{ - BlockMessage: &BlockMessage{ + Proposal: &BlockMessage{ Vote: *vote, Block: block, }, @@ -107,7 +108,7 @@ func newVote(block *testBlock, id NodeID) *Vote { func injectVote(t *testing.T, e *Epoch, block *testBlock, id NodeID) { err := e.HandleMessage(&Message{ - VoteMessage: newVote(block, id), + Vote: newVote(block, id), }, id) require.NoError(t, err) @@ -116,7 +117,7 @@ func injectVote(t *testing.T, e *Epoch, block *testBlock, id NodeID) { func injectFinalization(t *testing.T, e *Epoch, block *testBlock, id NodeID) { md := block.BlockHeader() err := e.HandleMessage(&Message{ - Finalization: &Finalization{ + Finalize: &Finalization{ Signature: Signature{ Signer: id, }, @@ -146,25 +147,30 @@ type testQCDeserializer struct { } func (t *testQCDeserializer) DeserializeQuorumCertificate(bytes []byte) (QuorumCertificate, error) { - var qc []Signature - rest, err := asn1.Unmarshal(bytes, &qc) + var tqc testQC + err := tqc.UnmarshalCanoto(bytes) require.NoError(t.t, err) - require.Empty(t.t, rest) - return testQC(qc), err + return tqc, err } type testSignatureAggregator struct { } func (t *testSignatureAggregator) Aggregate(signatures []Signature) (QuorumCertificate, error) { - return testQC(signatures), nil + return testQC{ + qc: signatures, + }, nil } -type testQC []Signature +type testQC struct { + qc []Signature `canoto:"repeated value,1"` + + canotoData canotoData_testQC +} func (t testQC) Signers() []NodeID { - res := make([]NodeID, 0, len(t)) - for _, sig := range t { + res := make([]NodeID, 0, len(t.qc)) + for _, sig := range t.qc { res = append(res, sig.Signer) } return res @@ -175,11 +181,7 @@ func (t testQC) Verify(msg []byte) error { } func (t testQC) Bytes() []byte { - bytes, err := asn1.Marshal(t) - if err != nil { - panic(err) - } - return bytes + return t.MarshalCanoto() } type testSigner struct { @@ -275,7 +277,7 @@ func (t *testBlock) Bytes() []byte { ProtocolMetadata: t.metadata, } - mdBuff := bh.Bytes() + mdBuff := bh.MarshalCanoto() buff := make([]byte, len(t.data)+len(mdBuff)+4) binary.BigEndian.PutUint32(buff, uint32(len(t.data))) @@ -357,8 +359,8 @@ type blockDeserializer struct { func (b *blockDeserializer) DeserializeBlock(buff []byte) (Block, error) { blockLen := binary.BigEndian.Uint32(buff[:4]) - bh := BlockHeader{} - if err := bh.FromBytes(buff[4+blockLen:]); err != nil { + var bh BlockHeader + if err := bh.UnmarshalCanoto(buff[4+blockLen:]); err != nil { return nil, err } diff --git a/global.go b/global.go deleted file mode 100644 index 0864d681..00000000 --- a/global.go +++ /dev/null @@ -1,19 +0,0 @@ -// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package simplex - -import ( - "bytes" - "encoding/hex" -) - -type NodeID []byte - -func (node NodeID) String() string { - return hex.EncodeToString(node) -} - -func (node NodeID) Equals(otherNode NodeID) bool { - return bytes.Equal(node, otherNode) -} diff --git a/go.mod b/go.mod index d4b198a6..29871b65 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,8 @@ module simplex go 1.23 require ( - github.com/stretchr/testify v1.9.0 + github.com/StephenButtolph/canoto v0.10.0 + github.com/stretchr/testify v1.10.0 go.uber.org/zap v1.26.0 ) diff --git a/go.sum b/go.sum index 56d9d664..83930439 100644 --- a/go.sum +++ b/go.sum @@ -1,9 +1,15 @@ +github.com/StephenButtolph/canoto v0.10.0 h1:KdW85TYQXH+gwR8vOxfOUf28TRpkLU+X06Kycg1IR7s= +github.com/StephenButtolph/canoto v0.10.0/go.mod h1:MxppdgKRApRBvIg4ZgO2e14m/NSBjFMuydy97OB/gYY= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/sanity-io/litter v1.5.1 h1:dwnrSypP6q56o3lFxTU+t2fwQ9A+U5qrXVO4Qg9KwVU= +github.com/sanity-io/litter v1.5.1/go.mod h1:5Z71SvaYy5kcGtyglXOC9rrUi3c1E8CamFWjQsazTh0= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/thepudds/fzgen v0.4.3 h1:srUP/34BulQaEwPP/uHZkdjUcUjIzL7Jkf4CBVryiP8= +github.com/thepudds/fzgen v0.4.3/go.mod h1:BhhwtRhzgvLWAjjcHDJ9pEiLD2Z9hrVIFjBCHJ//zJ4= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= diff --git a/metadata.go b/metadata.go deleted file mode 100644 index 691127b5..00000000 --- a/metadata.go +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package simplex - -import ( - "bytes" - "encoding/binary" - "fmt" -) - -const ( - metadataVersionLen = 1 - metadataEpochLen = 8 - metadataRoundLen = 8 - metadataSeqLen = 8 - metadataPrevLen = 32 - metadataDigestLen = 32 - - metadataLen = metadataVersionLen + metadataDigestLen + metadataEpochLen + metadataRoundLen + metadataSeqLen + metadataPrevLen -) - -const ( - digestFormatSize = 10 -) - -// ProtocolMetadata encodes information about the protocol state at a given point in time. -type ProtocolMetadata struct { - // Version defines the version of the protocol this block was created with. - Version uint8 - // Epoch returns the epoch in which the block was proposed - Epoch uint64 - // Round returns the round number in which the block was proposed. - // Can also be an empty block. - Round uint64 - // Seq is the order of the block among all blocks in the blockchain. - // Cannot correspond to an empty block. - Seq uint64 - // Prev returns the digest of the previous data block - Prev Digest -} - -// BlockHeader encodes a succinct and collision-free representation of a block. -// It's included in votes and finalizations in order to convey which block is voted on, -// or which block is finalized. -type BlockHeader struct { - ProtocolMetadata - // Digest returns a collision resistant short representation of the block's bytes - Digest Digest -} - -type Digest [metadataDigestLen]byte - -func (d Digest) String() string { - return fmt.Sprintf("%x...", (d)[:digestFormatSize]) -} - -func (bh *BlockHeader) Equals(other *BlockHeader) bool { - return bytes.Equal(bh.Digest[:], other.Digest[:]) && - bytes.Equal(bh.Prev[:], other.Prev[:]) && bh.Epoch == other.Epoch && - bh.Round == other.Round && bh.Seq == other.Seq && bh.Version == other.Version -} - -func (bh *BlockHeader) Bytes() []byte { - // Sanity check: check that digest and prev are 32 bytes - - if len(bh.Digest) != metadataDigestLen { - panic(fmt.Sprintf("digest is %d bytes, expected %d", len(bh.Digest), metadataDigestLen)) - } - - // Prev block's digest can be nil, or 32 bytes - if len(bh.Prev) != 0 && len(bh.Prev) != metadataPrevLen { - panic(fmt.Sprintf("digest is %d bytes, expected %d", len(bh.Prev), metadataPrevLen)) - } - - buff := make([]byte, metadataLen) - var pos int - - buff[pos] = bh.Version - pos++ - - copy(buff[pos:], bh.Digest[:]) - pos += metadataDigestLen - - binary.BigEndian.PutUint64(buff[pos:], bh.Epoch) - pos += metadataEpochLen - - binary.BigEndian.PutUint64(buff[pos:], bh.Round) - pos += metadataRoundLen - - binary.BigEndian.PutUint64(buff[pos:], bh.Seq) - pos += metadataSeqLen - - copy(buff[pos:], bh.Prev[:]) - - return buff -} - -func (bh *BlockHeader) FromBytes(buff []byte) error { - if len(buff) != metadataLen { - return fmt.Errorf("invalid buffer length %d, expected %d", len(buff), metadataLen) - } - - var pos int - - bh.Version = buff[pos] - pos++ - - copy(bh.Digest[:], buff[pos:pos+metadataDigestLen]) - pos += metadataDigestLen - - bh.Epoch = binary.BigEndian.Uint64(buff[pos:]) - pos += metadataEpochLen - - bh.Round = binary.BigEndian.Uint64(buff[pos:]) - pos += metadataRoundLen - - bh.Seq = binary.BigEndian.Uint64(buff[pos:]) - pos += metadataSeqLen - - copy(bh.Prev[:], buff[pos:pos+metadataPrevLen]) - - return nil -} diff --git a/metadata_test.go b/metadata_test.go index 6d8968f8..b09d1340 100644 --- a/metadata_test.go +++ b/metadata_test.go @@ -6,13 +6,16 @@ package simplex import ( "crypto/rand" "crypto/sha256" - "github.com/stretchr/testify/require" "testing" + + "github.com/stretchr/testify/require" ) func TestMetadata(t *testing.T) { - var prev [metadataPrevLen]byte - var digest [metadataDigestLen]byte + var ( + prev Digest + digest Digest + ) _, err := rand.Read(prev[:]) require.NoError(t, err) @@ -22,18 +25,17 @@ func TestMetadata(t *testing.T) { bh := BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: 1, - Round: 2, - Seq: 3, - Epoch: 4, - Prev: prev, + Round: 2, + Seq: 3, + Epoch: 4, + Prev: prev, }, Digest: digest, } var bh2 BlockHeader - require.NoError(t, bh2.FromBytes(bh.Bytes())) - require.Equal(t, bh, bh2) + require.NoError(t, bh2.UnmarshalCanoto(bh.MarshalCanoto())) + require.True(t, bh.Equals(&bh2)) } func FuzzMetadata(f *testing.F) { @@ -44,17 +46,16 @@ func FuzzMetadata(f *testing.F) { bh := BlockHeader{ ProtocolMetadata: ProtocolMetadata{ - Version: version, - Round: round, - Seq: seq, - Epoch: epoch, - Prev: prev, + Round: round, + Seq: seq, + Epoch: epoch, + Prev: prev, }, Digest: digest, } var bh2 BlockHeader - require.NoError(t, bh2.FromBytes(bh.Bytes())) - require.Equal(t, bh, bh2) + require.NoError(t, bh2.UnmarshalCanoto(bh.MarshalCanoto())) + require.True(t, bh.Equals(&bh2)) }) } diff --git a/msg.go b/msg.go index 3e2bf99c..28ba72ba 100644 --- a/msg.go +++ b/msg.go @@ -3,32 +3,21 @@ package simplex -import "encoding/asn1" - -type Message struct { - BlockMessage *BlockMessage - VoteMessage *Vote - Notarization *Notarization - Finalization *Finalization - FinalizationCertificate *FinalizationCertificate -} +const ( + voteContext = "ToBeSignedVote" + finalizationContext = "ToBeSignedFinalization" +) type ToBeSignedVote struct { BlockHeader } func (v *ToBeSignedVote) Sign(signer Signer) ([]byte, error) { - context := "ToBeSignedVote" - msg := v.Bytes() - - return signContext(signer, msg, context) + return signContext(signer, v.MarshalCanoto(), voteContext) } func (v *ToBeSignedVote) Verify(signature []byte, verifier SignatureVerifier, signers NodeID) error { - context := "ToBeSignedVote" - msg := v.Bytes() - - return verifyContext(signature, verifier, msg, context, signers) + return verifyContext(signature, verifier, v.MarshalCanoto(), voteContext, signers) } type ToBeSignedFinalization struct { @@ -36,94 +25,44 @@ type ToBeSignedFinalization struct { } func (f *ToBeSignedFinalization) Sign(signer Signer) ([]byte, error) { - context := "ToBeSignedFinalization" - msg := f.Bytes() - - return signContext(signer, msg, context) + return signContext(signer, f.MarshalCanoto(), finalizationContext) } func (f *ToBeSignedFinalization) Verify(signature []byte, verifier SignatureVerifier, signers NodeID) error { - context := "ToBeSignedFinalization" - msg := f.Bytes() - - return verifyContext(signature, verifier, msg, context, signers) + return verifyContext(signature, verifier, f.MarshalCanoto(), finalizationContext, signers) } func signContext(signer Signer, msg []byte, context string) ([]byte, error) { - sm := SignedMessage{Payload: msg, Context: context} - toBeSigned, err := asn1.Marshal(sm) - if err != nil { - return nil, err + sm := SignedMessage{ + Payload: msg, + Context: context, } + toBeSigned := sm.MarshalCanoto() return signer.Sign(toBeSigned) } func verifyContext(signature []byte, verifier SignatureVerifier, msg []byte, context string, signers NodeID) error { - sm := SignedMessage{Payload: msg, Context: context} - toBeSigned, err := asn1.Marshal(sm) - if err != nil { - return err + sm := SignedMessage{ + Payload: msg, + Context: context, } + toBeSigned := sm.MarshalCanoto() return verifier.Verify(toBeSigned, signature, signers) } func verifyContextQC(qc QuorumCertificate, msg []byte, context string) error { - sm := SignedMessage{Payload: msg, Context: context} - toBeSigned, err := asn1.Marshal(sm) - if err != nil { - return err + sm := SignedMessage{ + Payload: msg, + Context: context, } - + toBeSigned := sm.MarshalCanoto() return qc.Verify(toBeSigned) } -type Vote struct { - Vote ToBeSignedVote - Signature Signature -} - -type Finalization struct { - Finalization ToBeSignedFinalization - Signature Signature -} - -type FinalizationCertificate struct { - Finalization ToBeSignedFinalization - QC QuorumCertificate -} - func (fc *FinalizationCertificate) Verify() error { - context := "ToBeSignedFinalization" - return verifyContextQC(fc.QC, fc.Finalization.Bytes(), context) -} - -type Notarization struct { - Vote ToBeSignedVote - QC QuorumCertificate + return verifyContextQC(fc.QC, fc.Finalization.MarshalCanoto(), finalizationContext) } func (n *Notarization) Verify() error { - context := "ToBeSignedVote" - return verifyContextQC(n.QC, n.Vote.Bytes(), context) -} - -type BlockMessage struct { - Block Block - Vote Vote -} - -type SignedMessage struct { - Payload []byte - Context string -} - -// QuorumCertificate is equivalent to a collection of signatures from a quorum of nodes, -type QuorumCertificate interface { - // Signers returns who participated in creating this QuorumCertificate. - Signers() []NodeID - // Verify checks whether the nodes participated in creating this QuorumCertificate, - // signed the given message. - Verify(msg []byte) error - // Bytes returns a raw representation of the given QuorumCertificate. - Bytes() []byte + return verifyContextQC(n.QC, n.Vote.MarshalCanoto(), voteContext) } diff --git a/record/consts.go b/record/consts.go deleted file mode 100644 index a529fd10..00000000 --- a/record/consts.go +++ /dev/null @@ -1,11 +0,0 @@ -// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package record - -const ( - UndefinedRecordType uint16 = iota - BlockRecordType - NotarizationRecordType - FinalizationRecordType -) diff --git a/types.canoto.go b/types.canoto.go new file mode 100644 index 00000000..f6af4357 --- /dev/null +++ b/types.canoto.go @@ -0,0 +1,2786 @@ +// Code generated by canoto. DO NOT EDIT. +// versions: +// canoto v0.10.0 +// source: types.go + +package simplex + +import ( + "io" + "sync/atomic" + "unicode/utf8" + + "github.com/StephenButtolph/canoto" +) + +// Ensure that unused imports do not error +var ( + _ atomic.Int64 + + _ = io.ErrUnexpectedEOF + _ = utf8.ValidString +) + +const ( + canoto__Record__Block__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Record__Notarization__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Record__Finalization__tag = "\x1a" // canoto.Tag(3, canoto.Len) +) + +type canotoData_Record struct { + size int + + recordOneOf uint32 +} + +// MakeCanoto creates a new empty value. +func (*Record) MakeCanoto() *Record { + return new(Record) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Record) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Record) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.recordOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.recordOneOf = 1 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Block = canoto.MakePointer(c.Block) + err = (c.Block).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.recordOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.recordOneOf = 2 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Notarization = canoto.MakePointer(c.Notarization) + err = (c.Notarization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 3: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.recordOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.recordOneOf = 3 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Finalization = canoto.MakePointer(c.Finalization) + err = (c.Finalization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Record) ValidCanoto() bool { + if c == nil { + return true + } + var ( + recordOneOf uint32 + ) + if c.Block != nil { + (c.Block).CalculateCanotoCache() + if (c.Block).CachedCanotoSize() != 0 { + if recordOneOf != 0 { + return false + } + recordOneOf = 1 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if (c.Notarization).CachedCanotoSize() != 0 { + if recordOneOf != 0 { + return false + } + recordOneOf = 2 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if (c.Finalization).CachedCanotoSize() != 0 { + if recordOneOf != 0 { + return false + } + recordOneOf = 3 + } + } + if c.Block != nil && !(c.Block).ValidCanoto() { + return false + } + if c.Notarization != nil && !(c.Notarization).ValidCanoto() { + return false + } + if c.Finalization != nil && !(c.Finalization).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Record) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.recordOneOf = 0 + c.canotoData.size = 0 + if c.Block != nil { + (c.Block).CalculateCanotoCache() + if fieldSize := (c.Block).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Record__Block__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.recordOneOf = 1 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Record__Notarization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.recordOneOf = 2 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Record__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.recordOneOf = 3 + } + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Record) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// CachedWhichOneOfrecord returns the previously calculated field number used +// to represent record. +// +// This field is cached by UnmarshalCanoto, UnmarshalCanotoFrom, and +// CalculateCanotoCache. +// +// If the field has not yet been cached, it will return 0. +// +// If the struct has been modified since the field was last cached, the returned +// field number may be incorrect. +func (c *Record) CachedWhichOneOfrecord() uint32 { + return c.canotoData.recordOneOf +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Record) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Record) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if c.Block != nil { + if fieldSize := (c.Block).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Record__Block__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Block).MarshalCanotoInto(w) + } + } + if c.Notarization != nil { + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Record__Notarization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Notarization).MarshalCanotoInto(w) + } + } + if c.Finalization != nil { + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Record__Finalization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Finalization).MarshalCanotoInto(w) + } + } + return w +} + +const ( + canoto__QuorumRecord__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__QuorumRecord__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_QuorumRecord struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*QuorumRecord) MakeCanoto() *QuorumRecord { + return new(QuorumRecord) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *QuorumRecord) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *QuorumRecord) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Header).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.QC); err != nil { + return err + } + if len(c.QC) == 0 { + return canoto.ErrZeroValue + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *QuorumRecord) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Header).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *QuorumRecord) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Header).CalculateCanotoCache() + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__QuorumRecord__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + if len(c.QC) != 0 { + c.canotoData.size += len(canoto__QuorumRecord__QC__tag) + canoto.SizeBytes(c.QC) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *QuorumRecord) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *QuorumRecord) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *QuorumRecord) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__QuorumRecord__Header__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Header).MarshalCanotoInto(w) + } + if len(c.QC) != 0 { + canoto.Append(&w, canoto__QuorumRecord__QC__tag) + canoto.AppendBytes(&w, c.QC) + } + return w +} + +const ( + canoto__BlockRecord__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__BlockRecord__Payload__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_BlockRecord struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*BlockRecord) MakeCanoto() *BlockRecord { + return new(BlockRecord) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *BlockRecord) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *BlockRecord) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Header).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.Payload); err != nil { + return err + } + if len(c.Payload) == 0 { + return canoto.ErrZeroValue + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *BlockRecord) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Header).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *BlockRecord) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Header).CalculateCanotoCache() + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__BlockRecord__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + if len(c.Payload) != 0 { + c.canotoData.size += len(canoto__BlockRecord__Payload__tag) + canoto.SizeBytes(c.Payload) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *BlockRecord) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockRecord) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockRecord) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__BlockRecord__Header__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Header).MarshalCanotoInto(w) + } + if len(c.Payload) != 0 { + canoto.Append(&w, canoto__BlockRecord__Payload__tag) + canoto.AppendBytes(&w, c.Payload) + } + return w +} + +const ( + canoto__Message__Proposal__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Message__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Message__Notarization__tag = "\x1a" // canoto.Tag(3, canoto.Len) + canoto__Message__Finalize__tag = "\x22" // canoto.Tag(4, canoto.Len) + canoto__Message__Finalization__tag = "\x2a" // canoto.Tag(5, canoto.Len) +) + +type canotoData_Message struct { + size int + + messageOneOf uint32 +} + +// MakeCanoto creates a new empty value. +func (*Message) MakeCanoto() *Message { + return new(Message) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Message) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Message) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 1 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Proposal = canoto.MakePointer(c.Proposal) + err = (c.Proposal).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 2 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Vote = canoto.MakePointer(c.Vote) + err = (c.Vote).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 3: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 3 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Notarization = canoto.MakePointer(c.Notarization) + err = (c.Notarization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 4: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 4 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Finalize = canoto.MakePointer(c.Finalize) + err = (c.Finalize).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 5: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 5 + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + c.Finalization = canoto.MakePointer(c.Finalization) + err = (c.Finalization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Message) ValidCanoto() bool { + if c == nil { + return true + } + var ( + messageOneOf uint32 + ) + if c.Proposal != nil { + (c.Proposal).CalculateCanotoCache() + if (c.Proposal).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 1 + } + } + if c.Vote != nil { + (c.Vote).CalculateCanotoCache() + if (c.Vote).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 2 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if (c.Notarization).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 3 + } + } + if c.Finalize != nil { + (c.Finalize).CalculateCanotoCache() + if (c.Finalize).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 4 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if (c.Finalization).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 5 + } + } + if c.Proposal != nil && !(c.Proposal).ValidCanoto() { + return false + } + if c.Vote != nil && !(c.Vote).ValidCanoto() { + return false + } + if c.Notarization != nil && !(c.Notarization).ValidCanoto() { + return false + } + if c.Finalize != nil && !(c.Finalize).ValidCanoto() { + return false + } + if c.Finalization != nil && !(c.Finalization).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Message) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.messageOneOf = 0 + c.canotoData.size = 0 + if c.Proposal != nil { + (c.Proposal).CalculateCanotoCache() + if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Proposal__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 1 + } + } + if c.Vote != nil { + (c.Vote).CalculateCanotoCache() + if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 2 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Notarization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 3 + } + } + if c.Finalize != nil { + (c.Finalize).CalculateCanotoCache() + if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Finalize__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 4 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 5 + } + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Message) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// CachedWhichOneOfmessage returns the previously calculated field number used +// to represent message. +// +// This field is cached by UnmarshalCanoto, UnmarshalCanotoFrom, and +// CalculateCanotoCache. +// +// If the field has not yet been cached, it will return 0. +// +// If the struct has been modified since the field was last cached, the returned +// field number may be incorrect. +func (c *Message) CachedWhichOneOfmessage() uint32 { + return c.canotoData.messageOneOf +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Message) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Message) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if c.Proposal != nil { + if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Proposal__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Proposal).MarshalCanotoInto(w) + } + } + if c.Vote != nil { + if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Vote__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Vote).MarshalCanotoInto(w) + } + } + if c.Notarization != nil { + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Notarization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Notarization).MarshalCanotoInto(w) + } + } + if c.Finalize != nil { + if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Finalize__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Finalize).MarshalCanotoInto(w) + } + } + if c.Finalization != nil { + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Finalization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Finalization).MarshalCanotoInto(w) + } + } + return w +} + +const ( + canoto__BlockMessage__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_BlockMessage struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*BlockMessage) MakeCanoto() *BlockMessage { + return new(BlockMessage) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *BlockMessage) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *BlockMessage) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Vote).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *BlockMessage) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Vote).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *BlockMessage) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Vote).CalculateCanotoCache() + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__BlockMessage__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *BlockMessage) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockMessage) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockMessage) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__BlockMessage__Vote__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Vote).MarshalCanotoInto(w) + } + return w +} + +const ( + canoto__Vote__Vote__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Vote__Signature__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_Vote struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*Vote) MakeCanoto() *Vote { + return new(Vote) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Vote) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Vote).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Signature).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Vote) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Vote).ValidCanoto() { + return false + } + if !(&c.Signature).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Vote) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Vote).CalculateCanotoCache() + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Vote__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + (&c.Signature).CalculateCanotoCache() + if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Vote__Signature__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Vote) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Vote) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Vote) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Vote__Vote__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Vote).MarshalCanotoInto(w) + } + if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Vote__Signature__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Signature).MarshalCanotoInto(w) + } + return w +} + +const ( + canoto__Notarization__Vote__tag = "\x0a" // canoto.Tag(1, canoto.Len) +) + +type canotoData_Notarization struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*Notarization) MakeCanoto() *Notarization { + return new(Notarization) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Notarization) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Vote).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Notarization) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Vote).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Notarization) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Vote).CalculateCanotoCache() + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Notarization__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Notarization) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Notarization) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Notarization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Notarization__Vote__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Vote).MarshalCanotoInto(w) + } + return w +} + +const ( + canoto__Finalization__Finalization__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Finalization__Signature__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_Finalization struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*Finalization) MakeCanoto() *Finalization { + return new(Finalization) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Finalization) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Finalization) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Finalization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Signature).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Finalization) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Finalization).ValidCanoto() { + return false + } + if !(&c.Signature).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Finalization) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Finalization).CalculateCanotoCache() + if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Finalization__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + (&c.Signature).CalculateCanotoCache() + if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Finalization__Signature__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Finalization) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Finalization) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Finalization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Finalization__Finalization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Finalization).MarshalCanotoInto(w) + } + if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Finalization__Signature__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Signature).MarshalCanotoInto(w) + } + return w +} + +const ( + canoto__FinalizationCertificate__Finalization__tag = "\x0a" // canoto.Tag(1, canoto.Len) +) + +type canotoData_FinalizationCertificate struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*FinalizationCertificate) MakeCanoto() *FinalizationCertificate { + return new(FinalizationCertificate) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *FinalizationCertificate) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *FinalizationCertificate) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.Finalization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *FinalizationCertificate) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.Finalization).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *FinalizationCertificate) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.Finalization).CalculateCanotoCache() + if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__FinalizationCertificate__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *FinalizationCertificate) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *FinalizationCertificate) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *FinalizationCertificate) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__FinalizationCertificate__Finalization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.Finalization).MarshalCanotoInto(w) + } + return w +} + +const ( + canoto__SignedMessage__Payload__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__SignedMessage__Context__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_SignedMessage struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*SignedMessage) MakeCanoto() *SignedMessage { + return new(SignedMessage) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *SignedMessage) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *SignedMessage) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.Payload); err != nil { + return err + } + if len(c.Payload) == 0 { + return canoto.ErrZeroValue + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadString(&r, &c.Context); err != nil { + return err + } + if len(c.Context) == 0 { + return canoto.ErrZeroValue + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *SignedMessage) ValidCanoto() bool { + if c == nil { + return true + } + if !utf8.ValidString(string(c.Context)) { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *SignedMessage) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + if len(c.Payload) != 0 { + c.canotoData.size += len(canoto__SignedMessage__Payload__tag) + canoto.SizeBytes(c.Payload) + } + if len(c.Context) != 0 { + c.canotoData.size += len(canoto__SignedMessage__Context__tag) + canoto.SizeBytes(c.Context) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *SignedMessage) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *SignedMessage) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *SignedMessage) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if len(c.Payload) != 0 { + canoto.Append(&w, canoto__SignedMessage__Payload__tag) + canoto.AppendBytes(&w, c.Payload) + } + if len(c.Context) != 0 { + canoto.Append(&w, canoto__SignedMessage__Context__tag) + canoto.AppendBytes(&w, c.Context) + } + return w +} + +const ( + canoto__Signature__Signer__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Signature__Value__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_Signature struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*Signature) MakeCanoto() *Signature { + return new(Signature) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *Signature) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *Signature) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.Signer); err != nil { + return err + } + if len(c.Signer) == 0 { + return canoto.ErrZeroValue + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.Value); err != nil { + return err + } + if len(c.Value) == 0 { + return canoto.ErrZeroValue + } + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *Signature) ValidCanoto() bool { + if c == nil { + return true + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *Signature) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + if len(c.Signer) != 0 { + c.canotoData.size += len(canoto__Signature__Signer__tag) + canoto.SizeBytes(c.Signer) + } + if len(c.Value) != 0 { + c.canotoData.size += len(canoto__Signature__Value__tag) + canoto.SizeBytes(c.Value) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *Signature) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Signature) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *Signature) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if len(c.Signer) != 0 { + canoto.Append(&w, canoto__Signature__Signer__tag) + canoto.AppendBytes(&w, c.Signer) + } + if len(c.Value) != 0 { + canoto.Append(&w, canoto__Signature__Value__tag) + canoto.AppendBytes(&w, c.Value) + } + return w +} + +const ( + canoto__BlockHeader__ProtocolMetadata__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__BlockHeader__Digest__tag = "\x12" // canoto.Tag(2, canoto.Len) +) + +type canotoData_BlockHeader struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*BlockHeader) MakeCanoto() *BlockHeader { + return new(BlockHeader) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *BlockHeader) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *BlockHeader) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { + return err + } + if len(msgBytes) == 0 { + return canoto.ErrZeroValue + } + + remainingBytes := r.B + r.B = msgBytes + err = (&c.ProtocolMetadata).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + var length int64 + if err := canoto.ReadInt(&r, &length); err != nil { + return err + } + + const ( + expectedLength = len(c.Digest) + expectedLengthInt64 = int64(expectedLength) + ) + if length != expectedLengthInt64 { + return canoto.ErrInvalidLength + } + if expectedLength > len(r.B) { + return io.ErrUnexpectedEOF + } + + copy(c.Digest[:], r.B) + if canoto.IsZero(c.Digest) { + return canoto.ErrZeroValue + } + r.B = r.B[expectedLength:] + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *BlockHeader) ValidCanoto() bool { + if c == nil { + return true + } + if !(&c.ProtocolMetadata).ValidCanoto() { + return false + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *BlockHeader) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + (&c.ProtocolMetadata).CalculateCanotoCache() + if fieldSize := (&c.ProtocolMetadata).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__BlockHeader__ProtocolMetadata__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + if !canoto.IsZero(c.Digest) { + c.canotoData.size += len(canoto__BlockHeader__Digest__tag) + canoto.SizeBytes(c.Digest[:]) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *BlockHeader) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockHeader) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *BlockHeader) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if fieldSize := (&c.ProtocolMetadata).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__BlockHeader__ProtocolMetadata__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (&c.ProtocolMetadata).MarshalCanotoInto(w) + } + if !canoto.IsZero(c.Digest) { + canoto.Append(&w, canoto__BlockHeader__Digest__tag) + canoto.AppendBytes(&w, c.Digest[:]) + } + return w +} + +const ( + canoto__ProtocolMetadata__Epoch__tag = "\x08" // canoto.Tag(1, canoto.Varint) + canoto__ProtocolMetadata__Round__tag = "\x10" // canoto.Tag(2, canoto.Varint) + canoto__ProtocolMetadata__Seq__tag = "\x18" // canoto.Tag(3, canoto.Varint) + canoto__ProtocolMetadata__Prev__tag = "\x22" // canoto.Tag(4, canoto.Len) +) + +type canotoData_ProtocolMetadata struct { + size int +} + +// MakeCanoto creates a new empty value. +func (*ProtocolMetadata) MakeCanoto() *ProtocolMetadata { + return new(ProtocolMetadata) +} + +// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +func (c *ProtocolMetadata) UnmarshalCanoto(bytes []byte) error { + r := canoto.Reader{ + B: bytes, + } + return c.UnmarshalCanotoFrom(r) +} + +// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users +// should just use UnmarshalCanoto. +// +// OneOf fields are cached during the unmarshaling process. +// +// The struct is not cleared before unmarshaling, any fields not present in the +// bytes will retain their previous values. If a OneOf field was previously +// cached as being set, attempting to unmarshal that OneOf again will return +// canoto.ErrDuplicateOneOf. +// +// This function enables configuration of reader options. +func (c *ProtocolMetadata) UnmarshalCanotoFrom(r canoto.Reader) error { + var minField uint32 + for canoto.HasNext(&r) { + field, wireType, err := canoto.ReadTag(&r) + if err != nil { + return err + } + if field < minField { + return canoto.ErrInvalidFieldOrder + } + + switch field { + case 1: + if wireType != canoto.Varint { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadInt(&r, &c.Epoch); err != nil { + return err + } + if canoto.IsZero(c.Epoch) { + return canoto.ErrZeroValue + } + case 2: + if wireType != canoto.Varint { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadInt(&r, &c.Round); err != nil { + return err + } + if canoto.IsZero(c.Round) { + return canoto.ErrZeroValue + } + case 3: + if wireType != canoto.Varint { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadInt(&r, &c.Seq); err != nil { + return err + } + if canoto.IsZero(c.Seq) { + return canoto.ErrZeroValue + } + case 4: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + var length int64 + if err := canoto.ReadInt(&r, &length); err != nil { + return err + } + + const ( + expectedLength = len(c.Prev) + expectedLengthInt64 = int64(expectedLength) + ) + if length != expectedLengthInt64 { + return canoto.ErrInvalidLength + } + if expectedLength > len(r.B) { + return io.ErrUnexpectedEOF + } + + copy(c.Prev[:], r.B) + if canoto.IsZero(c.Prev) { + return canoto.ErrZeroValue + } + r.B = r.B[expectedLength:] + default: + return canoto.ErrUnknownField + } + + minField = field + 1 + } + return nil +} + +// ValidCanoto validates that the struct can be correctly marshaled into the +// Canoto format. +// +// Specifically, ValidCanoto ensures: +// 1. All OneOfs are specified at most once. +// 2. All strings are valid utf-8. +// 3. All custom fields are ValidCanoto. +func (c *ProtocolMetadata) ValidCanoto() bool { + if c == nil { + return true + } + return true +} + +// CalculateCanotoCache populates size and OneOf caches based on the current +// values in the struct. +// +// It is not safe to call this function concurrently. +func (c *ProtocolMetadata) CalculateCanotoCache() { + if c == nil { + return + } + c.canotoData.size = 0 + if !canoto.IsZero(c.Epoch) { + c.canotoData.size += len(canoto__ProtocolMetadata__Epoch__tag) + canoto.SizeInt(c.Epoch) + } + if !canoto.IsZero(c.Round) { + c.canotoData.size += len(canoto__ProtocolMetadata__Round__tag) + canoto.SizeInt(c.Round) + } + if !canoto.IsZero(c.Seq) { + c.canotoData.size += len(canoto__ProtocolMetadata__Seq__tag) + canoto.SizeInt(c.Seq) + } + if !canoto.IsZero(c.Prev) { + c.canotoData.size += len(canoto__ProtocolMetadata__Prev__tag) + canoto.SizeBytes(c.Prev[:]) + } +} + +// CachedCanotoSize returns the previously calculated size of the Canoto +// representation from CalculateCanotoCache. +// +// If CalculateCanotoCache has not yet been called, it will return 0. +// +// If the struct has been modified since the last call to CalculateCanotoCache, +// the returned size may be incorrect. +func (c *ProtocolMetadata) CachedCanotoSize() int { + if c == nil { + return 0 + } + return c.canotoData.size +} + +// MarshalCanoto returns the Canoto representation of this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *ProtocolMetadata) MarshalCanoto() []byte { + c.CalculateCanotoCache() + w := canoto.Writer{ + B: make([]byte, 0, c.CachedCanotoSize()), + } + w = c.MarshalCanotoInto(w) + return w.B +} + +// MarshalCanotoInto writes the struct into a canoto.Writer and returns the +// resulting canoto.Writer. Most users should just use MarshalCanoto. +// +// It is assumed that CalculateCanotoCache has been called since the last +// modification to this struct. +// +// It is assumed that this struct is ValidCanoto. +// +// It is not safe to call this function concurrently. +func (c *ProtocolMetadata) MarshalCanotoInto(w canoto.Writer) canoto.Writer { + if c == nil { + return w + } + if !canoto.IsZero(c.Epoch) { + canoto.Append(&w, canoto__ProtocolMetadata__Epoch__tag) + canoto.AppendInt(&w, c.Epoch) + } + if !canoto.IsZero(c.Round) { + canoto.Append(&w, canoto__ProtocolMetadata__Round__tag) + canoto.AppendInt(&w, c.Round) + } + if !canoto.IsZero(c.Seq) { + canoto.Append(&w, canoto__ProtocolMetadata__Seq__tag) + canoto.AppendInt(&w, c.Seq) + } + if !canoto.IsZero(c.Prev) { + canoto.Append(&w, canoto__ProtocolMetadata__Prev__tag) + canoto.AppendBytes(&w, c.Prev[:]) + } + return w +} diff --git a/types.go b/types.go new file mode 100644 index 00000000..065c4acb --- /dev/null +++ b/types.go @@ -0,0 +1,150 @@ +// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false $GOFILE + +package simplex + +import ( + "bytes" + "encoding/hex" + "fmt" +) + +const ( + digestLen = 32 + digestFormatSize = 10 +) + +type ( + Record struct { + Block *BlockRecord `canoto:"pointer,1,record"` + Notarization *QuorumRecord `canoto:"pointer,2,record"` + Finalization *QuorumRecord `canoto:"pointer,3,record"` + + canotoData canotoData_Record + } + QuorumRecord struct { + Header BlockHeader `canoto:"value,1"` + QC []byte `canoto:"bytes,2"` + + canotoData canotoData_QuorumRecord + } + BlockRecord struct { + // TODO: Do we need to include the Digest? Or should we just include the + // ProtocolMetadata? + Header BlockHeader `canoto:"value,1"` + Payload []byte `canoto:"bytes,2"` + + canotoData canotoData_BlockRecord + } + + Message struct { + Proposal *BlockMessage `canoto:"pointer,1,message"` + Vote *Vote `canoto:"pointer,2,message"` + Notarization *Notarization `canoto:"pointer,3,message"` + Finalize *Finalization `canoto:"pointer,4,message"` + Finalization *FinalizationCertificate `canoto:"pointer,5,message"` + + canotoData canotoData_Message + } + BlockMessage struct { + Block Block // `canoto:"bytes,1"` + Vote Vote `canoto:"value,2"` + + canotoData canotoData_BlockMessage + } + Vote struct { + Vote ToBeSignedVote `canoto:"value,1"` + Signature Signature `canoto:"value,2"` + + canotoData canotoData_Vote + } + Notarization struct { + Vote ToBeSignedVote `canoto:"value,1"` + QC QuorumCertificate // `canoto:"bytes,2"` + + canotoData canotoData_Notarization + } + Finalization struct { + Finalization ToBeSignedFinalization `canoto:"value,1"` + Signature Signature `canoto:"value,2"` + + canotoData canotoData_Finalization + } + FinalizationCertificate struct { + Finalization ToBeSignedFinalization `canoto:"value,1"` + QC QuorumCertificate // `canoto:"bytes,2"` + + canotoData canotoData_FinalizationCertificate + } + + SignedMessage struct { + Payload []byte `canoto:"bytes,1"` + Context string `canoto:"string,2"` + + canotoData canotoData_SignedMessage + } + + // Signature encodes a signature and the node that signed it, without the + // message it was signed on. + Signature struct { + // Signer is the NodeID of the creator of the signature. + Signer NodeID `canoto:"bytes,1"` + // Value is the byte representation of the signature. + Value []byte `canoto:"bytes,2"` + + canotoData canotoData_Signature + } + + // BlockHeader encodes a succinct and collision-free representation of a block. + // It's included in votes and finalizations in order to convey which block is voted on, + // or which block is finalized. + BlockHeader struct { + ProtocolMetadata `canoto:"value,1"` + // Digest returns a collision resistant short representation of the block's bytes + Digest Digest `canoto:"fixed bytes,2"` + + canotoData canotoData_BlockHeader + } + + // ProtocolMetadata encodes information about the protocol state at a given + // point in time. + ProtocolMetadata struct { + // Epoch returns the epoch in which the block was proposed + Epoch uint64 `canoto:"int,1"` + // Round returns the round number in which the block was proposed. + // Can also be an empty block. + Round uint64 `canoto:"int,2"` + // Seq is the order of the block among all blocks in the blockchain. + // Cannot correspond to an empty block. + Seq uint64 `canoto:"int,3"` + // Prev returns the digest of the previous data block + Prev Digest `canoto:"fixed bytes,4"` + + canotoData canotoData_ProtocolMetadata + } + + Digest [digestLen]byte + NodeID []byte +) + +func (b *BlockHeader) Equals(o *BlockHeader) bool { + return b.Digest == o.Digest && b.ProtocolMetadata.Equals(&o.ProtocolMetadata) +} + +func (p *ProtocolMetadata) Equals(o *ProtocolMetadata) bool { + return p.Epoch == o.Epoch && p.Round == o.Round && p.Seq == o.Seq && p.Prev == o.Prev +} + +func (d Digest) String() string { + return fmt.Sprintf("%x...", (d)[:digestFormatSize]) +} + +func (n NodeID) String() string { + return hex.EncodeToString(n) +} + +func (n NodeID) Equals(o NodeID) bool { + return bytes.Equal(n, o) +} From 36db7d5c8a9575bfda3330ebf1bb4b325bcf7e73 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 08:09:53 -0500 Subject: [PATCH 2/9] wip --- epoch.go | 16 ++++++++-------- msg.go | 8 ++++++-- types.canoto.go | 19 +++++++++++++++++++ types.go | 4 ++-- 4 files changed, 35 insertions(+), 12 deletions(-) diff --git a/epoch.go b/epoch.go index a81a8d50..4d8f62d4 100644 --- a/epoch.go +++ b/epoch.go @@ -489,16 +489,16 @@ func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, dige signatures = append(signatures, vote.Signature) } - var notarization Notarization - var err error - notarization.Vote = vote - notarization.QC, err = e.SignatureAggregator.Aggregate(signatures) + qc, err := e.SignatureAggregator.Aggregate(signatures) if err != nil { return fmt.Errorf("could not aggregate signatures for notarization: %w", err) } - err = e.storeNotarization(notarization) - if err != nil { + notarization := Notarization{ + Vote: vote, + QC: qc.Bytes(), + } + if err := e.storeNotarization(notarization); err != nil { return err } @@ -509,7 +509,7 @@ func (e *Epoch) persistNotarization(notarization Notarization, vote ToBeSignedVo record := Record{ Notarization: &QuorumRecord{ Header: vote.BlockHeader, - QC: notarization.QC.Bytes(), + QC: notarization.QC, }, } recordBytes := record.MarshalCanoto() @@ -573,7 +573,7 @@ func (e *Epoch) handleNotarizationMessage(message *Message, from NodeID) error { return nil } - if err := msg.Verify(); err != nil { + if err := msg.Verify(e.QCDeserializer); err != nil { e.Logger.Debug("Notarization quorum certificate is invalid", zap.Stringer("NodeID", from), zap.Error(err)) return nil diff --git a/msg.go b/msg.go index 28ba72ba..3617e1f4 100644 --- a/msg.go +++ b/msg.go @@ -63,6 +63,10 @@ func (fc *FinalizationCertificate) Verify() error { return verifyContextQC(fc.QC, fc.Finalization.MarshalCanoto(), finalizationContext) } -func (n *Notarization) Verify() error { - return verifyContextQC(n.QC, n.Vote.MarshalCanoto(), voteContext) +func (n *Notarization) Verify(p QCDeserializer) error { + qc, err := p.DeserializeQuorumCertificate(n.QC) + if err != nil { + return err + } + return verifyContextQC(qc, n.Vote.MarshalCanoto(), voteContext) } diff --git a/types.canoto.go b/types.canoto.go index f6af4357..ac966e4e 100644 --- a/types.canoto.go +++ b/types.canoto.go @@ -1502,6 +1502,7 @@ func (c *Vote) MarshalCanotoInto(w canoto.Writer) canoto.Writer { const ( canoto__Notarization__Vote__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Notarization__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) ) type canotoData_Notarization struct { @@ -1575,6 +1576,17 @@ func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { if err != nil { return err } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.QC); err != nil { + return err + } + if len(c.QC) == 0 { + return canoto.ErrZeroValue + } default: return canoto.ErrUnknownField } @@ -1614,6 +1626,9 @@ func (c *Notarization) CalculateCanotoCache() { if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { c.canotoData.size += len(canoto__Notarization__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize } + if len(c.QC) != 0 { + c.canotoData.size += len(canoto__Notarization__QC__tag) + canoto.SizeBytes(c.QC) + } } // CachedCanotoSize returns the previously calculated size of the Canoto @@ -1662,6 +1677,10 @@ func (c *Notarization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { canoto.AppendInt(&w, int64(fieldSize)) w = (&c.Vote).MarshalCanotoInto(w) } + if len(c.QC) != 0 { + canoto.Append(&w, canoto__Notarization__QC__tag) + canoto.AppendBytes(&w, c.QC) + } return w } diff --git a/types.go b/types.go index 065c4acb..5d86866e 100644 --- a/types.go +++ b/types.go @@ -61,8 +61,8 @@ type ( canotoData canotoData_Vote } Notarization struct { - Vote ToBeSignedVote `canoto:"value,1"` - QC QuorumCertificate // `canoto:"bytes,2"` + Vote ToBeSignedVote `canoto:"value,1"` + QC []byte `canoto:"bytes,2"` canotoData canotoData_Notarization } From beeacecfac54c32d189eae4a08f8900565999e40 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 10:24:15 -0500 Subject: [PATCH 3/9] wip --- api.go | 4 +- encoding_test.go | 100 +-- epoch.go | 366 +++++------ epoch_multinode_test.go | 1 + epoch_test.go | 35 +- msg.go | 72 --- types.canoto.go | 1350 +++++++++------------------------------ types.go | 127 ++-- util_test.go | 4 +- 9 files changed, 600 insertions(+), 1459 deletions(-) delete mode 100644 msg.go diff --git a/api.go b/api.go index e56b3e3e..404608a5 100644 --- a/api.go +++ b/api.go @@ -46,8 +46,8 @@ type BlockBuilder interface { type Storage interface { Height() uint64 - Retrieve(seq uint64) (Block, FinalizationCertificate, bool) - Index(block Block, certificate FinalizationCertificate) + Retrieve(seq uint64) (Block, Quorum, bool) + Index(block Block, certificate Quorum) } type Communication interface { diff --git a/encoding_test.go b/encoding_test.go index 2d8e3c60..1cd6a4a8 100644 --- a/encoding_test.go +++ b/encoding_test.go @@ -11,78 +11,16 @@ import ( "github.com/stretchr/testify/require" ) -func TestBlockRecord(t *testing.T) { - bh := BlockHeader{ - ProtocolMetadata: ProtocolMetadata{ - Round: 2, - Seq: 3, - Epoch: 4, - }, - } - - _, err := rand.Read(bh.Prev[:]) - require.NoError(t, err) - - _, err = rand.Read(bh.Digest[:]) - require.NoError(t, err) - - payload := []byte{11, 12, 13, 14, 15, 16} - - record := Record{ - Block: &BlockRecord{ - Header: bh, - Payload: payload, - }, - } - recordBytes := record.MarshalCanoto() - - var parsedRecord Record - require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) - require.True(t, bh.Equals(&parsedRecord.Block.Header)) - require.Equal(t, payload, parsedRecord.Block.Payload) -} - -func FuzzBlockRecord(f *testing.F) { - f.Fuzz(func(t *testing.T, version uint8, round, seq, epoch uint64, prevPreimage, digestPreimage []byte, payload []byte) { - prev := sha256.Sum256(prevPreimage) - digest := sha256.Sum256(digestPreimage) - bh := BlockHeader{ - ProtocolMetadata: ProtocolMetadata{ - Round: round, - Seq: seq, - Epoch: epoch, - Prev: prev, - }, - Digest: digest, - } - - record := Record{ - Block: &BlockRecord{ - Header: bh, - Payload: payload, - }, - } - recordBytes := record.MarshalCanoto() - - var parsedRecord Record - require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) - require.True(t, bh.Equals(&parsedRecord.Block.Header)) - require.Equal(t, payload, parsedRecord.Block.Payload) - }) -} - func TestNotarizationRecord(t *testing.T) { sig := make([]byte, 64) _, err := rand.Read(sig) require.NoError(t, err) - vote := ToBeSignedVote{ - BlockHeader{ - ProtocolMetadata: ProtocolMetadata{ - Round: 2, - Seq: 3, - Epoch: 4, - }, + vote := BlockHeader{ + ProtocolMetadata: ProtocolMetadata{ + Round: 2, + Seq: 3, + Epoch: 4, }, } @@ -93,8 +31,8 @@ func TestNotarizationRecord(t *testing.T) { require.NoError(t, err) record := Record{ - Notarization: &QuorumRecord{ - Header: vote.BlockHeader, + Notarization: &Quorum{ + Header: vote, QC: []byte{1, 2, 3}, }, } @@ -102,7 +40,7 @@ func TestNotarizationRecord(t *testing.T) { var parsedRecord Record require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) - require.True(t, vote.BlockHeader.Equals(&parsedRecord.Notarization.Header)) + require.True(t, vote.Equals(&parsedRecord.Notarization.Header)) require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) } @@ -111,21 +49,19 @@ func FuzzNotarizationRecord(f *testing.F) { prev := sha256.Sum256(prevPreimage) digest := sha256.Sum256(digestPreimage) - vote := ToBeSignedVote{ - BlockHeader{ - ProtocolMetadata: ProtocolMetadata{ - Round: round, - Seq: seq, - Epoch: epoch, - Prev: prev, - }, - Digest: digest, + vote := BlockHeader{ + ProtocolMetadata: ProtocolMetadata{ + Round: round, + Seq: seq, + Epoch: epoch, + Prev: prev, }, + Digest: digest, } record := Record{ - Notarization: &QuorumRecord{ - Header: vote.BlockHeader, + Notarization: &Quorum{ + Header: vote, QC: []byte{1, 2, 3}, }, } @@ -133,7 +69,7 @@ func FuzzNotarizationRecord(f *testing.F) { var parsedRecord Record require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) - require.True(t, vote.BlockHeader.Equals(&parsedRecord.Notarization.Header)) + require.True(t, vote.Equals(&parsedRecord.Notarization.Header)) require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) }) } diff --git a/epoch.go b/epoch.go index 4d8f62d4..d9d587d6 100644 --- a/epoch.go +++ b/epoch.go @@ -16,20 +16,20 @@ import ( const defaultMaxRoundWindow = 10 type Round struct { - num uint64 - block Block - votes map[string]*Vote // NodeID --> vote - notarization *Notarization - finalizations map[string]*Finalization // NodeID --> vote - fCert *FinalizationCertificate + num uint64 + block Block + votes map[string]*Vote // NodeID --> vote + notarization *Quorum + finalizes map[string]*Vote // NodeID --> vote + finalization *Quorum } func NewRound(block Block) *Round { return &Round{ - num: block.BlockHeader().Round, - block: block, - votes: make(map[string]*Vote), - finalizations: make(map[string]*Finalization), + num: block.BlockHeader().Round, + block: block, + votes: make(map[string]*Vote), + finalizes: make(map[string]*Vote), } } @@ -96,15 +96,15 @@ func (e *Epoch) HandleMessage(msg *Message, from NodeID) error { switch { case msg.Proposal != nil: - return e.handleBlockMessage(msg, from) + return e.handleBlockMessage(msg.Proposal, from) case msg.Vote != nil: - return e.handleVoteMessage(msg, from) + return e.handleVoteMessage(msg.Vote, from) case msg.Notarization != nil: - return e.handleNotarizationMessage(msg, from) + return e.handleNotarizationMessage(msg.Notarization, from) case msg.Finalize != nil: - return e.handleFinalizationMessage(msg, from) + return e.handleFinalizeMessage(msg.Finalize, from) case msg.Finalization != nil: - return e.handleFinalizationCertificateMessage(msg, from) + return e.handleFinalizationMessage(msg.Finalization, from) default: return fmt.Errorf("invalid message type: %v", msg) } @@ -118,7 +118,7 @@ func (e *Epoch) init() error { e.finishCtx, e.finishFn = context.WithCancel(context.Background()) e.nodes = e.Comm.ListNodes() - e.quorumSize = Quorum(len(e.nodes)) + e.quorumSize = QuorumSize(len(e.nodes)) e.round = e.Round e.rounds = make(map[uint64]*Round) e.maxRoundWindow = defaultMaxRoundWindow @@ -172,36 +172,35 @@ func (e *Epoch) Stop() { e.finishFn() } -func (e *Epoch) handleFinalizationCertificateMessage(message *Message, from NodeID) error { - fCert := message.Finalization - round, exists := e.rounds[fCert.Finalization.Round] +func (e *Epoch) handleFinalizationMessage(msg *Quorum, from NodeID) error { + round, exists := e.rounds[msg.Header.Round] if !exists { - e.Logger.Debug("Received finalization certificate for a non existent round", zap.Int("round", int(fCert.Finalization.Round))) + e.Logger.Debug("Received finalization certificate for a non existent round", zap.Int("round", int(msg.Header.Round))) return nil } - if round.fCert != nil { - e.Logger.Debug("Received finalization for an already finalized round", zap.Uint64("round", fCert.Finalization.Round)) + if round.finalization != nil { + e.Logger.Debug("Received finalization for an already finalized round", zap.Uint64("round", msg.Header.Round)) return nil } - valid, err := e.isFinalizationCertificateValid(fCert) + valid, err := e.isFinalizationCertificateValid(msg) if err != nil { return err } if !valid { e.Logger.Debug("Received an invalid finalization certificate", - zap.Int("round", int(fCert.Finalization.Round)), + zap.Int("round", int(msg.Header.Round)), zap.Stringer("NodeID", from)) return nil } - round.fCert = fCert + round.finalization = msg - return e.persistFinalizationCertificate(*fCert) + return e.persistFinalization(*msg) } -func (e *Epoch) isFinalizationCertificateValid(fCert *FinalizationCertificate) (bool, error) { - valid, err := e.validateFinalizationQC(fCert) +func (e *Epoch) isFinalizationCertificateValid(finalization *Quorum) (bool, error) { + valid, err := e.validateFinalizationQC(finalization) if err != nil { return false, err } @@ -213,35 +212,37 @@ func (e *Epoch) isFinalizationCertificateValid(fCert *FinalizationCertificate) ( return false, nil } -func (e *Epoch) validateFinalizationQC(fCert *FinalizationCertificate) (bool, error) { - if fCert.QC == nil { +func (e *Epoch) validateFinalizationQC(finalization *Quorum) (bool, error) { + qc, err := e.QCDeserializer.DeserializeQuorumCertificate(finalization.QC) + if err != nil { + e.Logger.Debug("FinalizationCertificate QC failed to be parsed", + zap.Error(err), + ) return false, nil } // Check enough signers signed the finalization certificate - if e.quorumSize > len(fCert.QC.Signers()) { + if e.quorumSize > len(qc.Signers()) { e.Logger.Debug("ToBeSignedFinalization certificate signed by insufficient nodes", - zap.Int("count", len(fCert.QC.Signers())), + zap.Int("count", len(qc.Signers())), zap.Int("Quorum", e.quorumSize)) return false, nil } - signedTwice := e.hasSomeNodeSignedTwice(fCert.QC.Signers()) - + signedTwice := e.hasSomeNodeSignedTwice(qc.Signers()) if signedTwice { return false, nil } - if err := fCert.Verify(); err != nil { + if err := finalization.Verify(e.QCDeserializer, FinalizeContext); err != nil { return false, nil } return true, nil } -func (e *Epoch) handleFinalizationMessage(message *Message, from NodeID) error { - msg := message.Finalize - finalization := msg.Finalization +func (e *Epoch) handleFinalizeMessage(msg *Vote, from NodeID) error { + finalization := msg.Header // Only process a point to point finalization if !from.Equals(msg.Signature.Signer) { @@ -256,33 +257,40 @@ func (e *Epoch) handleFinalizationMessage(message *Message, from NodeID) error { return nil } - if round.fCert != nil { + if round.finalization != nil { e.Logger.Debug("Received finalization for an already finalized round", zap.Uint64("round", finalization.Round)) return nil } - if !e.isFinalizationValid(msg.Signature.Value, finalization, from) { + if err := msg.Verify(e.Verifier, FinalizeContext); err != nil { + e.Logger.Debug("Received a finalization with an invalid signature", + zap.Uint64("round", finalization.Round), + zap.Error(err), + ) return nil } - round.finalizations[string(from)] = msg + round.finalizes[string(from)] = msg return e.maybeCollectFinalizationCertificate(round) } -func (e *Epoch) handleVoteMessage(message *Message, _ NodeID) error { - msg := message.Vote - vote := msg.Vote +func (e *Epoch) handleVoteMessage(msg *Vote, _ NodeID) error { + vote := msg.Header // TODO: what if we've received a vote for a round we didn't instantiate yet? round, exists := e.rounds[vote.Round] if !exists { - e.Logger.Debug("Received a vote for a non existent round", zap.Uint64("round", vote.Round)) + e.Logger.Debug("Received a vote for a non existent round", + zap.Uint64("round", vote.Round), + ) return nil } if round.notarization != nil { - e.Logger.Debug("Round already notarized", zap.Uint64("round", vote.Round)) + e.Logger.Debug("Round already notarized", + zap.Uint64("round", vote.Round), + ) return nil } @@ -293,8 +301,11 @@ func (e *Epoch) handleVoteMessage(message *Message, _ NodeID) error { // Only verify the vote if we haven't verified it in the past. signature := msg.Signature if _, exists := round.votes[string(signature.Signer)]; !exists { - if err := vote.Verify(signature.Value, e.Verifier, signature.Signer); err != nil { - e.Logger.Debug("ToBeSignedVote verification failed", zap.Stringer("NodeID", signature.Signer), zap.Error(err)) + if err := msg.Verify(e.Verifier, VoteContext); err != nil { + e.Logger.Debug("ToBeSignedVote verification failed", + zap.Stringer("NodeID", signature.Signer), + zap.Error(err), + ) return nil } } @@ -304,15 +315,7 @@ func (e *Epoch) handleVoteMessage(message *Message, _ NodeID) error { return e.maybeCollectNotarization() } -func (e *Epoch) isFinalizationValid(signature []byte, finalization ToBeSignedFinalization, from NodeID) bool { - if err := finalization.Verify(signature, e.Verifier, from); err != nil { - e.Logger.Debug("Received a finalization with an invalid signature", zap.Uint64("round", finalization.Round), zap.Error(err)) - return false - } - return true -} - -func (e *Epoch) isVoteValid(vote ToBeSignedVote) bool { +func (e *Epoch) isVoteValid(vote BlockHeader) bool { // Ignore votes for previous rounds if vote.Round < e.round { return false @@ -329,7 +332,7 @@ func (e *Epoch) isVoteValid(vote ToBeSignedVote) bool { } func (e *Epoch) maybeCollectFinalizationCertificate(round *Round) error { - finalizationCount := len(round.finalizations) + finalizationCount := len(round.finalizes) if finalizationCount < e.quorumSize { e.Logger.Verbo("Counting finalizations", zap.Uint64("round", e.round), zap.Int("votes", finalizationCount)) @@ -341,80 +344,78 @@ func (e *Epoch) maybeCollectFinalizationCertificate(round *Round) error { func (e *Epoch) assembleFinalizationCertificate(round *Round) error { // Divide finalizations into sets that agree on the same metadata - finalizationsByMD := make(map[string][]*Finalization) + finalizesByMD := make(map[string][]*Vote) - for _, vote := range round.finalizations { - key := string(vote.Finalization.MarshalCanoto()) - finalizationsByMD[key] = append(finalizationsByMD[key], vote) + for _, vote := range round.finalizes { + key := string(vote.Header.MarshalCanoto()) + finalizesByMD[key] = append(finalizesByMD[key], vote) } - var finalizations []*Finalization + var finalizes []*Vote - for _, finalizationsWithTheSameDigest := range finalizationsByMD { + for _, finalizationsWithTheSameDigest := range finalizesByMD { if len(finalizationsWithTheSameDigest) >= e.quorumSize { - finalizations = finalizationsWithTheSameDigest + finalizes = finalizationsWithTheSameDigest break } } - if len(finalizations) == 0 { + if len(finalizes) == 0 { e.Logger.Debug("Could not find enough finalizations for the same metadata") return nil } - finalization := finalizations[0] + finalize := finalizes[0] - voteCount := len(finalizations) + voteCount := len(finalizes) signatures := make([]Signature, 0, voteCount) e.Logger.Info("Collected Quorum of votes", zap.Uint64("round", e.round), zap.Int("votes", voteCount)) - for _, vote := range finalizations { + for _, vote := range finalizes { // TODO: ensure all finalizations agree on the same metadata! e.Logger.Debug("Collected finalization from node", zap.Stringer("NodeID", vote.Signature.Signer)) signatures = append(signatures, vote.Signature) } - var fCert FinalizationCertificate - var err error - fCert.Finalization = finalization.Finalization - fCert.QC, err = e.SignatureAggregator.Aggregate(signatures) + qc, err := e.SignatureAggregator.Aggregate(signatures) if err != nil { return fmt.Errorf("could not aggregate signatures for finalization certificate: %w", err) } - round.fCert = &fCert + finalization := Quorum{ + Header: finalize.Header, + QC: qc.Bytes(), + } + round.finalization = &finalization - return e.persistFinalizationCertificate(fCert) + return e.persistFinalization(finalization) } -func (e *Epoch) persistFinalizationCertificate(fCert FinalizationCertificate) error { +func (e *Epoch) persistFinalization(finalization Quorum) error { // Check to see if we should commit this finalization to the storage as part of a block commit, // or otherwise write it to the WAL in order to commit it later. nextSeqToCommit := e.Storage.Height() - if fCert.Finalization.Seq == nextSeqToCommit { - block := e.rounds[fCert.Finalization.Round].block - e.Storage.Index(block, fCert) + if finalization.Header.Seq == nextSeqToCommit { + block := e.rounds[finalization.Header.Round].block + e.Storage.Index(block, finalization) e.Logger.Info("Committed block", - zap.Uint64("round", fCert.Finalization.Round), - zap.Uint64("sequence", fCert.Finalization.Seq), - zap.Stringer("digest", fCert.Finalization.BlockHeader.Digest)) + zap.Uint64("round", finalization.Header.Round), + zap.Uint64("sequence", finalization.Header.Seq), + zap.Stringer("digest", finalization.Header.Digest)) e.lastBlock = block // If the round we're committing is too far in the past, don't keep it in the rounds cache. - if fCert.Finalization.Round+e.maxRoundWindow < e.round { - delete(e.rounds, fCert.Finalization.Round) + if finalization.Header.Round+e.maxRoundWindow < e.round { + delete(e.rounds, finalization.Header.Round) } // Clean up the future messages - Remove all messages we may have stored for the round // the finalization is about. for _, messagesFromNode := range e.futureMessages { - delete(messagesFromNode, fCert.Finalization.Round) + delete(messagesFromNode, finalization.Header.Round) } } else { record := Record{ - Finalization: &QuorumRecord{ - Header: fCert.Finalization.BlockHeader, - QC: fCert.QC.Bytes(), - }, + Finalization: &finalization, } recordBytes := record.MarshalCanoto() if err := e.WAL.Append(recordBytes); err != nil { @@ -424,16 +425,16 @@ func (e *Epoch) persistFinalizationCertificate(fCert FinalizationCertificate) er e.Logger.Debug("Persisted finalization certificate to WAL", zap.Int("size", len(recordBytes)), - zap.Uint64("round", fCert.Finalization.Round), - zap.Stringer("digest", fCert.Finalization.BlockHeader.Digest)) + zap.Uint64("round", finalization.Header.Round), + zap.Stringer("digest", finalization.Header.Digest)) } - finalizationCertificate := &Message{Finalization: &fCert} - e.Comm.Broadcast(finalizationCertificate) + finalizationMessage := &Message{Finalization: &finalization} + e.Comm.Broadcast(finalizationMessage) e.Logger.Debug("Broadcast finalization certificate", - zap.Uint64("round", fCert.Finalization.Round), - zap.Stringer("digest", fCert.Finalization.BlockHeader.Digest)) + zap.Uint64("round", finalization.Header.Round), + zap.Stringer("digest", finalization.Header.Digest)) return nil } @@ -455,7 +456,7 @@ func (e *Epoch) maybeCollectNotarization() error { // Ensure we have enough votes for the same digest var voteCountForOurDigest int for _, vote := range votesForCurrentRound { - if bytes.Equal(digestWeExpect[:], vote.Vote.Digest[:]) { + if bytes.Equal(digestWeExpect[:], vote.Header.Digest[:]) { voteCountForOurDigest++ } } @@ -470,20 +471,21 @@ func (e *Epoch) maybeCollectNotarization() error { } func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, digest Digest) error { - vote := ToBeSignedVote{ - BlockHeader{ - ProtocolMetadata: ProtocolMetadata{ - Epoch: e.Epoch, - Round: e.round, - }, - Digest: digest, + vote := BlockHeader{ + ProtocolMetadata: ProtocolMetadata{ + Epoch: e.Epoch, + Round: e.round, }, + Digest: digest, } voteCount := len(votesForCurrentRound) signatures := make([]Signature, 0, voteCount) - e.Logger.Info("Collected Quorum of votes", zap.Uint64("round", e.round), zap.Int("votes", voteCount)) + e.Logger.Info("Collected Quorum of votes", + zap.Uint64("round", e.round), + zap.Int("votes", voteCount), + ) for _, vote := range votesForCurrentRound { e.Logger.Debug("Collected vote from node", zap.Stringer("NodeID", vote.Signature.Signer)) signatures = append(signatures, vote.Signature) @@ -494,23 +496,20 @@ func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, dige return fmt.Errorf("could not aggregate signatures for notarization: %w", err) } - notarization := Notarization{ - Vote: vote, - QC: qc.Bytes(), + notarization := Quorum{ + Header: vote, + QC: qc.Bytes(), } if err := e.storeNotarization(notarization); err != nil { return err } - return e.persistNotarization(notarization, vote) + return e.persistNotarization(notarization) } -func (e *Epoch) persistNotarization(notarization Notarization, vote ToBeSignedVote) error { +func (e *Epoch) persistNotarization(notarization Quorum) error { record := Record{ - Notarization: &QuorumRecord{ - Header: vote.BlockHeader, - QC: notarization.QC, - }, + Notarization: ¬arization, } recordBytes := record.MarshalCanoto() if err := e.WAL.Append(recordBytes); err != nil { @@ -520,24 +519,23 @@ func (e *Epoch) persistNotarization(notarization Notarization, vote ToBeSignedVo e.Logger.Debug("Persisted notarization to WAL", zap.Int("size", len(recordBytes)), - zap.Uint64("round", notarization.Vote.Round), - zap.Stringer("digest", notarization.Vote.BlockHeader.Digest)) + zap.Uint64("round", notarization.Header.Round), + zap.Stringer("digest", notarization.Header.Digest)) notarizationMessage := &Message{Notarization: ¬arization} e.Comm.Broadcast(notarizationMessage) e.Logger.Debug("Broadcast notarization", - zap.Uint64("round", notarization.Vote.Round), - zap.Stringer("digest", notarization.Vote.BlockHeader.Digest)) + zap.Uint64("round", notarization.Header.Round), + zap.Stringer("digest", notarization.Header.Digest)) - e.rounds[notarization.Vote.Round].notarization = ¬arization + e.rounds[notarization.Header.Round].notarization = ¬arization return e.doNotarized() } -func (e *Epoch) handleNotarizationMessage(message *Message, from NodeID) error { - msg := message.Notarization - vote := msg.Vote +func (e *Epoch) handleNotarizationMessage(msg *Quorum, from NodeID) error { + vote := msg.Header // Ignore votes for previous rounds if vote.Round < e.round { @@ -573,13 +571,13 @@ func (e *Epoch) handleNotarizationMessage(message *Message, from NodeID) error { return nil } - if err := msg.Verify(e.QCDeserializer); err != nil { + if err := msg.Verify(e.QCDeserializer, VoteContext); err != nil { e.Logger.Debug("Notarization quorum certificate is invalid", zap.Stringer("NodeID", from), zap.Error(err)) return nil } - return e.persistNotarization(*msg, vote) + return e.persistNotarization(*msg) } func (e *Epoch) hasSomeNodeSignedTwice(nodeIDs []NodeID) bool { @@ -596,14 +594,16 @@ func (e *Epoch) hasSomeNodeSignedTwice(nodeIDs []NodeID) bool { return false } -func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { - block := message.Proposal.Block - if block == nil { - e.Logger.Debug("Got empty block in a BlockMessage") +func (e *Epoch) handleBlockMessage(msg *Proposal, _ NodeID) error { + block, err := e.BlockDeserializer.DeserializeBlock(msg.Block) + if err != nil { + e.Logger.Debug("Got invalid block in a BlockMessage", + zap.Error(err), + ) return nil } - vote := message.Proposal.Vote + vote := msg.Vote from := vote.Signature.Signer md := block.BlockHeader() @@ -629,13 +629,13 @@ func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { // Ensure the block was voted on by its block producer: // 1) Verify block digest corresponds to the digest voted on - if !bytes.Equal(vote.Vote.Digest[:], md.Digest[:]) { - e.Logger.Debug("ToBeSignedVote digest mismatches block digest", zap.Stringer("voteDigest", vote.Vote.Digest), + if !bytes.Equal(vote.Header.Digest[:], md.Digest[:]) { + e.Logger.Debug("ToBeSignedVote digest mismatches block digest", zap.Stringer("voteDigest", vote.Header.Digest), zap.Stringer("blockDigest", md.Digest)) return nil } // 2) Verify the vote is properly signed - if err := vote.Vote.Verify(vote.Signature.Value, e.Verifier, vote.Signature.Signer); err != nil { + if err := vote.Verify(e.Verifier, VoteContext); err != nil { e.Logger.Debug("ToBeSignedVote verification failed", zap.Stringer("NodeID", vote.Signature.Signer), zap.Error(err)) return nil } @@ -651,7 +651,7 @@ func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { msgsForRound = &messagesForRound{} e.futureMessages[string(from)][md.Round] = msgsForRound } - msgsForRound.proposal = message + msgsForRound.proposal = msg return nil } @@ -680,10 +680,7 @@ func (e *Epoch) handleBlockMessage(message *Message, _ NodeID) error { } record := Record{ - Block: &BlockRecord{ - Header: md, - Payload: block.Bytes(), - }, + Block: block.Bytes(), } recordBytes := record.MarshalCanoto() if err := e.WAL.Append(recordBytes); err != nil { @@ -698,7 +695,11 @@ func (e *Epoch) wasBlockAlreadyVerified(from NodeID, md BlockHeader) bool { var alreadyVerified bool msgsForRound, exists := e.futureMessages[string(from)][md.Round] if exists && msgsForRound.proposal != nil { - bh := msgsForRound.proposal.Proposal.Block.BlockHeader() + block, err := e.BlockDeserializer.DeserializeBlock(msgsForRound.proposal.Block) + if err != nil { + panic("failed to deserialize block") + } + bh := block.BlockHeader() alreadyVerified = bh.Equals(&md) } return alreadyVerified @@ -804,16 +805,14 @@ func (e *Epoch) proposeBlock() error { return errors.New("failed to build block") } - md := block.BlockHeader() + blockHeader := block.BlockHeader() + blockBytes := block.Bytes() // Write record to WAL before broadcasting it, so that // if we crash during broadcasting, we know what we sent. record := Record{ - Block: &BlockRecord{ - Header: block.BlockHeader(), - Payload: block.Bytes(), - }, + Block: blockBytes, } recordBytes := record.MarshalCanoto() if err := e.WAL.Append(recordBytes); err != nil { @@ -821,9 +820,9 @@ func (e *Epoch) proposeBlock() error { return err } e.Logger.Debug("Wrote block to WAL", - zap.Uint64("round", md.Round), - zap.Int("size", len(record.Block.Payload)), - zap.Stringer("digest", md.Digest)) + zap.Uint64("round", blockHeader.Round), + zap.Int("size", len(record.Block)), + zap.Stringer("digest", blockHeader.Digest)) vote, err := e.voteOnBlock(block) if err != nil { @@ -831,8 +830,8 @@ func (e *Epoch) proposeBlock() error { } proposal := &Message{ - Proposal: &BlockMessage{ - Block: block, + Proposal: &Proposal{ + Block: blockBytes, Vote: vote, }, } @@ -843,11 +842,11 @@ func (e *Epoch) proposeBlock() error { e.Comm.Broadcast(proposal) e.Logger.Debug("Proposal broadcast", - zap.Uint64("round", md.Round), - zap.Int("size", len(record.Block.Payload)), - zap.Stringer("digest", md.Digest)) + zap.Uint64("round", blockHeader.Round), + zap.Int("size", len(record.Block)), + zap.Stringer("digest", blockHeader.Digest)) - return e.handleVoteMessage(&Message{Vote: &vote}, e.ID) + return e.handleVoteMessage(&vote, e.ID) } func (e *Epoch) Metadata() ProtocolMetadata { @@ -878,7 +877,7 @@ func (e *Epoch) startRound() error { // If we're not the leader, check if we have received a proposal earlier for this round msgsForRound, exists := e.futureMessages[string(leaderForCurrentRound)][e.round] - if !exists || msgsForRound.proposal == nil || msgsForRound.proposal.Proposal == nil { + if !exists || msgsForRound.proposal == nil { return nil } @@ -905,28 +904,21 @@ func (e *Epoch) doProposed(block Block, voteFromLeader Vote) error { e.Comm.Broadcast(voteMsg) // Send yourself a vote message - if err := e.handleVoteMessage(voteMsg, e.ID); err != nil { + if err := e.handleVoteMessage(&vote, e.ID); err != nil { return err } - return e.handleVoteMessage(&Message{Vote: &voteFromLeader}, e.ID) + return e.handleVoteMessage(&voteFromLeader, e.ID) } func (e *Epoch) voteOnBlock(block Block) (Vote, error) { - vote := ToBeSignedVote{BlockHeader: block.BlockHeader()} - sig, err := vote.Sign(e.Signer) - if err != nil { - return Vote{}, fmt.Errorf("failed signing vote %w", err) + vote := Vote{ + Header: block.BlockHeader(), } - - sv := Vote{ - Signature: Signature{ - Signer: e.ID, - Value: sig, - }, - Vote: vote, + if err := vote.Sign(e.ID, e.Signer, VoteContext); err != nil { + return Vote{}, fmt.Errorf("failed signing vote %w", err) } - return sv, nil + return vote, nil } func (e *Epoch) increaseRound() { @@ -940,38 +932,28 @@ func (e *Epoch) doNotarized() error { md := block.BlockHeader() - f := ToBeSignedFinalization{BlockHeader: md} - signature, err := f.Sign(e.Signer) - if err != nil { - return fmt.Errorf("failed signing vote %w", err) + f := Vote{ + Header: md, } - - sf := Finalization{ - Signature: Signature{ - Signer: e.ID, - Value: signature, - }, - Finalization: ToBeSignedFinalization{ - BlockHeader: md, - }, + if err := f.Sign(e.ID, e.Signer, FinalizeContext); err != nil { + return fmt.Errorf("failed signing finalize %w", err) } finalizationMsg := &Message{ - Finalize: &sf, + Finalize: &f, } - e.Comm.Broadcast(finalizationMsg) e.increaseRound() err1 := e.startRound() - err2 := e.handleFinalizationMessage(finalizationMsg, e.ID) + err2 := e.handleFinalizeMessage(&f, e.ID) return errors.Join(err1, err2) } -func (e *Epoch) storeNotarization(notarization Notarization) error { - round := notarization.Vote.Round +func (e *Epoch) storeNotarization(notarization Quorum) error { + round := notarization.Header.Round r, exists := e.rounds[round] if !exists { fmt.Errorf("attempted to store notarization of a non existent round %d", round) @@ -992,12 +974,12 @@ func (e *Epoch) maybeLoadFutureMessages(round uint64) { e.handleVoteMessage(msgs.vote, NodeID(from)) msgs.vote = nil } - if msgs.finalization != nil { - e.handleFinalizationMessage(msgs.finalization, NodeID(from)) - msgs.finalization = nil + if msgs.finalize != nil { + e.handleFinalizeMessage(msgs.finalize, NodeID(from)) + msgs.finalize = nil } - if msgs.proposal == nil && msgs.vote == nil && msgs.finalization == nil { + if msgs.proposal == nil && msgs.vote == nil && msgs.finalize == nil { delete(messagesFromNode, round) } } @@ -1046,7 +1028,7 @@ func LeaderForRound(nodes []NodeID, r uint64) NodeID { return nodes[r%uint64(n)] } -func Quorum(n int) int { +func QuorumSize(n int) int { f := (n - 1) / 3 // Obtained from the equation: // Quorum * 2 = N + F + 1 @@ -1057,7 +1039,7 @@ func Quorum(n int) int { type messagesFromNode map[string]map[uint64]*messagesForRound type messagesForRound struct { - proposal *Message - vote *Message - finalization *Message + proposal *Proposal + vote *Vote + finalize *Vote } diff --git a/epoch_multinode_test.go b/epoch_multinode_test.go index 35a9af33..f84c507c 100644 --- a/epoch_multinode_test.go +++ b/epoch_multinode_test.go @@ -71,6 +71,7 @@ func newSimplexNode(t *testing.T, id uint8, net *inMemNetwork, bb BlockBuilder) Signer: &testSigner{}, WAL: wal, Verifier: &testVerifier{}, + BlockDeserializer: &blockDeserializer{}, Storage: storage, BlockBuilder: bb, SignatureAggregator: &testSignatureAggregator{}, diff --git a/epoch_test.go b/epoch_test.go index e1160e99..9efb2c9b 100644 --- a/epoch_test.go +++ b/epoch_test.go @@ -28,13 +28,14 @@ func TestEpochSimpleFlow(t *testing.T) { storage := newInMemStorage() nodes := []NodeID{{1}, {2}, {3}, {4}} - quorum := Quorum(len(nodes)) + quorum := QuorumSize(len(nodes)) conf := EpochConfig{ Logger: l, ID: NodeID{1}, Signer: &testSigner{}, WAL: &wal.InMemWAL{}, Verifier: &testVerifier{}, + BlockDeserializer: &blockDeserializer{}, Storage: storage, Comm: noopComm(nodes), BlockBuilder: bb, @@ -64,9 +65,9 @@ func TestEpochSimpleFlow(t *testing.T) { // send node a message from the leader vote := newVote(block, leader) e.HandleMessage(&Message{ - Proposal: &BlockMessage{ + Proposal: &Proposal{ Vote: *vote, - Block: block, + Block: block.Bytes(), }, }, leader) } @@ -100,9 +101,7 @@ func newVote(block *testBlock, id NodeID) *Vote { Signature: Signature{ Signer: id, }, - Vote: ToBeSignedVote{ - BlockHeader: block.BlockHeader(), - }, + Header: block.BlockHeader(), } } @@ -117,13 +116,11 @@ func injectVote(t *testing.T, e *Epoch, block *testBlock, id NodeID) { func injectFinalization(t *testing.T, e *Epoch, block *testBlock, id NodeID) { md := block.BlockHeader() err := e.HandleMessage(&Message{ - Finalize: &Finalization{ + Finalize: &Vote{ Signature: Signature{ Signer: id, }, - Finalization: ToBeSignedFinalization{ - BlockHeader: md, - }, + Header: md, }, }, id) require.NoError(t, err) @@ -289,7 +286,7 @@ func (t *testBlock) Bytes() []byte { type InMemStorage struct { data map[uint64]struct { Block - FinalizationCertificate + Quorum } lock sync.Mutex @@ -300,7 +297,7 @@ func newInMemStorage() *InMemStorage { s := &InMemStorage{ data: make(map[uint64]struct { Block - FinalizationCertificate + Quorum }), } @@ -326,15 +323,15 @@ func (mem *InMemStorage) Height() uint64 { return uint64(len(mem.data)) } -func (mem *InMemStorage) Retrieve(seq uint64) (Block, FinalizationCertificate, bool) { +func (mem *InMemStorage) Retrieve(seq uint64) (Block, Quorum, bool) { item, ok := mem.data[seq] if !ok { - return nil, FinalizationCertificate{}, false + return nil, Quorum{}, false } - return item.Block, item.FinalizationCertificate, true + return item.Block, item.Quorum, true } -func (mem *InMemStorage) Index(block Block, certificate FinalizationCertificate) { +func (mem *InMemStorage) Index(block Block, certificate Quorum) { mem.lock.Lock() defer mem.lock.Unlock() @@ -346,7 +343,7 @@ func (mem *InMemStorage) Index(block Block, certificate FinalizationCertificate) } mem.data[seq] = struct { Block - FinalizationCertificate + Quorum }{block, certificate, } @@ -383,7 +380,7 @@ func TestBlockDeserializer(t *testing.T) { require.Equal(t, tb, tb2) } -func TestQuorum(t *testing.T) { +func TestQuorumSize(t *testing.T) { for _, testCase := range []struct { n int f int @@ -439,7 +436,7 @@ func TestQuorum(t *testing.T) { }, } { t.Run(fmt.Sprintf("%d", testCase.n), func(t *testing.T) { - require.Equal(t, testCase.q, Quorum(testCase.n)) + require.Equal(t, testCase.q, QuorumSize(testCase.n)) }) } } diff --git a/msg.go b/msg.go deleted file mode 100644 index 3617e1f4..00000000 --- a/msg.go +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package simplex - -const ( - voteContext = "ToBeSignedVote" - finalizationContext = "ToBeSignedFinalization" -) - -type ToBeSignedVote struct { - BlockHeader -} - -func (v *ToBeSignedVote) Sign(signer Signer) ([]byte, error) { - return signContext(signer, v.MarshalCanoto(), voteContext) -} - -func (v *ToBeSignedVote) Verify(signature []byte, verifier SignatureVerifier, signers NodeID) error { - return verifyContext(signature, verifier, v.MarshalCanoto(), voteContext, signers) -} - -type ToBeSignedFinalization struct { - BlockHeader -} - -func (f *ToBeSignedFinalization) Sign(signer Signer) ([]byte, error) { - return signContext(signer, f.MarshalCanoto(), finalizationContext) -} - -func (f *ToBeSignedFinalization) Verify(signature []byte, verifier SignatureVerifier, signers NodeID) error { - return verifyContext(signature, verifier, f.MarshalCanoto(), finalizationContext, signers) -} - -func signContext(signer Signer, msg []byte, context string) ([]byte, error) { - sm := SignedMessage{ - Payload: msg, - Context: context, - } - toBeSigned := sm.MarshalCanoto() - return signer.Sign(toBeSigned) -} - -func verifyContext(signature []byte, verifier SignatureVerifier, msg []byte, context string, signers NodeID) error { - sm := SignedMessage{ - Payload: msg, - Context: context, - } - toBeSigned := sm.MarshalCanoto() - return verifier.Verify(toBeSigned, signature, signers) -} - -func verifyContextQC(qc QuorumCertificate, msg []byte, context string) error { - sm := SignedMessage{ - Payload: msg, - Context: context, - } - toBeSigned := sm.MarshalCanoto() - return qc.Verify(toBeSigned) -} - -func (fc *FinalizationCertificate) Verify() error { - return verifyContextQC(fc.QC, fc.Finalization.MarshalCanoto(), finalizationContext) -} - -func (n *Notarization) Verify(p QCDeserializer) error { - qc, err := p.DeserializeQuorumCertificate(n.QC) - if err != nil { - return err - } - return verifyContextQC(qc, n.Vote.MarshalCanoto(), voteContext) -} diff --git a/types.canoto.go b/types.canoto.go index ac966e4e..ca8ef8e1 100644 --- a/types.canoto.go +++ b/types.canoto.go @@ -85,26 +85,12 @@ func (c *Record) UnmarshalCanotoFrom(r canoto.Reader) error { } c.canotoData.recordOneOf = 1 - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { + if err := canoto.ReadBytes(&r, &c.Block); err != nil { return err } - if len(msgBytes) == 0 { + if len(c.Block) == 0 { return canoto.ErrZeroValue } - - remainingBytes := r.B - r.B = msgBytes - c.Block = canoto.MakePointer(c.Block) - err = (c.Block).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } case 2: if wireType != canoto.Len { return canoto.ErrUnexpectedWireType @@ -186,14 +172,11 @@ func (c *Record) ValidCanoto() bool { var ( recordOneOf uint32 ) - if c.Block != nil { - (c.Block).CalculateCanotoCache() - if (c.Block).CachedCanotoSize() != 0 { - if recordOneOf != 0 { - return false - } - recordOneOf = 1 + if len(c.Block) != 0 { + if recordOneOf != 0 { + return false } + recordOneOf = 1 } if c.Notarization != nil { (c.Notarization).CalculateCanotoCache() @@ -213,9 +196,6 @@ func (c *Record) ValidCanoto() bool { recordOneOf = 3 } } - if c.Block != nil && !(c.Block).ValidCanoto() { - return false - } if c.Notarization != nil && !(c.Notarization).ValidCanoto() { return false } @@ -235,12 +215,9 @@ func (c *Record) CalculateCanotoCache() { } c.canotoData.recordOneOf = 0 c.canotoData.size = 0 - if c.Block != nil { - (c.Block).CalculateCanotoCache() - if fieldSize := (c.Block).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Record__Block__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + if len(c.Block) != 0 { + c.canotoData.size += len(canoto__Record__Block__tag) + canoto.SizeBytes(c.Block) c.canotoData.recordOneOf = 1 - } } if c.Notarization != nil { (c.Notarization).CalculateCanotoCache() @@ -313,12 +290,9 @@ func (c *Record) MarshalCanotoInto(w canoto.Writer) canoto.Writer { if c == nil { return w } - if c.Block != nil { - if fieldSize := (c.Block).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Record__Block__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Block).MarshalCanotoInto(w) - } + if len(c.Block) != 0 { + canoto.Append(&w, canoto__Record__Block__tag) + canoto.AppendBytes(&w, c.Block) } if c.Notarization != nil { if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { @@ -338,17 +312,22 @@ func (c *Record) MarshalCanotoInto(w canoto.Writer) canoto.Writer { } const ( - canoto__QuorumRecord__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__QuorumRecord__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Message__Proposal__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Message__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Message__Notarization__tag = "\x1a" // canoto.Tag(3, canoto.Len) + canoto__Message__Finalize__tag = "\x22" // canoto.Tag(4, canoto.Len) + canoto__Message__Finalization__tag = "\x2a" // canoto.Tag(5, canoto.Len) ) -type canotoData_QuorumRecord struct { +type canotoData_Message struct { size int + + messageOneOf uint32 } // MakeCanoto creates a new empty value. -func (*QuorumRecord) MakeCanoto() *QuorumRecord { - return new(QuorumRecord) +func (*Message) MakeCanoto() *Message { + return new(Message) } // UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. @@ -359,7 +338,7 @@ func (*QuorumRecord) MakeCanoto() *QuorumRecord { // bytes will retain their previous values. If a OneOf field was previously // cached as being set, attempting to unmarshal that OneOf again will return // canoto.ErrDuplicateOneOf. -func (c *QuorumRecord) UnmarshalCanoto(bytes []byte) error { +func (c *Message) UnmarshalCanoto(bytes []byte) error { r := canoto.Reader{ B: bytes, } @@ -377,7 +356,7 @@ func (c *QuorumRecord) UnmarshalCanoto(bytes []byte) error { // canoto.ErrDuplicateOneOf. // // This function enables configuration of reader options. -func (c *QuorumRecord) UnmarshalCanotoFrom(r canoto.Reader) error { +func (c *Message) UnmarshalCanotoFrom(r canoto.Reader) error { var minField uint32 for canoto.HasNext(&r) { field, wireType, err := canoto.ReadTag(&r) @@ -393,6 +372,10 @@ func (c *QuorumRecord) UnmarshalCanotoFrom(r canoto.Reader) error { if wireType != canoto.Len { return canoto.ErrUnexpectedWireType } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 1 originalUnsafe := r.Unsafe r.Unsafe = true @@ -408,7 +391,8 @@ func (c *QuorumRecord) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Header).UnmarshalCanotoFrom(r) + c.Proposal = canoto.MakePointer(c.Proposal) + err = (c.Proposal).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err @@ -417,166 +401,10 @@ func (c *QuorumRecord) UnmarshalCanotoFrom(r canoto.Reader) error { if wireType != canoto.Len { return canoto.ErrUnexpectedWireType } - - if err := canoto.ReadBytes(&r, &c.QC); err != nil { - return err - } - if len(c.QC) == 0 { - return canoto.ErrZeroValue - } - default: - return canoto.ErrUnknownField - } - - minField = field + 1 - } - return nil -} - -// ValidCanoto validates that the struct can be correctly marshaled into the -// Canoto format. -// -// Specifically, ValidCanoto ensures: -// 1. All OneOfs are specified at most once. -// 2. All strings are valid utf-8. -// 3. All custom fields are ValidCanoto. -func (c *QuorumRecord) ValidCanoto() bool { - if c == nil { - return true - } - if !(&c.Header).ValidCanoto() { - return false - } - return true -} - -// CalculateCanotoCache populates size and OneOf caches based on the current -// values in the struct. -// -// It is not safe to call this function concurrently. -func (c *QuorumRecord) CalculateCanotoCache() { - if c == nil { - return - } - c.canotoData.size = 0 - (&c.Header).CalculateCanotoCache() - if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__QuorumRecord__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - } - if len(c.QC) != 0 { - c.canotoData.size += len(canoto__QuorumRecord__QC__tag) + canoto.SizeBytes(c.QC) - } -} - -// CachedCanotoSize returns the previously calculated size of the Canoto -// representation from CalculateCanotoCache. -// -// If CalculateCanotoCache has not yet been called, it will return 0. -// -// If the struct has been modified since the last call to CalculateCanotoCache, -// the returned size may be incorrect. -func (c *QuorumRecord) CachedCanotoSize() int { - if c == nil { - return 0 - } - return c.canotoData.size -} - -// MarshalCanoto returns the Canoto representation of this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *QuorumRecord) MarshalCanoto() []byte { - c.CalculateCanotoCache() - w := canoto.Writer{ - B: make([]byte, 0, c.CachedCanotoSize()), - } - w = c.MarshalCanotoInto(w) - return w.B -} - -// MarshalCanotoInto writes the struct into a canoto.Writer and returns the -// resulting canoto.Writer. Most users should just use MarshalCanoto. -// -// It is assumed that CalculateCanotoCache has been called since the last -// modification to this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *QuorumRecord) MarshalCanotoInto(w canoto.Writer) canoto.Writer { - if c == nil { - return w - } - if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__QuorumRecord__Header__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Header).MarshalCanotoInto(w) - } - if len(c.QC) != 0 { - canoto.Append(&w, canoto__QuorumRecord__QC__tag) - canoto.AppendBytes(&w, c.QC) - } - return w -} - -const ( - canoto__BlockRecord__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__BlockRecord__Payload__tag = "\x12" // canoto.Tag(2, canoto.Len) -) - -type canotoData_BlockRecord struct { - size int -} - -// MakeCanoto creates a new empty value. -func (*BlockRecord) MakeCanoto() *BlockRecord { - return new(BlockRecord) -} - -// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -func (c *BlockRecord) UnmarshalCanoto(bytes []byte) error { - r := canoto.Reader{ - B: bytes, - } - return c.UnmarshalCanotoFrom(r) -} - -// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users -// should just use UnmarshalCanoto. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -// -// This function enables configuration of reader options. -func (c *BlockRecord) UnmarshalCanotoFrom(r canoto.Reader) error { - var minField uint32 - for canoto.HasNext(&r) { - field, wireType, err := canoto.ReadTag(&r) - if err != nil { - return err - } - if field < minField { - return canoto.ErrInvalidFieldOrder - } - - switch field { - case 1: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf } + c.canotoData.messageOneOf = 2 originalUnsafe := r.Unsafe r.Unsafe = true @@ -592,768 +420,49 @@ func (c *BlockRecord) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Header).UnmarshalCanotoFrom(r) + c.Vote = canoto.MakePointer(c.Vote) + err = (c.Vote).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err } - case 2: + case 3: if wireType != canoto.Len { return canoto.ErrUnexpectedWireType } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 3 - if err := canoto.ReadBytes(&r, &c.Payload); err != nil { + originalUnsafe := r.Unsafe + r.Unsafe = true + var msgBytes []byte + err := canoto.ReadBytes(&r, &msgBytes) + r.Unsafe = originalUnsafe + if err != nil { return err } - if len(c.Payload) == 0 { - return canoto.ErrZeroValue - } - default: - return canoto.ErrUnknownField - } - - minField = field + 1 - } - return nil -} - -// ValidCanoto validates that the struct can be correctly marshaled into the -// Canoto format. -// -// Specifically, ValidCanoto ensures: -// 1. All OneOfs are specified at most once. -// 2. All strings are valid utf-8. -// 3. All custom fields are ValidCanoto. -func (c *BlockRecord) ValidCanoto() bool { - if c == nil { - return true - } - if !(&c.Header).ValidCanoto() { - return false - } - return true -} - -// CalculateCanotoCache populates size and OneOf caches based on the current -// values in the struct. -// -// It is not safe to call this function concurrently. -func (c *BlockRecord) CalculateCanotoCache() { - if c == nil { - return - } - c.canotoData.size = 0 - (&c.Header).CalculateCanotoCache() - if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__BlockRecord__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - } - if len(c.Payload) != 0 { - c.canotoData.size += len(canoto__BlockRecord__Payload__tag) + canoto.SizeBytes(c.Payload) - } -} - -// CachedCanotoSize returns the previously calculated size of the Canoto -// representation from CalculateCanotoCache. -// -// If CalculateCanotoCache has not yet been called, it will return 0. -// -// If the struct has been modified since the last call to CalculateCanotoCache, -// the returned size may be incorrect. -func (c *BlockRecord) CachedCanotoSize() int { - if c == nil { - return 0 - } - return c.canotoData.size -} - -// MarshalCanoto returns the Canoto representation of this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *BlockRecord) MarshalCanoto() []byte { - c.CalculateCanotoCache() - w := canoto.Writer{ - B: make([]byte, 0, c.CachedCanotoSize()), - } - w = c.MarshalCanotoInto(w) - return w.B -} - -// MarshalCanotoInto writes the struct into a canoto.Writer and returns the -// resulting canoto.Writer. Most users should just use MarshalCanoto. -// -// It is assumed that CalculateCanotoCache has been called since the last -// modification to this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *BlockRecord) MarshalCanotoInto(w canoto.Writer) canoto.Writer { - if c == nil { - return w - } - if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__BlockRecord__Header__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Header).MarshalCanotoInto(w) - } - if len(c.Payload) != 0 { - canoto.Append(&w, canoto__BlockRecord__Payload__tag) - canoto.AppendBytes(&w, c.Payload) - } - return w -} - -const ( - canoto__Message__Proposal__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__Message__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) - canoto__Message__Notarization__tag = "\x1a" // canoto.Tag(3, canoto.Len) - canoto__Message__Finalize__tag = "\x22" // canoto.Tag(4, canoto.Len) - canoto__Message__Finalization__tag = "\x2a" // canoto.Tag(5, canoto.Len) -) - -type canotoData_Message struct { - size int - - messageOneOf uint32 -} - -// MakeCanoto creates a new empty value. -func (*Message) MakeCanoto() *Message { - return new(Message) -} - -// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -func (c *Message) UnmarshalCanoto(bytes []byte) error { - r := canoto.Reader{ - B: bytes, - } - return c.UnmarshalCanotoFrom(r) -} - -// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users -// should just use UnmarshalCanoto. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -// -// This function enables configuration of reader options. -func (c *Message) UnmarshalCanotoFrom(r canoto.Reader) error { - var minField uint32 - for canoto.HasNext(&r) { - field, wireType, err := canoto.ReadTag(&r) - if err != nil { - return err - } - if field < minField { - return canoto.ErrInvalidFieldOrder - } - - switch field { - case 1: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - if c.canotoData.messageOneOf != 0 { - return canoto.ErrDuplicateOneOf - } - c.canotoData.messageOneOf = 1 - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { - return canoto.ErrZeroValue - } - - remainingBytes := r.B - r.B = msgBytes - c.Proposal = canoto.MakePointer(c.Proposal) - err = (c.Proposal).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - case 2: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - if c.canotoData.messageOneOf != 0 { - return canoto.ErrDuplicateOneOf - } - c.canotoData.messageOneOf = 2 - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { - return canoto.ErrZeroValue - } - - remainingBytes := r.B - r.B = msgBytes - c.Vote = canoto.MakePointer(c.Vote) - err = (c.Vote).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - case 3: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - if c.canotoData.messageOneOf != 0 { - return canoto.ErrDuplicateOneOf - } - c.canotoData.messageOneOf = 3 - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { - return canoto.ErrZeroValue - } - - remainingBytes := r.B - r.B = msgBytes - c.Notarization = canoto.MakePointer(c.Notarization) - err = (c.Notarization).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - case 4: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - if c.canotoData.messageOneOf != 0 { - return canoto.ErrDuplicateOneOf - } - c.canotoData.messageOneOf = 4 - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { - return canoto.ErrZeroValue - } - - remainingBytes := r.B - r.B = msgBytes - c.Finalize = canoto.MakePointer(c.Finalize) - err = (c.Finalize).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - case 5: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - if c.canotoData.messageOneOf != 0 { - return canoto.ErrDuplicateOneOf - } - c.canotoData.messageOneOf = 5 - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { - return canoto.ErrZeroValue - } - - remainingBytes := r.B - r.B = msgBytes - c.Finalization = canoto.MakePointer(c.Finalization) - err = (c.Finalization).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - default: - return canoto.ErrUnknownField - } - - minField = field + 1 - } - return nil -} - -// ValidCanoto validates that the struct can be correctly marshaled into the -// Canoto format. -// -// Specifically, ValidCanoto ensures: -// 1. All OneOfs are specified at most once. -// 2. All strings are valid utf-8. -// 3. All custom fields are ValidCanoto. -func (c *Message) ValidCanoto() bool { - if c == nil { - return true - } - var ( - messageOneOf uint32 - ) - if c.Proposal != nil { - (c.Proposal).CalculateCanotoCache() - if (c.Proposal).CachedCanotoSize() != 0 { - if messageOneOf != 0 { - return false - } - messageOneOf = 1 - } - } - if c.Vote != nil { - (c.Vote).CalculateCanotoCache() - if (c.Vote).CachedCanotoSize() != 0 { - if messageOneOf != 0 { - return false - } - messageOneOf = 2 - } - } - if c.Notarization != nil { - (c.Notarization).CalculateCanotoCache() - if (c.Notarization).CachedCanotoSize() != 0 { - if messageOneOf != 0 { - return false - } - messageOneOf = 3 - } - } - if c.Finalize != nil { - (c.Finalize).CalculateCanotoCache() - if (c.Finalize).CachedCanotoSize() != 0 { - if messageOneOf != 0 { - return false - } - messageOneOf = 4 - } - } - if c.Finalization != nil { - (c.Finalization).CalculateCanotoCache() - if (c.Finalization).CachedCanotoSize() != 0 { - if messageOneOf != 0 { - return false - } - messageOneOf = 5 - } - } - if c.Proposal != nil && !(c.Proposal).ValidCanoto() { - return false - } - if c.Vote != nil && !(c.Vote).ValidCanoto() { - return false - } - if c.Notarization != nil && !(c.Notarization).ValidCanoto() { - return false - } - if c.Finalize != nil && !(c.Finalize).ValidCanoto() { - return false - } - if c.Finalization != nil && !(c.Finalization).ValidCanoto() { - return false - } - return true -} - -// CalculateCanotoCache populates size and OneOf caches based on the current -// values in the struct. -// -// It is not safe to call this function concurrently. -func (c *Message) CalculateCanotoCache() { - if c == nil { - return - } - c.canotoData.messageOneOf = 0 - c.canotoData.size = 0 - if c.Proposal != nil { - (c.Proposal).CalculateCanotoCache() - if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Message__Proposal__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - c.canotoData.messageOneOf = 1 - } - } - if c.Vote != nil { - (c.Vote).CalculateCanotoCache() - if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Message__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - c.canotoData.messageOneOf = 2 - } - } - if c.Notarization != nil { - (c.Notarization).CalculateCanotoCache() - if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Message__Notarization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - c.canotoData.messageOneOf = 3 - } - } - if c.Finalize != nil { - (c.Finalize).CalculateCanotoCache() - if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Message__Finalize__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - c.canotoData.messageOneOf = 4 - } - } - if c.Finalization != nil { - (c.Finalization).CalculateCanotoCache() - if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Message__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - c.canotoData.messageOneOf = 5 - } - } -} - -// CachedCanotoSize returns the previously calculated size of the Canoto -// representation from CalculateCanotoCache. -// -// If CalculateCanotoCache has not yet been called, it will return 0. -// -// If the struct has been modified since the last call to CalculateCanotoCache, -// the returned size may be incorrect. -func (c *Message) CachedCanotoSize() int { - if c == nil { - return 0 - } - return c.canotoData.size -} - -// CachedWhichOneOfmessage returns the previously calculated field number used -// to represent message. -// -// This field is cached by UnmarshalCanoto, UnmarshalCanotoFrom, and -// CalculateCanotoCache. -// -// If the field has not yet been cached, it will return 0. -// -// If the struct has been modified since the field was last cached, the returned -// field number may be incorrect. -func (c *Message) CachedWhichOneOfmessage() uint32 { - return c.canotoData.messageOneOf -} - -// MarshalCanoto returns the Canoto representation of this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *Message) MarshalCanoto() []byte { - c.CalculateCanotoCache() - w := canoto.Writer{ - B: make([]byte, 0, c.CachedCanotoSize()), - } - w = c.MarshalCanotoInto(w) - return w.B -} - -// MarshalCanotoInto writes the struct into a canoto.Writer and returns the -// resulting canoto.Writer. Most users should just use MarshalCanoto. -// -// It is assumed that CalculateCanotoCache has been called since the last -// modification to this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *Message) MarshalCanotoInto(w canoto.Writer) canoto.Writer { - if c == nil { - return w - } - if c.Proposal != nil { - if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Message__Proposal__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Proposal).MarshalCanotoInto(w) - } - } - if c.Vote != nil { - if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Message__Vote__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Vote).MarshalCanotoInto(w) - } - } - if c.Notarization != nil { - if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Message__Notarization__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Notarization).MarshalCanotoInto(w) - } - } - if c.Finalize != nil { - if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Message__Finalize__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Finalize).MarshalCanotoInto(w) - } - } - if c.Finalization != nil { - if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Message__Finalization__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (c.Finalization).MarshalCanotoInto(w) - } - } - return w -} - -const ( - canoto__BlockMessage__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) -) - -type canotoData_BlockMessage struct { - size int -} - -// MakeCanoto creates a new empty value. -func (*BlockMessage) MakeCanoto() *BlockMessage { - return new(BlockMessage) -} - -// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -func (c *BlockMessage) UnmarshalCanoto(bytes []byte) error { - r := canoto.Reader{ - B: bytes, - } - return c.UnmarshalCanotoFrom(r) -} - -// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users -// should just use UnmarshalCanoto. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -// -// This function enables configuration of reader options. -func (c *BlockMessage) UnmarshalCanotoFrom(r canoto.Reader) error { - var minField uint32 - for canoto.HasNext(&r) { - field, wireType, err := canoto.ReadTag(&r) - if err != nil { - return err - } - if field < minField { - return canoto.ErrInvalidFieldOrder - } - - switch field { - case 2: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - - originalUnsafe := r.Unsafe - r.Unsafe = true - var msgBytes []byte - err := canoto.ReadBytes(&r, &msgBytes) - r.Unsafe = originalUnsafe - if err != nil { - return err - } - if len(msgBytes) == 0 { + if len(msgBytes) == 0 { return canoto.ErrZeroValue } remainingBytes := r.B - r.B = msgBytes - err = (&c.Vote).UnmarshalCanotoFrom(r) - r.B = remainingBytes - if err != nil { - return err - } - default: - return canoto.ErrUnknownField - } - - minField = field + 1 - } - return nil -} - -// ValidCanoto validates that the struct can be correctly marshaled into the -// Canoto format. -// -// Specifically, ValidCanoto ensures: -// 1. All OneOfs are specified at most once. -// 2. All strings are valid utf-8. -// 3. All custom fields are ValidCanoto. -func (c *BlockMessage) ValidCanoto() bool { - if c == nil { - return true - } - if !(&c.Vote).ValidCanoto() { - return false - } - return true -} - -// CalculateCanotoCache populates size and OneOf caches based on the current -// values in the struct. -// -// It is not safe to call this function concurrently. -func (c *BlockMessage) CalculateCanotoCache() { - if c == nil { - return - } - c.canotoData.size = 0 - (&c.Vote).CalculateCanotoCache() - if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__BlockMessage__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - } -} - -// CachedCanotoSize returns the previously calculated size of the Canoto -// representation from CalculateCanotoCache. -// -// If CalculateCanotoCache has not yet been called, it will return 0. -// -// If the struct has been modified since the last call to CalculateCanotoCache, -// the returned size may be incorrect. -func (c *BlockMessage) CachedCanotoSize() int { - if c == nil { - return 0 - } - return c.canotoData.size -} - -// MarshalCanoto returns the Canoto representation of this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *BlockMessage) MarshalCanoto() []byte { - c.CalculateCanotoCache() - w := canoto.Writer{ - B: make([]byte, 0, c.CachedCanotoSize()), - } - w = c.MarshalCanotoInto(w) - return w.B -} - -// MarshalCanotoInto writes the struct into a canoto.Writer and returns the -// resulting canoto.Writer. Most users should just use MarshalCanoto. -// -// It is assumed that CalculateCanotoCache has been called since the last -// modification to this struct. -// -// It is assumed that this struct is ValidCanoto. -// -// It is not safe to call this function concurrently. -func (c *BlockMessage) MarshalCanotoInto(w canoto.Writer) canoto.Writer { - if c == nil { - return w - } - if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__BlockMessage__Vote__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Vote).MarshalCanotoInto(w) - } - return w -} - -const ( - canoto__Vote__Vote__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__Vote__Signature__tag = "\x12" // canoto.Tag(2, canoto.Len) -) - -type canotoData_Vote struct { - size int -} - -// MakeCanoto creates a new empty value. -func (*Vote) MakeCanoto() *Vote { - return new(Vote) -} - -// UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -func (c *Vote) UnmarshalCanoto(bytes []byte) error { - r := canoto.Reader{ - B: bytes, - } - return c.UnmarshalCanotoFrom(r) -} - -// UnmarshalCanotoFrom populates the struct from a canoto.Reader. Most users -// should just use UnmarshalCanoto. -// -// OneOf fields are cached during the unmarshaling process. -// -// The struct is not cleared before unmarshaling, any fields not present in the -// bytes will retain their previous values. If a OneOf field was previously -// cached as being set, attempting to unmarshal that OneOf again will return -// canoto.ErrDuplicateOneOf. -// -// This function enables configuration of reader options. -func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { - var minField uint32 - for canoto.HasNext(&r) { - field, wireType, err := canoto.ReadTag(&r) - if err != nil { - return err - } - if field < minField { - return canoto.ErrInvalidFieldOrder - } - - switch field { - case 1: + r.B = msgBytes + c.Notarization = canoto.MakePointer(c.Notarization) + err = (c.Notarization).UnmarshalCanotoFrom(r) + r.B = remainingBytes + if err != nil { + return err + } + case 4: if wireType != canoto.Len { return canoto.ErrUnexpectedWireType } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 4 originalUnsafe := r.Unsafe r.Unsafe = true @@ -1369,15 +478,20 @@ func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Vote).UnmarshalCanotoFrom(r) + c.Finalize = canoto.MakePointer(c.Finalize) + err = (c.Finalize).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err } - case 2: + case 5: if wireType != canoto.Len { return canoto.ErrUnexpectedWireType } + if c.canotoData.messageOneOf != 0 { + return canoto.ErrDuplicateOneOf + } + c.canotoData.messageOneOf = 5 originalUnsafe := r.Unsafe r.Unsafe = true @@ -1393,7 +507,8 @@ func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Signature).UnmarshalCanotoFrom(r) + c.Finalization = canoto.MakePointer(c.Finalization) + err = (c.Finalization).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err @@ -1414,14 +529,71 @@ func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { // 1. All OneOfs are specified at most once. // 2. All strings are valid utf-8. // 3. All custom fields are ValidCanoto. -func (c *Vote) ValidCanoto() bool { +func (c *Message) ValidCanoto() bool { if c == nil { return true } - if !(&c.Vote).ValidCanoto() { + var ( + messageOneOf uint32 + ) + if c.Proposal != nil { + (c.Proposal).CalculateCanotoCache() + if (c.Proposal).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 1 + } + } + if c.Vote != nil { + (c.Vote).CalculateCanotoCache() + if (c.Vote).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 2 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if (c.Notarization).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 3 + } + } + if c.Finalize != nil { + (c.Finalize).CalculateCanotoCache() + if (c.Finalize).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 4 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if (c.Finalization).CachedCanotoSize() != 0 { + if messageOneOf != 0 { + return false + } + messageOneOf = 5 + } + } + if c.Proposal != nil && !(c.Proposal).ValidCanoto() { return false } - if !(&c.Signature).ValidCanoto() { + if c.Vote != nil && !(c.Vote).ValidCanoto() { + return false + } + if c.Notarization != nil && !(c.Notarization).ValidCanoto() { + return false + } + if c.Finalize != nil && !(c.Finalize).ValidCanoto() { + return false + } + if c.Finalization != nil && !(c.Finalization).ValidCanoto() { return false } return true @@ -1431,18 +603,46 @@ func (c *Vote) ValidCanoto() bool { // values in the struct. // // It is not safe to call this function concurrently. -func (c *Vote) CalculateCanotoCache() { +func (c *Message) CalculateCanotoCache() { if c == nil { return } + c.canotoData.messageOneOf = 0 c.canotoData.size = 0 - (&c.Vote).CalculateCanotoCache() - if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Vote__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + if c.Proposal != nil { + (c.Proposal).CalculateCanotoCache() + if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Proposal__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 1 + } } - (&c.Signature).CalculateCanotoCache() - if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Vote__Signature__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + if c.Vote != nil { + (c.Vote).CalculateCanotoCache() + if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 2 + } + } + if c.Notarization != nil { + (c.Notarization).CalculateCanotoCache() + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Notarization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 3 + } + } + if c.Finalize != nil { + (c.Finalize).CalculateCanotoCache() + if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Finalize__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 4 + } + } + if c.Finalization != nil { + (c.Finalization).CalculateCanotoCache() + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Message__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.messageOneOf = 5 + } } } @@ -1453,19 +653,33 @@ func (c *Vote) CalculateCanotoCache() { // // If the struct has been modified since the last call to CalculateCanotoCache, // the returned size may be incorrect. -func (c *Vote) CachedCanotoSize() int { +func (c *Message) CachedCanotoSize() int { if c == nil { return 0 } return c.canotoData.size } +// CachedWhichOneOfmessage returns the previously calculated field number used +// to represent message. +// +// This field is cached by UnmarshalCanoto, UnmarshalCanotoFrom, and +// CalculateCanotoCache. +// +// If the field has not yet been cached, it will return 0. +// +// If the struct has been modified since the field was last cached, the returned +// field number may be incorrect. +func (c *Message) CachedWhichOneOfmessage() uint32 { + return c.canotoData.messageOneOf +} + // MarshalCanoto returns the Canoto representation of this struct. // // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Vote) MarshalCanoto() []byte { +func (c *Message) MarshalCanoto() []byte { c.CalculateCanotoCache() w := canoto.Writer{ B: make([]byte, 0, c.CachedCanotoSize()), @@ -1483,35 +697,60 @@ func (c *Vote) MarshalCanoto() []byte { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Vote) MarshalCanotoInto(w canoto.Writer) canoto.Writer { +func (c *Message) MarshalCanotoInto(w canoto.Writer) canoto.Writer { if c == nil { return w } - if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Vote__Vote__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Vote).MarshalCanotoInto(w) + if c.Proposal != nil { + if fieldSize := (c.Proposal).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Proposal__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Proposal).MarshalCanotoInto(w) + } } - if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Vote__Signature__tag) - canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Signature).MarshalCanotoInto(w) + if c.Vote != nil { + if fieldSize := (c.Vote).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Vote__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Vote).MarshalCanotoInto(w) + } + } + if c.Notarization != nil { + if fieldSize := (c.Notarization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Notarization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Notarization).MarshalCanotoInto(w) + } + } + if c.Finalize != nil { + if fieldSize := (c.Finalize).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Finalize__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Finalize).MarshalCanotoInto(w) + } + } + if c.Finalization != nil { + if fieldSize := (c.Finalization).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Message__Finalization__tag) + canoto.AppendInt(&w, int64(fieldSize)) + w = (c.Finalization).MarshalCanotoInto(w) + } } return w } const ( - canoto__Notarization__Vote__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__Notarization__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Proposal__Block__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Proposal__Vote__tag = "\x12" // canoto.Tag(2, canoto.Len) ) -type canotoData_Notarization struct { +type canotoData_Proposal struct { size int } // MakeCanoto creates a new empty value. -func (*Notarization) MakeCanoto() *Notarization { - return new(Notarization) +func (*Proposal) MakeCanoto() *Proposal { + return new(Proposal) } // UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. @@ -1522,7 +761,7 @@ func (*Notarization) MakeCanoto() *Notarization { // bytes will retain their previous values. If a OneOf field was previously // cached as being set, attempting to unmarshal that OneOf again will return // canoto.ErrDuplicateOneOf. -func (c *Notarization) UnmarshalCanoto(bytes []byte) error { +func (c *Proposal) UnmarshalCanoto(bytes []byte) error { r := canoto.Reader{ B: bytes, } @@ -1540,7 +779,7 @@ func (c *Notarization) UnmarshalCanoto(bytes []byte) error { // canoto.ErrDuplicateOneOf. // // This function enables configuration of reader options. -func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { +func (c *Proposal) UnmarshalCanotoFrom(r canoto.Reader) error { var minField uint32 for canoto.HasNext(&r) { field, wireType, err := canoto.ReadTag(&r) @@ -1557,6 +796,17 @@ func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { return canoto.ErrUnexpectedWireType } + if err := canoto.ReadBytes(&r, &c.Block); err != nil { + return err + } + if len(c.Block) == 0 { + return canoto.ErrZeroValue + } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + originalUnsafe := r.Unsafe r.Unsafe = true var msgBytes []byte @@ -1576,17 +826,6 @@ func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { if err != nil { return err } - case 2: - if wireType != canoto.Len { - return canoto.ErrUnexpectedWireType - } - - if err := canoto.ReadBytes(&r, &c.QC); err != nil { - return err - } - if len(c.QC) == 0 { - return canoto.ErrZeroValue - } default: return canoto.ErrUnknownField } @@ -1603,7 +842,7 @@ func (c *Notarization) UnmarshalCanotoFrom(r canoto.Reader) error { // 1. All OneOfs are specified at most once. // 2. All strings are valid utf-8. // 3. All custom fields are ValidCanoto. -func (c *Notarization) ValidCanoto() bool { +func (c *Proposal) ValidCanoto() bool { if c == nil { return true } @@ -1617,17 +856,17 @@ func (c *Notarization) ValidCanoto() bool { // values in the struct. // // It is not safe to call this function concurrently. -func (c *Notarization) CalculateCanotoCache() { +func (c *Proposal) CalculateCanotoCache() { if c == nil { return } c.canotoData.size = 0 + if len(c.Block) != 0 { + c.canotoData.size += len(canoto__Proposal__Block__tag) + canoto.SizeBytes(c.Block) + } (&c.Vote).CalculateCanotoCache() if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Notarization__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize - } - if len(c.QC) != 0 { - c.canotoData.size += len(canoto__Notarization__QC__tag) + canoto.SizeBytes(c.QC) + c.canotoData.size += len(canoto__Proposal__Vote__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize } } @@ -1638,7 +877,7 @@ func (c *Notarization) CalculateCanotoCache() { // // If the struct has been modified since the last call to CalculateCanotoCache, // the returned size may be incorrect. -func (c *Notarization) CachedCanotoSize() int { +func (c *Proposal) CachedCanotoSize() int { if c == nil { return 0 } @@ -1650,7 +889,7 @@ func (c *Notarization) CachedCanotoSize() int { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Notarization) MarshalCanoto() []byte { +func (c *Proposal) MarshalCanoto() []byte { c.CalculateCanotoCache() w := canoto.Writer{ B: make([]byte, 0, c.CachedCanotoSize()), @@ -1668,34 +907,34 @@ func (c *Notarization) MarshalCanoto() []byte { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Notarization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { +func (c *Proposal) MarshalCanotoInto(w canoto.Writer) canoto.Writer { if c == nil { return w } + if len(c.Block) != 0 { + canoto.Append(&w, canoto__Proposal__Block__tag) + canoto.AppendBytes(&w, c.Block) + } if fieldSize := (&c.Vote).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Notarization__Vote__tag) + canoto.Append(&w, canoto__Proposal__Vote__tag) canoto.AppendInt(&w, int64(fieldSize)) w = (&c.Vote).MarshalCanotoInto(w) } - if len(c.QC) != 0 { - canoto.Append(&w, canoto__Notarization__QC__tag) - canoto.AppendBytes(&w, c.QC) - } return w } const ( - canoto__Finalization__Finalization__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__Finalization__Signature__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Vote__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Vote__Signature__tag = "\x12" // canoto.Tag(2, canoto.Len) ) -type canotoData_Finalization struct { +type canotoData_Vote struct { size int } // MakeCanoto creates a new empty value. -func (*Finalization) MakeCanoto() *Finalization { - return new(Finalization) +func (*Vote) MakeCanoto() *Vote { + return new(Vote) } // UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. @@ -1706,7 +945,7 @@ func (*Finalization) MakeCanoto() *Finalization { // bytes will retain their previous values. If a OneOf field was previously // cached as being set, attempting to unmarshal that OneOf again will return // canoto.ErrDuplicateOneOf. -func (c *Finalization) UnmarshalCanoto(bytes []byte) error { +func (c *Vote) UnmarshalCanoto(bytes []byte) error { r := canoto.Reader{ B: bytes, } @@ -1724,7 +963,7 @@ func (c *Finalization) UnmarshalCanoto(bytes []byte) error { // canoto.ErrDuplicateOneOf. // // This function enables configuration of reader options. -func (c *Finalization) UnmarshalCanotoFrom(r canoto.Reader) error { +func (c *Vote) UnmarshalCanotoFrom(r canoto.Reader) error { var minField uint32 for canoto.HasNext(&r) { field, wireType, err := canoto.ReadTag(&r) @@ -1755,7 +994,7 @@ func (c *Finalization) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Finalization).UnmarshalCanotoFrom(r) + err = (&c.Header).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err @@ -1800,11 +1039,11 @@ func (c *Finalization) UnmarshalCanotoFrom(r canoto.Reader) error { // 1. All OneOfs are specified at most once. // 2. All strings are valid utf-8. // 3. All custom fields are ValidCanoto. -func (c *Finalization) ValidCanoto() bool { +func (c *Vote) ValidCanoto() bool { if c == nil { return true } - if !(&c.Finalization).ValidCanoto() { + if !(&c.Header).ValidCanoto() { return false } if !(&c.Signature).ValidCanoto() { @@ -1817,18 +1056,18 @@ func (c *Finalization) ValidCanoto() bool { // values in the struct. // // It is not safe to call this function concurrently. -func (c *Finalization) CalculateCanotoCache() { +func (c *Vote) CalculateCanotoCache() { if c == nil { return } c.canotoData.size = 0 - (&c.Finalization).CalculateCanotoCache() - if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Finalization__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + (&c.Header).CalculateCanotoCache() + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Vote__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize } (&c.Signature).CalculateCanotoCache() if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__Finalization__Signature__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + c.canotoData.size += len(canoto__Vote__Signature__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize } } @@ -1839,7 +1078,7 @@ func (c *Finalization) CalculateCanotoCache() { // // If the struct has been modified since the last call to CalculateCanotoCache, // the returned size may be incorrect. -func (c *Finalization) CachedCanotoSize() int { +func (c *Vote) CachedCanotoSize() int { if c == nil { return 0 } @@ -1851,7 +1090,7 @@ func (c *Finalization) CachedCanotoSize() int { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Finalization) MarshalCanoto() []byte { +func (c *Vote) MarshalCanoto() []byte { c.CalculateCanotoCache() w := canoto.Writer{ B: make([]byte, 0, c.CachedCanotoSize()), @@ -1869,17 +1108,17 @@ func (c *Finalization) MarshalCanoto() []byte { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *Finalization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { +func (c *Vote) MarshalCanotoInto(w canoto.Writer) canoto.Writer { if c == nil { return w } - if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Finalization__Finalization__tag) + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Vote__Header__tag) canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Finalization).MarshalCanotoInto(w) + w = (&c.Header).MarshalCanotoInto(w) } if fieldSize := (&c.Signature).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__Finalization__Signature__tag) + canoto.Append(&w, canoto__Vote__Signature__tag) canoto.AppendInt(&w, int64(fieldSize)) w = (&c.Signature).MarshalCanotoInto(w) } @@ -1887,16 +1126,17 @@ func (c *Finalization) MarshalCanotoInto(w canoto.Writer) canoto.Writer { } const ( - canoto__FinalizationCertificate__Finalization__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Quorum__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Quorum__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) ) -type canotoData_FinalizationCertificate struct { +type canotoData_Quorum struct { size int } // MakeCanoto creates a new empty value. -func (*FinalizationCertificate) MakeCanoto() *FinalizationCertificate { - return new(FinalizationCertificate) +func (*Quorum) MakeCanoto() *Quorum { + return new(Quorum) } // UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the struct. @@ -1907,7 +1147,7 @@ func (*FinalizationCertificate) MakeCanoto() *FinalizationCertificate { // bytes will retain their previous values. If a OneOf field was previously // cached as being set, attempting to unmarshal that OneOf again will return // canoto.ErrDuplicateOneOf. -func (c *FinalizationCertificate) UnmarshalCanoto(bytes []byte) error { +func (c *Quorum) UnmarshalCanoto(bytes []byte) error { r := canoto.Reader{ B: bytes, } @@ -1925,7 +1165,7 @@ func (c *FinalizationCertificate) UnmarshalCanoto(bytes []byte) error { // canoto.ErrDuplicateOneOf. // // This function enables configuration of reader options. -func (c *FinalizationCertificate) UnmarshalCanotoFrom(r canoto.Reader) error { +func (c *Quorum) UnmarshalCanotoFrom(r canoto.Reader) error { var minField uint32 for canoto.HasNext(&r) { field, wireType, err := canoto.ReadTag(&r) @@ -1956,11 +1196,22 @@ func (c *FinalizationCertificate) UnmarshalCanotoFrom(r canoto.Reader) error { remainingBytes := r.B r.B = msgBytes - err = (&c.Finalization).UnmarshalCanotoFrom(r) + err = (&c.Header).UnmarshalCanotoFrom(r) r.B = remainingBytes if err != nil { return err } + case 2: + if wireType != canoto.Len { + return canoto.ErrUnexpectedWireType + } + + if err := canoto.ReadBytes(&r, &c.QC); err != nil { + return err + } + if len(c.QC) == 0 { + return canoto.ErrZeroValue + } default: return canoto.ErrUnknownField } @@ -1977,11 +1228,11 @@ func (c *FinalizationCertificate) UnmarshalCanotoFrom(r canoto.Reader) error { // 1. All OneOfs are specified at most once. // 2. All strings are valid utf-8. // 3. All custom fields are ValidCanoto. -func (c *FinalizationCertificate) ValidCanoto() bool { +func (c *Quorum) ValidCanoto() bool { if c == nil { return true } - if !(&c.Finalization).ValidCanoto() { + if !(&c.Header).ValidCanoto() { return false } return true @@ -1991,14 +1242,17 @@ func (c *FinalizationCertificate) ValidCanoto() bool { // values in the struct. // // It is not safe to call this function concurrently. -func (c *FinalizationCertificate) CalculateCanotoCache() { +func (c *Quorum) CalculateCanotoCache() { if c == nil { return } c.canotoData.size = 0 - (&c.Finalization).CalculateCanotoCache() - if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { - c.canotoData.size += len(canoto__FinalizationCertificate__Finalization__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + (&c.Header).CalculateCanotoCache() + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + c.canotoData.size += len(canoto__Quorum__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize + } + if len(c.QC) != 0 { + c.canotoData.size += len(canoto__Quorum__QC__tag) + canoto.SizeBytes(c.QC) } } @@ -2009,7 +1263,7 @@ func (c *FinalizationCertificate) CalculateCanotoCache() { // // If the struct has been modified since the last call to CalculateCanotoCache, // the returned size may be incorrect. -func (c *FinalizationCertificate) CachedCanotoSize() int { +func (c *Quorum) CachedCanotoSize() int { if c == nil { return 0 } @@ -2021,7 +1275,7 @@ func (c *FinalizationCertificate) CachedCanotoSize() int { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *FinalizationCertificate) MarshalCanoto() []byte { +func (c *Quorum) MarshalCanoto() []byte { c.CalculateCanotoCache() w := canoto.Writer{ B: make([]byte, 0, c.CachedCanotoSize()), @@ -2039,14 +1293,18 @@ func (c *FinalizationCertificate) MarshalCanoto() []byte { // It is assumed that this struct is ValidCanoto. // // It is not safe to call this function concurrently. -func (c *FinalizationCertificate) MarshalCanotoInto(w canoto.Writer) canoto.Writer { +func (c *Quorum) MarshalCanotoInto(w canoto.Writer) canoto.Writer { if c == nil { return w } - if fieldSize := (&c.Finalization).CachedCanotoSize(); fieldSize != 0 { - canoto.Append(&w, canoto__FinalizationCertificate__Finalization__tag) + if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { + canoto.Append(&w, canoto__Quorum__Header__tag) canoto.AppendInt(&w, int64(fieldSize)) - w = (&c.Finalization).MarshalCanotoInto(w) + w = (&c.Header).MarshalCanotoInto(w) + } + if len(c.QC) != 0 { + canoto.Append(&w, canoto__Quorum__QC__tag) + canoto.AppendBytes(&w, c.QC) } return w } diff --git a/types.go b/types.go index 5d86866e..6a263603 100644 --- a/types.go +++ b/types.go @@ -14,69 +14,46 @@ import ( const ( digestLen = 32 digestFormatSize = 10 + + VoteContext = "ToBeSignedVote" + FinalizeContext = "ToBeSignedFinalization" ) type ( Record struct { - Block *BlockRecord `canoto:"pointer,1,record"` - Notarization *QuorumRecord `canoto:"pointer,2,record"` - Finalization *QuorumRecord `canoto:"pointer,3,record"` + Block []byte `canoto:"bytes,1,record"` + Notarization *Quorum `canoto:"pointer,2,record"` + Finalization *Quorum `canoto:"pointer,3,record"` canotoData canotoData_Record } - QuorumRecord struct { - Header BlockHeader `canoto:"value,1"` - QC []byte `canoto:"bytes,2"` - - canotoData canotoData_QuorumRecord - } - BlockRecord struct { - // TODO: Do we need to include the Digest? Or should we just include the - // ProtocolMetadata? - Header BlockHeader `canoto:"value,1"` - Payload []byte `canoto:"bytes,2"` - - canotoData canotoData_BlockRecord - } Message struct { - Proposal *BlockMessage `canoto:"pointer,1,message"` - Vote *Vote `canoto:"pointer,2,message"` - Notarization *Notarization `canoto:"pointer,3,message"` - Finalize *Finalization `canoto:"pointer,4,message"` - Finalization *FinalizationCertificate `canoto:"pointer,5,message"` + Proposal *Proposal `canoto:"pointer,1,message"` + Vote *Vote `canoto:"pointer,2,message"` + Notarization *Quorum `canoto:"pointer,3,message"` + Finalize *Vote `canoto:"pointer,4,message"` + Finalization *Quorum `canoto:"pointer,5,message"` canotoData canotoData_Message } - BlockMessage struct { - Block Block // `canoto:"bytes,1"` - Vote Vote `canoto:"value,2"` + Proposal struct { + Block []byte `canoto:"bytes,1"` + Vote Vote `canoto:"value,2"` - canotoData canotoData_BlockMessage + canotoData canotoData_Proposal } Vote struct { - Vote ToBeSignedVote `canoto:"value,1"` - Signature Signature `canoto:"value,2"` + Header BlockHeader `canoto:"value,1"` + Signature Signature `canoto:"value,2"` canotoData canotoData_Vote } - Notarization struct { - Vote ToBeSignedVote `canoto:"value,1"` - QC []byte `canoto:"bytes,2"` - - canotoData canotoData_Notarization - } - Finalization struct { - Finalization ToBeSignedFinalization `canoto:"value,1"` - Signature Signature `canoto:"value,2"` - - canotoData canotoData_Finalization - } - FinalizationCertificate struct { - Finalization ToBeSignedFinalization `canoto:"value,1"` - QC QuorumCertificate // `canoto:"bytes,2"` + Quorum struct { + Header BlockHeader `canoto:"value,1"` + QC []byte `canoto:"bytes,2"` - canotoData canotoData_FinalizationCertificate + canotoData canotoData_Quorum } SignedMessage struct { @@ -129,6 +106,41 @@ type ( NodeID []byte ) +func (v *Vote) Sign(nodeID NodeID, signer Signer, context string) error { + signature, err := signContext( + signer, + v.Header.MarshalCanoto(), + context, + ) + if err != nil { + return err + } + + v.Signature = Signature{ + Signer: nodeID, + Value: signature, + } + return nil +} + +func (v *Vote) Verify(verifier SignatureVerifier, context string) error { + return verifyContext( + v.Signature.Value, + verifier, + v.Header.MarshalCanoto(), + context, + v.Signature.Signer, + ) +} + +func (q *Quorum) Verify(p QCDeserializer, context string) error { + qc, err := p.DeserializeQuorumCertificate(q.QC) + if err != nil { + return err + } + return verifyContextQC(qc, q.Header.MarshalCanoto(), context) +} + func (b *BlockHeader) Equals(o *BlockHeader) bool { return b.Digest == o.Digest && b.ProtocolMetadata.Equals(&o.ProtocolMetadata) } @@ -148,3 +160,30 @@ func (n NodeID) String() string { func (n NodeID) Equals(o NodeID) bool { return bytes.Equal(n, o) } + +func signContext(signer Signer, msg []byte, context string) ([]byte, error) { + sm := SignedMessage{ + Payload: msg, + Context: context, + } + toBeSigned := sm.MarshalCanoto() + return signer.Sign(toBeSigned) +} + +func verifyContext(signature []byte, verifier SignatureVerifier, msg []byte, context string, signers NodeID) error { + sm := SignedMessage{ + Payload: msg, + Context: context, + } + toBeSigned := sm.MarshalCanoto() + return verifier.Verify(toBeSigned, signature, signers) +} + +func verifyContextQC(qc QuorumCertificate, msg []byte, context string) error { + sm := SignedMessage{ + Payload: msg, + Context: context, + } + toBeSigned := sm.MarshalCanoto() + return qc.Verify(toBeSigned) +} diff --git a/util_test.go b/util_test.go index 2314ccd1..1c5b5f6c 100644 --- a/util_test.go +++ b/util_test.go @@ -15,14 +15,14 @@ func TestRetrieveFromStorage(t *testing.T) { brokenStorage := newInMemStorage() brokenStorage.data[41] = struct { Block - FinalizationCertificate + Quorum }{Block: newTestBlock(ProtocolMetadata{Seq: 41})} block := newTestBlock(ProtocolMetadata{Seq: 0}) normalStorage := newInMemStorage() normalStorage.data[0] = struct { Block - FinalizationCertificate + Quorum }{Block: block} for _, testCase := range []struct { From 6b778f28eefb596a5e8e251f6b3752e1eb4f1e13 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 10:25:37 -0500 Subject: [PATCH 4/9] nit --- types.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/types.go b/types.go index 6a263603..2567846d 100644 --- a/types.go +++ b/types.go @@ -15,8 +15,8 @@ const ( digestLen = 32 digestFormatSize = 10 - VoteContext = "ToBeSignedVote" - FinalizeContext = "ToBeSignedFinalization" + VoteContext = "vote" + FinalizeContext = "finalize" ) type ( From 20845b6627d878518b09b9ea2955ca472a3882f6 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 10:29:16 -0500 Subject: [PATCH 5/9] Remove dependency --- epoch.canoto_test.go | 2 +- epoch_test.go | 2 +- go.mod | 1 - go.sum | 6 - internal/canoto/canoto.go | 506 ++++++++++++++++++++++++++++++++++++++ types.canoto.go | 2 +- types.go | 2 +- 7 files changed, 510 insertions(+), 11 deletions(-) create mode 100644 internal/canoto/canoto.go diff --git a/epoch.canoto_test.go b/epoch.canoto_test.go index 0dd120d8..44ef6b01 100644 --- a/epoch.canoto_test.go +++ b/epoch.canoto_test.go @@ -10,7 +10,7 @@ import ( "sync/atomic" "unicode/utf8" - "github.com/StephenButtolph/canoto" + "simplex/internal/canoto" ) // Ensure that unused imports do not error diff --git a/epoch_test.go b/epoch_test.go index 9efb2c9b..e284baf7 100644 --- a/epoch_test.go +++ b/epoch_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false $GOFILE +//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false --import=simplex/internal/canoto $GOFILE package simplex_test diff --git a/go.mod b/go.mod index 29871b65..33367e60 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,6 @@ module simplex go 1.23 require ( - github.com/StephenButtolph/canoto v0.10.0 github.com/stretchr/testify v1.10.0 go.uber.org/zap v1.26.0 ) diff --git a/go.sum b/go.sum index 83930439..d809e1db 100644 --- a/go.sum +++ b/go.sum @@ -1,15 +1,9 @@ -github.com/StephenButtolph/canoto v0.10.0 h1:KdW85TYQXH+gwR8vOxfOUf28TRpkLU+X06Kycg1IR7s= -github.com/StephenButtolph/canoto v0.10.0/go.mod h1:MxppdgKRApRBvIg4ZgO2e14m/NSBjFMuydy97OB/gYY= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/sanity-io/litter v1.5.1 h1:dwnrSypP6q56o3lFxTU+t2fwQ9A+U5qrXVO4Qg9KwVU= -github.com/sanity-io/litter v1.5.1/go.mod h1:5Z71SvaYy5kcGtyglXOC9rrUi3c1E8CamFWjQsazTh0= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -github.com/thepudds/fzgen v0.4.3 h1:srUP/34BulQaEwPP/uHZkdjUcUjIzL7Jkf4CBVryiP8= -github.com/thepudds/fzgen v0.4.3/go.mod h1:BhhwtRhzgvLWAjjcHDJ9pEiLD2Z9hrVIFjBCHJ//zJ4= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= diff --git a/internal/canoto/canoto.go b/internal/canoto/canoto.go new file mode 100644 index 00000000..dd237112 --- /dev/null +++ b/internal/canoto/canoto.go @@ -0,0 +1,506 @@ +// Code generated by canoto. DO NOT EDIT. +// versions: +// canoto v0.10.0 + +// Canoto provides common functionality required for reading and writing the +// canoto format. +package canoto + +import ( + "encoding/binary" + "errors" + "io" + "math/bits" + "slices" + "unicode/utf8" + "unsafe" + + _ "embed" +) + +const ( + Varint WireType = iota + I64 + Len + _ // SGROUP is deprecated and not supported + _ // EGROUP is deprecated and not supported + I32 + + // SizeFint32 is the size of a 32-bit fixed size integer in bytes. + SizeFint32 = 4 + // SizeFint64 is the size of a 64-bit fixed size integer in bytes. + SizeFint64 = 8 + // SizeBool is the size of a boolean in bytes. + SizeBool = 1 + + // MaxFieldNumber is the maximum field number allowed to be used in a Tag. + MaxFieldNumber = 1<<29 - 1 + + // Version is the current version of the canoto library. + Version = "v0.10.0" + + wireTypeLength = 3 + wireTypeMask = 0x07 + + falseByte = 0 + trueByte = 1 + continuationMask = 0x80 +) + +var ( + // Code is the actual golang code for this library; including this comment. + // + // This variable is not used internally, so the compiler is smart enough to + // omit this value from the binary if the user of this library does not + // utilize this variable; at least at the time of writing. + // + // This can be used during codegen to generate this library. + // + //go:embed canoto.go + Code string + + ErrInvalidFieldOrder = errors.New("invalid field order") + ErrUnexpectedWireType = errors.New("unexpected wire type") + ErrDuplicateOneOf = errors.New("duplicate oneof field") + ErrInvalidLength = errors.New("decoded length is invalid") + ErrZeroValue = errors.New("zero value") + ErrUnknownField = errors.New("unknown field") + ErrPaddedZeroes = errors.New("padded zeroes") + + ErrOverflow = errors.New("overflow") + ErrInvalidWireType = errors.New("invalid wire type") + ErrInvalidBool = errors.New("decoded bool is neither true nor false") + ErrStringNotUTF8 = errors.New("decoded string is not UTF-8") +) + +type ( + Sint interface { + ~int8 | ~int16 | ~int32 | ~int64 + } + Uint interface { + ~uint8 | ~uint16 | ~uint32 | ~uint64 + } + Int interface{ Sint | Uint } + Int32 interface{ ~int32 | ~uint32 } + Int64 interface{ ~int64 | ~uint64 } + Bytes interface{ ~string | ~[]byte } + + // Message defines a type that can be a stand-alone Canoto message. + Message interface { + Field + // MarshalCanoto returns the Canoto representation of this message. + // + // It is assumed that this message is ValidCanoto. + MarshalCanoto() []byte + // UnmarshalCanoto unmarshals a Canoto-encoded byte slice into the message. + UnmarshalCanoto(bytes []byte) error + } + + // Field defines a type that can be included inside of a Canoto message. + Field interface { + // MarshalCanotoInto writes the field into a canoto.Writer and returns + // the resulting canoto.Writer. + // + // It is assumed that CalculateCanotoCache has been called since the + // last modification to this field. + // + // It is assumed that this field is ValidCanoto. + MarshalCanotoInto(w Writer) Writer + // CalculateCanotoCache populates internal caches based on the current + // values in the struct. + CalculateCanotoCache() + // CachedCanotoSize returns the previously calculated size of the Canoto + // representation from CalculateCanotoCache. + // + // If CalculateCanotoCache has not yet been called, or the field has + // been modified since the last call to CalculateCanotoCache, the + // returned size may be incorrect. + CachedCanotoSize() int + // UnmarshalCanotoFrom populates the field from a canoto.Reader. + UnmarshalCanotoFrom(r Reader) error + // ValidCanoto validates that the field can be correctly marshaled into + // the Canoto format. + ValidCanoto() bool + } + + // FieldPointer is a pointer to a concrete Field value T. + // + // This type must be used when implementing a value for a generic Field. + FieldPointer[T any] interface { + Field + *T + } + + // FieldMaker is a Field that can create a new value of type T. + // + // The returned value must be able to be unmarshaled into. + // + // This type can be used when implementing a generic Field. However, if T is + // an interface, it is possible for generated code to compile and panic at + // runtime. + FieldMaker[T any] interface { + Field + MakeCanoto() T + } + + // WireType represents the Proto wire description of a field. Within Proto + // it is used to provide forwards compatibility. For Canoto, it exists to + // provide compatibility with Proto. + WireType byte + + // Reader contains all the state needed to unmarshal a Canoto type. + // + // The functions in this package are not methods on the Reader type to + // enable the usage of generics. + Reader struct { + B []byte + Unsafe bool + } + + // Writer contains all the state needed to marshal a Canoto type. + // + // The functions in this package are not methods on the Writer type to + // enable the usage of generics. + Writer struct { + B []byte + } +) + +func (w WireType) IsValid() bool { + switch w { + case Varint, I64, Len, I32: + return true + default: + return false + } +} + +func (w WireType) String() string { + switch w { + case Varint: + return "Varint" + case I64: + return "I64" + case Len: + return "Len" + case I32: + return "I32" + default: + return "Invalid" + } +} + +// HasNext returns true if there are more bytes to read. +func HasNext(r *Reader) bool { + return len(r.B) > 0 +} + +// Append writes unprefixed bytes to the writer. +func Append[T Bytes](w *Writer, v T) { + w.B = append(w.B, v...) +} + +// Tag calculates the tag for a field number and wire type. +// +// This function should not typically be used during marshaling, as tags can be +// precomputed. +func Tag(fieldNumber uint32, wireType WireType) []byte { + w := Writer{} + AppendInt(&w, fieldNumber<> wireTypeLength, wireType, nil +} + +// SizeInt calculates the size of an integer when encoded as a varint. +func SizeInt[T Int](v T) int { + if v == 0 { + return 1 + } + return (bits.Len64(uint64(v)) + 6) / 7 +} + +// CountInts counts the number of varints that are encoded in bytes. +func CountInts(bytes []byte) int { + var count int + for _, b := range bytes { + if b < continuationMask { + count++ + } + } + return count +} + +// ReadInt reads a varint encoded integer from the reader. +func ReadInt[T Int](r *Reader, v *T) error { + val, bytesRead := binary.Uvarint(r.B) + switch { + case bytesRead == 0: + return io.ErrUnexpectedEOF + case bytesRead < 0 || uint64(T(val)) != val: + return ErrOverflow + // To ensure decoding is canonical, we check for padded zeroes in the + // varint. + // The last byte of the varint includes the most significant bits. + // If the last byte is 0, then the number should have been encoded more + // efficiently by removing this zero. + case bytesRead > 1 && r.B[bytesRead-1] == 0x00: + return ErrPaddedZeroes + default: + r.B = r.B[bytesRead:] + *v = T(val) + return nil + } +} + +// AppendInt writes an integer to the writer as a varint. +func AppendInt[T Int](w *Writer, v T) { + w.B = binary.AppendUvarint(w.B, uint64(v)) +} + +// SizeSint calculates the size of an integer when zigzag encoded as a varint. +func SizeSint[T Sint](v T) int { + if v == 0 { + return 1 + } + + var uv uint64 + if v > 0 { + uv = uint64(v) << 1 + } else { + uv = ^uint64(v)<<1 | 1 + } + return (bits.Len64(uv) + 6) / 7 +} + +// ReadSint reads a zigzag encoded integer from the reader. +func ReadSint[T Sint](r *Reader, v *T) error { + var largeVal uint64 + if err := ReadInt(r, &largeVal); err != nil { + return err + } + + uVal := largeVal >> 1 + val := T(uVal) + // If T is an int32, it's possible that some bits were truncated during the + // cast. In this case, casting back to uint64 would result in a different + // value. + if uint64(val) != uVal { + return ErrOverflow + } + + if largeVal&1 != 0 { + val = ^val + } + *v = val + return nil +} + +// AppendSint writes an integer to the writer as a zigzag encoded varint. +func AppendSint[T Sint](w *Writer, v T) { + if v >= 0 { + w.B = binary.AppendUvarint(w.B, uint64(v)<<1) + } else { + w.B = binary.AppendUvarint(w.B, ^uint64(v)<<1|1) + } +} + +// ReadFint32 reads a 32-bit fixed size integer from the reader. +func ReadFint32[T Int32](r *Reader, v *T) error { + if len(r.B) < SizeFint32 { + return io.ErrUnexpectedEOF + } + + *v = T(binary.LittleEndian.Uint32(r.B)) + r.B = r.B[SizeFint32:] + return nil +} + +// AppendFint32 writes a 32-bit fixed size integer to the writer. +func AppendFint32[T Int32](w *Writer, v T) { + w.B = binary.LittleEndian.AppendUint32(w.B, uint32(v)) +} + +// ReadFint64 reads a 64-bit fixed size integer from the reader. +func ReadFint64[T Int64](r *Reader, v *T) error { + if len(r.B) < SizeFint64 { + return io.ErrUnexpectedEOF + } + + *v = T(binary.LittleEndian.Uint64(r.B)) + r.B = r.B[SizeFint64:] + return nil +} + +// AppendFint64 writes a 64-bit fixed size integer to the writer. +func AppendFint64[T Int64](w *Writer, v T) { + w.B = binary.LittleEndian.AppendUint64(w.B, uint64(v)) +} + +// ReadBool reads a boolean from the reader. +func ReadBool[T ~bool](r *Reader, v *T) error { + switch { + case len(r.B) < SizeBool: + return io.ErrUnexpectedEOF + case r.B[0] > trueByte: + return ErrInvalidBool + default: + *v = r.B[0] == trueByte + r.B = r.B[SizeBool:] + return nil + } +} + +// AppendBool writes a boolean to the writer. +func AppendBool[T ~bool](w *Writer, b T) { + if b { + w.B = append(w.B, trueByte) + } else { + w.B = append(w.B, falseByte) + } +} + +// SizeBytes calculates the size the length-prefixed bytes would take if +// written. +func SizeBytes[T Bytes](v T) int { + return SizeInt(int64(len(v))) + len(v) +} + +// CountBytes counts the consecutive number of length-prefixed fields with the +// given tag. +func CountBytes(bytes []byte, tag string) (int, error) { + var ( + r = Reader{B: bytes} + count = 0 + ) + for HasPrefix(r.B, tag) { + r.B = r.B[len(tag):] + var length int64 + if err := ReadInt(&r, &length); err != nil { + return 0, err + } + if length < 0 { + return 0, ErrInvalidLength + } + if length > int64(len(r.B)) { + return 0, io.ErrUnexpectedEOF + } + r.B = r.B[length:] + count++ + } + return count, nil +} + +// HasPrefix returns true if the bytes start with the given prefix. +func HasPrefix(bytes []byte, prefix string) bool { + return len(bytes) >= len(prefix) && string(bytes[:len(prefix)]) == prefix +} + +// ReadString reads a string from the reader. The string is verified to be valid +// UTF-8. +func ReadString[T ~string](r *Reader, v *T) error { + var length int64 + if err := ReadInt[int64](r, &length); err != nil { + return err + } + if length < 0 { + return ErrInvalidLength + } + if length > int64(len(r.B)) { + return io.ErrUnexpectedEOF + } + + bytes := r.B[:length] + if !utf8.Valid(bytes) { + return ErrStringNotUTF8 + } + + r.B = r.B[length:] + if r.Unsafe { + *v = T(unsafeString(bytes)) + } else { + *v = T(bytes) + } + return nil +} + +// ReadBytes reads a byte slice from the reader. +func ReadBytes[T ~[]byte](r *Reader, v *T) error { + var length int64 + if err := ReadInt[int64](r, &length); err != nil { + return err + } + if length < 0 { + return ErrInvalidLength + } + if length > int64(len(r.B)) { + return io.ErrUnexpectedEOF + } + + bytes := r.B[:length] + r.B = r.B[length:] + if !r.Unsafe { + bytes = slices.Clone(bytes) + } + *v = T(bytes) + return nil +} + +// AppendBytes writes a length-prefixed byte slice to the writer. +func AppendBytes[T Bytes](w *Writer, v T) { + AppendInt(w, int64(len(v))) + w.B = append(w.B, v...) +} + +// MakePointer creates a new pointer. It is equivalent to `new(T)`. +// +// This function is useful to use in auto-generated code, when the type of a +// variable is unknown. For example, if we have a variable `v` which we know to +// be a pointer, but we do not know the type of the pointer, we can use this +// function to leverage golang's type inference to create the new pointer. +func MakePointer[T any](_ *T) *T { + return new(T) +} + +// MakeSlice creates a new slice with the given length. It is equivalent to +// `make([]T, length)`. +// +// This function is useful to use in auto-generated code, when the type of a +// variable is unknown. For example, if we have a variable `v` which we know to +// be a slice, but we do not know the type of the elements, we can use this +// function to leverage golang's type inference to create the new slice. +func MakeSlice[T any](_ []T, length int) []T { + return make([]T, length) +} + +// Zero returns the zero value for its type. +func Zero[T any](_ T) (_ T) { + return +} + +// IsZero returns true if the value is the zero value for its type. +func IsZero[T comparable](v T) bool { + var zero T + return v == zero +} + +// unsafeString converts a []byte to an unsafe string. +// +// Invariant: The input []byte must not be modified. +func unsafeString(b []byte) string { + // avoid copying during the conversion + return unsafe.String(unsafe.SliceData(b), len(b)) +} diff --git a/types.canoto.go b/types.canoto.go index ca8ef8e1..a88554f3 100644 --- a/types.canoto.go +++ b/types.canoto.go @@ -10,7 +10,7 @@ import ( "sync/atomic" "unicode/utf8" - "github.com/StephenButtolph/canoto" + "simplex/internal/canoto" ) // Ensure that unused imports do not error diff --git a/types.go b/types.go index 2567846d..25159808 100644 --- a/types.go +++ b/types.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false $GOFILE +//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --library=./internal --concurrent=false --import=simplex/internal/canoto $GOFILE package simplex From 3fea3def55882f07483c278c1ee0212e5888d195 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 10:34:48 -0500 Subject: [PATCH 6/9] reduce diff --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 33367e60..d4b198a6 100644 --- a/go.mod +++ b/go.mod @@ -3,7 +3,7 @@ module simplex go 1.23 require ( - github.com/stretchr/testify v1.10.0 + github.com/stretchr/testify v1.9.0 go.uber.org/zap v1.26.0 ) diff --git a/go.sum b/go.sum index d809e1db..56d9d664 100644 --- a/go.sum +++ b/go.sum @@ -2,8 +2,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= -github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= go.uber.org/multierr v1.10.0 h1:S0h4aNzvfcFsC3dRF1jLoaov7oRaKqRGC/pUEJ2yvPQ= From 1683857d9927798f2b187148f86f2d4812ee00c1 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 10:53:35 -0500 Subject: [PATCH 7/9] nit --- epoch_multinode_test.go | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/epoch_multinode_test.go b/epoch_multinode_test.go index f84c507c..74f0b802 100644 --- a/epoch_multinode_test.go +++ b/epoch_multinode_test.go @@ -63,6 +63,7 @@ func newSimplexNode(t *testing.T, id uint8, net *inMemNetwork, bb BlockBuilder) conf := EpochConfig{ Comm: &testComm{ + t: t, from: nodeID, net: net, }, @@ -159,6 +160,7 @@ func (tw *testWAL) Append(b []byte) error { } type testComm struct { + t testing.TB from NodeID net *inMemNetwork } @@ -180,15 +182,25 @@ func (c *testComm) SendMessage(msg *Message, destination NodeID) { } func (c *testComm) Broadcast(msg *Message) { + msgBytes := msg.MarshalCanoto() + for _, instance := range c.net.instances { // Skip sending the message to yourself if bytes.Equal(c.from, instance.e.ID) { continue } + + // Copy the message before sending it to avoid racy modifications + var peerMessage Message + require.NoError(c.t, peerMessage.UnmarshalCanoto(msgBytes)) + instance.ingress <- struct { msg *Message from NodeID - }{msg: msg, from: c.from} + }{ + msg: &peerMessage, + from: c.from, + } } } From 6a2ebd90c520efbc23ca56e170337c21cee66008 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 11:08:35 -0500 Subject: [PATCH 8/9] nit --- types.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/types.go b/types.go index 25159808..9bfcc004 100644 --- a/types.go +++ b/types.go @@ -170,13 +170,13 @@ func signContext(signer Signer, msg []byte, context string) ([]byte, error) { return signer.Sign(toBeSigned) } -func verifyContext(signature []byte, verifier SignatureVerifier, msg []byte, context string, signers NodeID) error { +func verifyContext(signature []byte, verifier SignatureVerifier, msg []byte, context string, signer NodeID) error { sm := SignedMessage{ Payload: msg, Context: context, } toBeSigned := sm.MarshalCanoto() - return verifier.Verify(toBeSigned, signature, signers) + return verifier.Verify(toBeSigned, signature, signer) } func verifyContextQC(qc QuorumCertificate, msg []byte, context string) error { From 0b9bc36aa69970d33628f25f030645c867966c6e Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Sat, 11 Jan 2025 11:20:34 -0500 Subject: [PATCH 9/9] nits --- encoding_test.go | 12 ++++++------ epoch.go | 10 +++++----- types.canoto.go | 18 +++++++++--------- types.go | 8 ++++---- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/encoding_test.go b/encoding_test.go index 1cd6a4a8..e34e6f9b 100644 --- a/encoding_test.go +++ b/encoding_test.go @@ -32,8 +32,8 @@ func TestNotarizationRecord(t *testing.T) { record := Record{ Notarization: &Quorum{ - Header: vote, - QC: []byte{1, 2, 3}, + Header: vote, + Certificate: []byte{1, 2, 3}, }, } recordBytes := record.MarshalCanoto() @@ -41,7 +41,7 @@ func TestNotarizationRecord(t *testing.T) { var parsedRecord Record require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) require.True(t, vote.Equals(&parsedRecord.Notarization.Header)) - require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) + require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.Certificate) } func FuzzNotarizationRecord(f *testing.F) { @@ -61,8 +61,8 @@ func FuzzNotarizationRecord(f *testing.F) { record := Record{ Notarization: &Quorum{ - Header: vote, - QC: []byte{1, 2, 3}, + Header: vote, + Certificate: []byte{1, 2, 3}, }, } recordBytes := record.MarshalCanoto() @@ -70,6 +70,6 @@ func FuzzNotarizationRecord(f *testing.F) { var parsedRecord Record require.NoError(t, parsedRecord.UnmarshalCanoto(recordBytes)) require.True(t, vote.Equals(&parsedRecord.Notarization.Header)) - require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.QC) + require.Equal(t, []byte{1, 2, 3}, parsedRecord.Notarization.Certificate) }) } diff --git a/epoch.go b/epoch.go index d9d587d6..989d1fc0 100644 --- a/epoch.go +++ b/epoch.go @@ -213,7 +213,7 @@ func (e *Epoch) isFinalizationCertificateValid(finalization *Quorum) (bool, erro } func (e *Epoch) validateFinalizationQC(finalization *Quorum) (bool, error) { - qc, err := e.QCDeserializer.DeserializeQuorumCertificate(finalization.QC) + qc, err := e.QCDeserializer.DeserializeQuorumCertificate(finalization.Certificate) if err != nil { e.Logger.Debug("FinalizationCertificate QC failed to be parsed", zap.Error(err), @@ -383,8 +383,8 @@ func (e *Epoch) assembleFinalizationCertificate(round *Round) error { } finalization := Quorum{ - Header: finalize.Header, - QC: qc.Bytes(), + Header: finalize.Header, + Certificate: qc.Bytes(), } round.finalization = &finalization @@ -497,8 +497,8 @@ func (e *Epoch) assembleNotarization(votesForCurrentRound map[string]*Vote, dige } notarization := Quorum{ - Header: vote, - QC: qc.Bytes(), + Header: vote, + Certificate: qc.Bytes(), } if err := e.storeNotarization(notarization); err != nil { return err diff --git a/types.canoto.go b/types.canoto.go index a88554f3..fb1a1943 100644 --- a/types.canoto.go +++ b/types.canoto.go @@ -1126,8 +1126,8 @@ func (c *Vote) MarshalCanotoInto(w canoto.Writer) canoto.Writer { } const ( - canoto__Quorum__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) - canoto__Quorum__QC__tag = "\x12" // canoto.Tag(2, canoto.Len) + canoto__Quorum__Header__tag = "\x0a" // canoto.Tag(1, canoto.Len) + canoto__Quorum__Certificate__tag = "\x12" // canoto.Tag(2, canoto.Len) ) type canotoData_Quorum struct { @@ -1206,10 +1206,10 @@ func (c *Quorum) UnmarshalCanotoFrom(r canoto.Reader) error { return canoto.ErrUnexpectedWireType } - if err := canoto.ReadBytes(&r, &c.QC); err != nil { + if err := canoto.ReadBytes(&r, &c.Certificate); err != nil { return err } - if len(c.QC) == 0 { + if len(c.Certificate) == 0 { return canoto.ErrZeroValue } default: @@ -1251,8 +1251,8 @@ func (c *Quorum) CalculateCanotoCache() { if fieldSize := (&c.Header).CachedCanotoSize(); fieldSize != 0 { c.canotoData.size += len(canoto__Quorum__Header__tag) + canoto.SizeInt(int64(fieldSize)) + fieldSize } - if len(c.QC) != 0 { - c.canotoData.size += len(canoto__Quorum__QC__tag) + canoto.SizeBytes(c.QC) + if len(c.Certificate) != 0 { + c.canotoData.size += len(canoto__Quorum__Certificate__tag) + canoto.SizeBytes(c.Certificate) } } @@ -1302,9 +1302,9 @@ func (c *Quorum) MarshalCanotoInto(w canoto.Writer) canoto.Writer { canoto.AppendInt(&w, int64(fieldSize)) w = (&c.Header).MarshalCanotoInto(w) } - if len(c.QC) != 0 { - canoto.Append(&w, canoto__Quorum__QC__tag) - canoto.AppendBytes(&w, c.QC) + if len(c.Certificate) != 0 { + canoto.Append(&w, canoto__Quorum__Certificate__tag) + canoto.AppendBytes(&w, c.Certificate) } return w } diff --git a/types.go b/types.go index 9bfcc004..266e37e1 100644 --- a/types.go +++ b/types.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --library=./internal --concurrent=false --import=simplex/internal/canoto $GOFILE +//go:generate go run github.com/StephenButtolph/canoto/canoto@v0.10.0 --concurrent=false --import=simplex/internal/canoto --library=./internal $GOFILE package simplex @@ -50,8 +50,8 @@ type ( canotoData canotoData_Vote } Quorum struct { - Header BlockHeader `canoto:"value,1"` - QC []byte `canoto:"bytes,2"` + Header BlockHeader `canoto:"value,1"` + Certificate []byte `canoto:"bytes,2"` canotoData canotoData_Quorum } @@ -134,7 +134,7 @@ func (v *Vote) Verify(verifier SignatureVerifier, context string) error { } func (q *Quorum) Verify(p QCDeserializer, context string) error { - qc, err := p.DeserializeQuorumCertificate(q.QC) + qc, err := p.DeserializeQuorumCertificate(q.Certificate) if err != nil { return err }