mirror of
				https://source.quilibrium.com/quilibrium/ceremonyclient.git
				synced 2025-04-04 19:36:54 +00:00 
			
		
		
		
	v1.2.4 (#43)
This commit is contained in:
		
							parent
							
								
									1a1ef542d0
								
							
						
					
					
						commit
						0cd6b41f5d
					
				| @ -23,7 +23,7 @@ import ( | ||||
| 	"google.golang.org/grpc/connectivity" | ||||
| 	"google.golang.org/grpc/credentials/insecure" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| @ -886,11 +886,11 @@ func logoVersion(width int) string { | ||||
| 		out += "                     #######################################        ########\n" | ||||
| 		out += "                          #############################                ##\n" | ||||
| 		out += " \n" | ||||
| 		out += "                         Quilibrium Node - v1.2.3 – Dawn\n" | ||||
| 		out += "                         Quilibrium Node - v1.2.4 – Dawn\n" | ||||
| 		out += " \n" | ||||
| 		out += "                                   DB Console\n" | ||||
| 	} else { | ||||
| 		out = "Quilibrium Node - v1.2.3 – Dawn - DB Console\n" | ||||
| 		out = "Quilibrium Node - v1.2.4 – Dawn - DB Console\n" | ||||
| 	} | ||||
| 	return out | ||||
| } | ||||
|  | ||||
| @ -6,7 +6,7 @@ import ( | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
|  | ||||
| @ -9,7 +9,9 @@ import ( | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/master" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| @ -54,6 +56,11 @@ var pubSubSet = wire.NewSet( | ||||
| 
 | ||||
| var engineSet = wire.NewSet( | ||||
| 	wire.FieldsOf(new(*config.Config), "Engine"), | ||||
| 	crypto.NewWesolowskiFrameProver, | ||||
| 	wire.Bind(new(crypto.FrameProver), new(*crypto.WesolowskiFrameProver)), | ||||
| 	crypto.NewKZGInclusionProver, | ||||
| 	wire.Bind(new(crypto.InclusionProver), new(*crypto.KZGInclusionProver)), | ||||
| 	time.NewMasterTimeReel, | ||||
| 	ceremony.NewCeremonyExecutionEngine, | ||||
| ) | ||||
| 
 | ||||
|  | ||||
| @ -12,7 +12,9 @@ import ( | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/master" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| @ -30,9 +32,12 @@ func NewNode(configConfig *config.Config) (*Node, error) { | ||||
| 	p2PConfig := configConfig.P2P | ||||
| 	blossomSub := p2p.NewBlossomSub(p2PConfig, zapLogger) | ||||
| 	engineConfig := configConfig.Engine | ||||
| 	wesolowskiFrameProver := crypto.NewWesolowskiFrameProver(zapLogger) | ||||
| 	kzgInclusionProver := crypto.NewKZGInclusionProver(zapLogger) | ||||
| 	masterTimeReel := time.NewMasterTimeReel(zapLogger, pebbleClockStore, engineConfig, wesolowskiFrameProver) | ||||
| 	pebbleKeyStore := store.NewPebbleKeyStore(pebbleDB, zapLogger) | ||||
| 	ceremonyExecutionEngine := ceremony.NewCeremonyExecutionEngine(zapLogger, engineConfig, fileKeyManager, blossomSub, pebbleClockStore, pebbleKeyStore) | ||||
| 	masterClockConsensusEngine := master.NewMasterClockConsensusEngine(engineConfig, zapLogger, pebbleClockStore, fileKeyManager, blossomSub) | ||||
| 	ceremonyExecutionEngine := ceremony.NewCeremonyExecutionEngine(zapLogger, engineConfig, fileKeyManager, blossomSub, wesolowskiFrameProver, kzgInclusionProver, pebbleClockStore, masterTimeReel, pebbleKeyStore) | ||||
| 	masterClockConsensusEngine := master.NewMasterClockConsensusEngine(engineConfig, zapLogger, pebbleClockStore, fileKeyManager, blossomSub, wesolowskiFrameProver, masterTimeReel) | ||||
| 	node, err := newNode(zapLogger, pebbleClockStore, fileKeyManager, blossomSub, ceremonyExecutionEngine, masterClockConsensusEngine) | ||||
| 	if err != nil { | ||||
| 		return nil, err | ||||
| @ -77,7 +82,7 @@ var storeSet = wire.NewSet(wire.FieldsOf(new(*config.Config), "DB"), store.NewPe | ||||
| 
 | ||||
| var pubSubSet = wire.NewSet(wire.FieldsOf(new(*config.Config), "P2P"), p2p.NewBlossomSub, wire.Bind(new(p2p.PubSub), new(*p2p.BlossomSub))) | ||||
| 
 | ||||
| var engineSet = wire.NewSet(wire.FieldsOf(new(*config.Config), "Engine"), ceremony.NewCeremonyExecutionEngine) | ||||
| var engineSet = wire.NewSet(wire.FieldsOf(new(*config.Config), "Engine"), crypto.NewWesolowskiFrameProver, wire.Bind(new(crypto.FrameProver), new(*crypto.WesolowskiFrameProver)), crypto.NewKZGInclusionProver, wire.Bind(new(crypto.InclusionProver), new(*crypto.KZGInclusionProver)), time.NewMasterTimeReel, ceremony.NewCeremonyExecutionEngine) | ||||
| 
 | ||||
| var consensusSet = wire.NewSet(master.NewMasterClockConsensusEngine, wire.Bind( | ||||
| 	new(consensus.ConsensusEngine), | ||||
|  | ||||
| @ -11,17 +11,13 @@ import ( | ||||
| 	"github.com/libp2p/go-libp2p/core/peer" | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"golang.org/x/sync/errgroup" | ||||
| 	"google.golang.org/protobuf/proto" | ||||
| 	"google.golang.org/protobuf/types/known/anypb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/go-libp2p-blossomsub/pb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	qcrypto "source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| ) | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) handleMessage( | ||||
| @ -304,17 +300,8 @@ func (e *CeremonyDataClockConsensusEngine) handleClockFrameData( | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	earliestFrame, _, count := e.frameProverTrie.Get(addr.Bytes()) | ||||
| 	_, latestFrame, _ := e.frameSeenProverTrie.Get(addr.Bytes()) | ||||
| 	if !isSync && frame.FrameNumber == latestFrame { | ||||
| 		e.logger.Info( | ||||
| 			"already received frame from address", | ||||
| 			zap.Binary("address", address), | ||||
| 			zap.Binary("filter", frame.Filter), | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 		) | ||||
| 		return nil | ||||
| 	} else if frame.FrameNumber <= earliestFrame || count == 0 { | ||||
| 	prover := e.frameProverTrie.FindNearest(addr.Bytes()) | ||||
| 	if !bytes.Equal(prover.External.Key, addr.Bytes()) { | ||||
| 		e.logger.Info( | ||||
| 			"prover not in trie at frame, address may be in fork", | ||||
| 			zap.Binary("address", address), | ||||
| @ -332,167 +319,14 @@ func (e *CeremonyDataClockConsensusEngine) handleClockFrameData( | ||||
| 		zap.Int("proof_count", len(frame.AggregateProofs)), | ||||
| 	) | ||||
| 
 | ||||
| 	if err := frame.VerifyDataClockFrame(); err != nil { | ||||
| 	if err := e.frameProver.VerifyDataClockFrame(frame); err != nil { | ||||
| 		e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	aggregateCommitments := []curves.PairingPoint{} | ||||
| 	for i := 0; i < (len(frame.Input)-516)/74; i++ { | ||||
| 		c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 			frame.Input[516+(i*74) : 516+(i*74)+74], | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "handle clock frame data") | ||||
| 		} | ||||
| 		aggregateCommitments = append(aggregateCommitments, c.(curves.PairingPoint)) | ||||
| 	} | ||||
| 
 | ||||
| 	for i, proof := range frame.AggregateProofs { | ||||
| 		aggregatePoly := [][]curves.PairingScalar{} | ||||
| 		commitments := []curves.PairingPoint{} | ||||
| 
 | ||||
| 		for _, commit := range proof.GetInclusionCommitments() { | ||||
| 			switch commit.TypeUrl { | ||||
| 			case protobufs.IntrinsicExecutionOutputType: | ||||
| 				e.logger.Debug("confirming inclusion in aggregate") | ||||
| 				digest := sha3.NewShake256() | ||||
| 				_, err := digest.Write(commit.Data) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 
 | ||||
| 				expand := make([]byte, 1024) | ||||
| 				_, err = digest.Read(expand) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 
 | ||||
| 				poly, err := e.prover.BytesToPolynomial(expand) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 
 | ||||
| 				evalPoly, err := qcrypto.FFT( | ||||
| 					poly, | ||||
| 					*curves.BLS48581( | ||||
| 						curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 					), | ||||
| 					16, | ||||
| 					false, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error performing fast fourier transform on key bundle", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 				e.logger.Debug( | ||||
| 					"created fft of polynomial", | ||||
| 					zap.Int("poly_size", len(evalPoly)), | ||||
| 				) | ||||
| 
 | ||||
| 				aggregatePoly = append(aggregatePoly, evalPoly) | ||||
| 
 | ||||
| 				c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 					commit.Commitment, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 				commitments = append(commitments, c.(curves.PairingPoint)) | ||||
| 			default: | ||||
| 				e.logger.Debug("confirming inclusion in aggregate") | ||||
| 				poly, err := e.prover.BytesToPolynomial(commit.Data) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 
 | ||||
| 				for i := 0; i < 128-len(poly); i++ { | ||||
| 					poly = append( | ||||
| 						poly, | ||||
| 						curves.BLS48581G1().Scalar.Zero().(curves.PairingScalar), | ||||
| 					) | ||||
| 				} | ||||
| 
 | ||||
| 				evalPoly, err := qcrypto.FFT( | ||||
| 					poly, | ||||
| 					*curves.BLS48581( | ||||
| 						curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 					), | ||||
| 					128, | ||||
| 					false, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"error performing fast fourier transform on key bundle", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 				e.logger.Debug( | ||||
| 					"created fft of polynomial", | ||||
| 					zap.Int("poly_size", len(evalPoly)), | ||||
| 				) | ||||
| 
 | ||||
| 				aggregatePoly = append(aggregatePoly, evalPoly) | ||||
| 
 | ||||
| 				c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 					commit.Commitment, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 					return errors.Wrap(err, "handle clock frame data") | ||||
| 				} | ||||
| 				commitments = append(commitments, c.(curves.PairingPoint)) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		p, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 			proof.Proof, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "handle clock frame data") | ||||
| 		} | ||||
| 
 | ||||
| 		result, err := e.prover.VerifyAggregateProof( | ||||
| 			aggregatePoly, | ||||
| 			commitments, | ||||
| 			aggregateCommitments[i], | ||||
| 			p.(curves.PairingPoint), | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "handle clock frame data") | ||||
| 		} | ||||
| 
 | ||||
| 		if !result { | ||||
| 			e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap( | ||||
| 				errors.New("invalid proof"), | ||||
| 				"handle clock frame data", | ||||
| 			) | ||||
| 		} | ||||
| 	if err := e.inclusionProver.VerifyFrame(frame); err != nil { | ||||
| 		e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	e.logger.Info( | ||||
| @ -501,102 +335,23 @@ func (e *CeremonyDataClockConsensusEngine) handleClockFrameData( | ||||
| 		zap.Binary("filter", frame.Filter), | ||||
| 		zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 	) | ||||
| 	masterFrame, err := e.clockStore.GetMasterClockFrame( | ||||
| 		[]byte{ | ||||
| 			0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 			0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 			0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 			0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 		}, | ||||
| 		frame.FrameNumber-1, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		e.logger.Info("received frame with no known master, needs sync") | ||||
| 		return nil | ||||
| 	} | ||||
| 
 | ||||
| 	discriminator, err := masterFrame.GetSelector() | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	parentSelector, distance, selector, err := | ||||
| 		frame.GetParentSelectorAndDistance(discriminator) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if _, err := e.clockStore.GetParentDataClockFrame( | ||||
| 		frame.Filter, | ||||
| 		frame.FrameNumber-1, | ||||
| 		frame.ParentSelector, | ||||
| 	); errors.Is(err, store.ErrNotFound) { | ||||
| 		// If this is a frame number higher than what we're already caught up to,
 | ||||
| 		// push a request to fill the gap, unless we're syncing or it's in step,
 | ||||
| 		// then just lazily seek.
 | ||||
| 		from := e.frame.FrameNumber | ||||
| 		if from >= frame.FrameNumber-1 { | ||||
| 			from = frame.FrameNumber - 1 | ||||
| 		} | ||||
| 
 | ||||
| 		if err := e.publishMessage(e.filter, &protobufs.ClockFramesRequest{ | ||||
| 			Filter:          e.filter, | ||||
| 			FromFrameNumber: from, | ||||
| 			ToFrameNumber:   frame.FrameNumber, | ||||
| 		}); err != nil { | ||||
| 			e.logger.Error( | ||||
| 				"could not publish clock frame parent request, skipping", | ||||
| 				zap.Error(err), | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	txn, err := e.clockStore.NewTransaction() | ||||
| 	if err != nil { | ||||
| 		e.logger.Error("could not save candidate clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if err := e.clockStore.PutCandidateDataClockFrame( | ||||
| 		parentSelector.FillBytes(make([]byte, 32)), | ||||
| 		distance.FillBytes(make([]byte, 32)), | ||||
| 		selector.FillBytes(make([]byte, 32)), | ||||
| 		frame, | ||||
| 		txn, | ||||
| 	); err != nil { | ||||
| 		e.logger.Error("could not save candidate clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if err := txn.Commit(); err != nil { | ||||
| 		e.logger.Error("could not save candidate clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if e.frame.FrameNumber < frame.FrameNumber { | ||||
| 		e.latestFrameReceived = frame.FrameNumber | ||||
| 		e.lastFrameReceivedAt = time.Now().UTC() | ||||
| 	} | ||||
| 	e.frameSeenProverTrie.Add(address, frame.FrameNumber) | ||||
| 	e.dataTimeReel.Insert(frame) | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) publishProof( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	if e.state == consensus.EngineStatePublishing { | ||||
| 		e.logger.Debug( | ||||
| 			"publishing frame and aggregations", | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 	e.logger.Debug( | ||||
| 		"publishing frame and aggregations", | ||||
| 		zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 	) | ||||
| 	if err := e.publishMessage(e.filter, frame); err != nil { | ||||
| 		return errors.Wrap( | ||||
| 			err, | ||||
| 			"publish proof", | ||||
| 		) | ||||
| 		if err := e.publishMessage(e.filter, frame); err != nil { | ||||
| 			return errors.Wrap( | ||||
| 				err, | ||||
| 				"publish proof", | ||||
| 			) | ||||
| 		} | ||||
| 
 | ||||
| 		e.state = consensus.EngineStateCollecting | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
|  | ||||
| @ -1,9 +1,9 @@ | ||||
| package ceremony | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"crypto" | ||||
| 	"encoding/binary" | ||||
| 	"math/big" | ||||
| 	"sync" | ||||
| 	"time" | ||||
| 
 | ||||
| @ -14,6 +14,7 @@ import ( | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	qtime "source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	qcrypto "source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| @ -53,7 +54,6 @@ type ChannelServer = protobufs.CeremonyService_GetPublicChannelServer | ||||
| 
 | ||||
| type CeremonyDataClockConsensusEngine struct { | ||||
| 	protobufs.UnimplementedCeremonyServiceServer | ||||
| 	frame                       *protobufs.ClockFrame | ||||
| 	difficulty                  uint32 | ||||
| 	logger                      *zap.Logger | ||||
| 	state                       consensus.EngineState | ||||
| @ -61,6 +61,8 @@ type CeremonyDataClockConsensusEngine struct { | ||||
| 	keyStore                    store.KeyStore | ||||
| 	pubSub                      p2p.PubSub | ||||
| 	keyManager                  keys.KeyManager | ||||
| 	masterTimeReel              *qtime.MasterTimeReel | ||||
| 	dataTimeReel                *qtime.DataTimeReel | ||||
| 	provingKey                  crypto.Signer | ||||
| 	provingKeyBytes             []byte | ||||
| 	provingKeyType              keys.KeyType | ||||
| @ -68,13 +70,11 @@ type CeremonyDataClockConsensusEngine struct { | ||||
| 	lastFrameReceivedAt         time.Time | ||||
| 	latestFrameReceived         uint64 | ||||
| 	frameProverTrie             *tries.RollingFrecencyCritbitTrie | ||||
| 	frameSeenProverTrie         *tries.RollingFrecencyCritbitTrie | ||||
| 	dependencyMap               map[string]*anypb.Any | ||||
| 	pendingCommits              chan *anypb.Any | ||||
| 	pendingCommitWorkers        int64 | ||||
| 	prover                      *qcrypto.KZGProver | ||||
| 	stagedKeyCommits            InclusionMap | ||||
| 	stagedKeyPolynomials        PolynomialMap | ||||
| 	inclusionProver             qcrypto.InclusionProver | ||||
| 	frameProver                 qcrypto.FrameProver | ||||
| 	stagedLobbyStateTransitions *protobufs.CeremonyLobbyStateTransition | ||||
| 	minimumPeersRequired        int | ||||
| 
 | ||||
| @ -86,10 +86,8 @@ type CeremonyDataClockConsensusEngine struct { | ||||
| 	syncingStatus                  SyncStatusType | ||||
| 	syncingTarget                  []byte | ||||
| 	previousHead                   *protobufs.ClockFrame | ||||
| 	currentDistance                *big.Int | ||||
| 	engineMx                       sync.Mutex | ||||
| 	dependencyMapMx                sync.Mutex | ||||
| 	stagedKeyCommitsMx             sync.Mutex | ||||
| 	stagedLobbyStateTransitionsMx  sync.Mutex | ||||
| 	peerMapMx                      sync.Mutex | ||||
| 	peerAnnounceMapMx              sync.Mutex | ||||
| @ -112,6 +110,10 @@ func NewCeremonyDataClockConsensusEngine( | ||||
| 	clockStore store.ClockStore, | ||||
| 	keyStore store.KeyStore, | ||||
| 	pubSub p2p.PubSub, | ||||
| 	frameProver qcrypto.FrameProver, | ||||
| 	inclusionProver qcrypto.InclusionProver, | ||||
| 	masterTimeReel *qtime.MasterTimeReel, | ||||
| 	dataTimeReel *qtime.DataTimeReel, | ||||
| 	filter []byte, | ||||
| 	seed []byte, | ||||
| ) *CeremonyDataClockConsensusEngine { | ||||
| @ -139,6 +141,22 @@ func NewCeremonyDataClockConsensusEngine( | ||||
| 		panic(errors.New("pubsub is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if frameProver == nil { | ||||
| 		panic(errors.New("frame prover is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if inclusionProver == nil { | ||||
| 		panic(errors.New("inclusion prover is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if masterTimeReel == nil { | ||||
| 		panic(errors.New("master time reel is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if dataTimeReel == nil { | ||||
| 		panic(errors.New("data time reel is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	minimumPeersRequired := engineConfig.MinimumPeersRequired | ||||
| 	if minimumPeersRequired == 0 { | ||||
| 		minimumPeersRequired = 3 | ||||
| @ -150,7 +168,6 @@ func NewCeremonyDataClockConsensusEngine( | ||||
| 	} | ||||
| 
 | ||||
| 	e := &CeremonyDataClockConsensusEngine{ | ||||
| 		frame:            nil, | ||||
| 		difficulty:       difficulty, | ||||
| 		logger:           logger, | ||||
| 		state:            consensus.EngineStateStopped, | ||||
| @ -169,17 +186,15 @@ func NewCeremonyDataClockConsensusEngine( | ||||
| 		}, | ||||
| 		lastFrameReceivedAt:   time.Time{}, | ||||
| 		frameProverTrie:       &tries.RollingFrecencyCritbitTrie{}, | ||||
| 		frameSeenProverTrie:   &tries.RollingFrecencyCritbitTrie{}, | ||||
| 		pendingCommits:        make(chan *anypb.Any), | ||||
| 		pendingCommitWorkers:  engineConfig.PendingCommitWorkers, | ||||
| 		prover:                qcrypto.DefaultKZGProver(), | ||||
| 		stagedKeyCommits:      make(InclusionMap), | ||||
| 		stagedKeyPolynomials:  make(PolynomialMap), | ||||
| 		inclusionProver:       inclusionProver, | ||||
| 		syncingStatus:         SyncStatusNotSyncing, | ||||
| 		peerAnnounceMap:       map[string]*protobufs.CeremonyPeerListAnnounce{}, | ||||
| 		peerMap:               map[string]*peerInfo{}, | ||||
| 		uncooperativePeersMap: map[string]*peerInfo{}, | ||||
| 		minimumPeersRequired:  minimumPeersRequired, | ||||
| 		frameProver:           frameProver, | ||||
| 		masterTimeReel:        masterTimeReel, | ||||
| 		dataTimeReel:          dataTimeReel, | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("constructing consensus engine") | ||||
| @ -205,30 +220,12 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 	e.state = consensus.EngineStateLoading | ||||
| 
 | ||||
| 	e.logger.Info("loading last seen state") | ||||
| 	latestFrame, err := e.clockStore.GetLatestDataClockFrame( | ||||
| 		e.filter, | ||||
| 		e.frameProverTrie, | ||||
| 	) | ||||
| 	if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 	err := e.dataTimeReel.Start() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	candidateLatestFrame, err := e.clockStore.GetLatestCandidateDataClockFrame( | ||||
| 		e.filter, | ||||
| 	) | ||||
| 	if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if candidateLatestFrame != nil { | ||||
| 		latestFrame = candidateLatestFrame | ||||
| 	} | ||||
| 
 | ||||
| 	if latestFrame != nil { | ||||
| 		e.setFrame(latestFrame) | ||||
| 	} else { | ||||
| 		latestFrame = e.CreateGenesisFrame(nil) | ||||
| 	} | ||||
| 	e.frameProverTrie = e.dataTimeReel.GetFrameProverTrie() | ||||
| 
 | ||||
| 	err = e.createCommunicationKeys() | ||||
| 	if err != nil { | ||||
| @ -255,10 +252,6 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 
 | ||||
| 	e.state = consensus.EngineStateCollecting | ||||
| 
 | ||||
| 	for i := int64(0); i < e.pendingCommitWorkers; i++ { | ||||
| 		go e.handlePendingCommits(i) | ||||
| 	} | ||||
| 
 | ||||
| 	go func() { | ||||
| 		thresholdBeforeConfirming := 4 | ||||
| 
 | ||||
| @ -269,8 +262,13 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 				PeerList: []*protobufs.CeremonyPeer{}, | ||||
| 			} | ||||
| 
 | ||||
| 			frame, err := e.dataTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			timestamp := time.Now().UnixMilli() | ||||
| 			msg := binary.BigEndian.AppendUint64([]byte{}, e.frame.FrameNumber) | ||||
| 			msg := binary.BigEndian.AppendUint64([]byte{}, frame.FrameNumber) | ||||
| 			msg = append(msg, consensus.GetVersion()...) | ||||
| 			msg = binary.BigEndian.AppendUint64(msg, uint64(timestamp)) | ||||
| 			sig, err := e.pubSub.SignMessage(msg) | ||||
| @ -282,7 +280,7 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 			e.peerMap[string(e.pubSub.GetPeerID())] = &peerInfo{ | ||||
| 				peerId:    e.pubSub.GetPeerID(), | ||||
| 				multiaddr: "", | ||||
| 				maxFrame:  e.frame.FrameNumber, | ||||
| 				maxFrame:  frame.FrameNumber, | ||||
| 				version:   consensus.GetVersion(), | ||||
| 				signature: sig, | ||||
| 				publicKey: e.pubSub.GetPublicKey(), | ||||
| @ -325,42 +323,7 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 	}() | ||||
| 
 | ||||
| 	go func() { | ||||
| 		e.logger.Info("waiting for peer list mappings") | ||||
| 		// We need to re-tune this so that libp2p's peerstore activation threshold
 | ||||
| 		// considers DHT peers to be correct:
 | ||||
| 		time.Sleep(30 * time.Second) | ||||
| 		for e.state < consensus.EngineStateStopping { | ||||
| 			peerCount := e.pubSub.GetNetworkPeersCount() | ||||
| 			if peerCount < e.minimumPeersRequired { | ||||
| 				e.logger.Info( | ||||
| 					"waiting for minimum peers", | ||||
| 					zap.Int("peer_count", peerCount), | ||||
| 				) | ||||
| 				time.Sleep(1 * time.Second) | ||||
| 			} else { | ||||
| 				switch e.state { | ||||
| 				case consensus.EngineStateCollecting: | ||||
| 					currentFrame := latestFrame | ||||
| 					if latestFrame, err = e.collect(latestFrame); err != nil { | ||||
| 						e.logger.Error("could not collect", zap.Error(err)) | ||||
| 						e.state = consensus.EngineStateCollecting | ||||
| 						latestFrame = currentFrame | ||||
| 					} | ||||
| 				case consensus.EngineStateProving: | ||||
| 					currentFrame := latestFrame | ||||
| 					if latestFrame, err = e.prove(latestFrame); err != nil { | ||||
| 						e.logger.Error("could not prove", zap.Error(err)) | ||||
| 						e.state = consensus.EngineStateCollecting | ||||
| 						latestFrame = currentFrame | ||||
| 					} | ||||
| 				case consensus.EngineStatePublishing: | ||||
| 					if err = e.publishProof(latestFrame); err != nil { | ||||
| 						e.logger.Error("could not publish", zap.Error(err)) | ||||
| 						e.state = consensus.EngineStateCollecting | ||||
| 					} | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 		e.runLoop() | ||||
| 	}() | ||||
| 
 | ||||
| 	go func() { | ||||
| @ -370,6 +333,86 @@ func (e *CeremonyDataClockConsensusEngine) Start() <-chan error { | ||||
| 	return errChan | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) runLoop() { | ||||
| 	dataFrameCh := e.dataTimeReel.NewFrameCh() | ||||
| 
 | ||||
| 	e.logger.Info("waiting for peer list mappings") | ||||
| 	// We need to re-tune this so that libp2p's peerstore activation threshold
 | ||||
| 	// considers DHT peers to be correct:
 | ||||
| 	time.Sleep(30 * time.Second) | ||||
| 
 | ||||
| 	for e.state < consensus.EngineStateStopping { | ||||
| 		peerCount := e.pubSub.GetNetworkPeersCount() | ||||
| 		if peerCount < e.minimumPeersRequired { | ||||
| 			e.logger.Info( | ||||
| 				"waiting for minimum peers", | ||||
| 				zap.Int("peer_count", peerCount), | ||||
| 			) | ||||
| 			time.Sleep(1 * time.Second) | ||||
| 		} else { | ||||
| 			latestFrame, err := e.dataTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 			select { | ||||
| 			case dataFrame := <-dataFrameCh: | ||||
| 				if latestFrame, err = e.collect(dataFrame); err != nil { | ||||
| 					e.logger.Error("could not collect", zap.Error(err)) | ||||
| 					continue | ||||
| 				} | ||||
| 				go func() { | ||||
| 					e.frameChan <- latestFrame | ||||
| 				}() | ||||
| 
 | ||||
| 				var nextFrame *protobufs.ClockFrame | ||||
| 				if nextFrame, err = e.prove(latestFrame); err != nil { | ||||
| 					e.logger.Error("could not prove", zap.Error(err)) | ||||
| 					e.state = consensus.EngineStateCollecting | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if bytes.Equal( | ||||
| 					e.frameProverTrie.FindNearest(e.provingKeyAddress).External.Key, | ||||
| 					e.provingKeyAddress, | ||||
| 				) { | ||||
| 					if err = e.publishProof(nextFrame); err != nil { | ||||
| 						e.logger.Error("could not publish", zap.Error(err)) | ||||
| 						e.state = consensus.EngineStateCollecting | ||||
| 					} | ||||
| 				} | ||||
| 			case <-time.After(20 * time.Second): | ||||
| 				e.logger.Info("no frames received, kicking off") | ||||
| 				dataFrame, err := e.dataTimeReel.Head() | ||||
| 				if err != nil { | ||||
| 					panic(err) | ||||
| 				} | ||||
| 
 | ||||
| 				if latestFrame, err = e.collect(dataFrame); err != nil { | ||||
| 					e.logger.Error("could not collect", zap.Error(err)) | ||||
| 					continue | ||||
| 				} | ||||
| 				go func() { | ||||
| 					e.frameChan <- latestFrame | ||||
| 				}() | ||||
| 
 | ||||
| 				var nextFrame *protobufs.ClockFrame | ||||
| 				if nextFrame, err = e.prove(latestFrame); err != nil { | ||||
| 					e.logger.Error("could not prove", zap.Error(err)) | ||||
| 					e.state = consensus.EngineStateCollecting | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if e.frameProverTrie.Contains(e.provingKeyAddress) { | ||||
| 					if err = e.publishProof(nextFrame); err != nil { | ||||
| 						e.logger.Error("could not publish", zap.Error(err)) | ||||
| 						e.state = consensus.EngineStateCollecting | ||||
| 					} | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) Stop(force bool) <-chan error { | ||||
| 	e.logger.Info("stopping ceremony consensus engine") | ||||
| 	e.state = consensus.EngineStateStopping | ||||
| @ -380,7 +423,12 @@ func (e *CeremonyDataClockConsensusEngine) Stop(force bool) <-chan error { | ||||
| 	for name := range e.executionEngines { | ||||
| 		name := name | ||||
| 		go func(name string) { | ||||
| 			err := <-e.UnregisterExecutor(name, e.frame.FrameNumber, force) | ||||
| 			frame, err := e.dataTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			err = <-e.UnregisterExecutor(name, frame.FrameNumber, force) | ||||
| 			if err != nil { | ||||
| 				errChan <- err | ||||
| 			} | ||||
| @ -402,133 +450,17 @@ func (e *CeremonyDataClockConsensusEngine) Stop(force bool) <-chan error { | ||||
| 	return errChan | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) performSanityCheck( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) *protobufs.ClockFrame { | ||||
| 	e.logger.Info("performing sanity check") | ||||
| 	start := uint64(0) | ||||
| 	idx := start | ||||
| 	end := frame.FrameNumber + 1 | ||||
| 	var prior *protobufs.ClockFrame | ||||
| 	for start < end { | ||||
| 		tail := end | ||||
| 		if start+16 < tail { | ||||
| 			tail = start + 16 | ||||
| 		} | ||||
| 		iter, err := e.clockStore.RangeDataClockFrames( | ||||
| 			e.filter, | ||||
| 			start, | ||||
| 			tail, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		for iter.First(); iter.Valid(); iter.Next() { | ||||
| 			v, err := iter.Value() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			if v.FrameNumber != idx { | ||||
| 				e.logger.Warn( | ||||
| 					"discontinuity found, attempting to fix", | ||||
| 					zap.Uint64("expected_frame_number", idx), | ||||
| 					zap.Uint64("found_frame_number", v.FrameNumber), | ||||
| 				) | ||||
| 
 | ||||
| 				disc := v | ||||
| 				for disc.FrameNumber-idx > 0 { | ||||
| 					frames, err := e.clockStore.GetCandidateDataClockFrames( | ||||
| 						e.filter, | ||||
| 						disc.FrameNumber-1, | ||||
| 					) | ||||
| 					if err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					found := false | ||||
| 					for _, candidate := range frames { | ||||
| 						selector, err := candidate.GetSelector() | ||||
| 						if err != nil { | ||||
| 							panic(err) | ||||
| 						} | ||||
| 
 | ||||
| 						parentSelector, _, _, err := disc.GetParentSelectorAndDistance(nil) | ||||
| 						if err != nil { | ||||
| 							panic(err) | ||||
| 						} | ||||
| 
 | ||||
| 						if selector.Cmp(parentSelector) == 0 { | ||||
| 							found = true | ||||
| 							_, priorTrie, err := e.clockStore.GetDataClockFrame( | ||||
| 								e.filter, | ||||
| 								prior.FrameNumber, | ||||
| 							) | ||||
| 							if err != nil { | ||||
| 								panic(err) | ||||
| 							} | ||||
| 
 | ||||
| 							txn, err := e.clockStore.NewTransaction() | ||||
| 							if err != nil { | ||||
| 								panic(err) | ||||
| 							} | ||||
| 
 | ||||
| 							err = e.clockStore.PutDataClockFrame( | ||||
| 								candidate, | ||||
| 								priorTrie, | ||||
| 								txn, | ||||
| 								true, | ||||
| 							) | ||||
| 							if err != nil { | ||||
| 								panic(err) | ||||
| 							} | ||||
| 
 | ||||
| 							if err = txn.Commit(); err != nil { | ||||
| 								panic(err) | ||||
| 							} | ||||
| 
 | ||||
| 							disc = candidate | ||||
| 						} | ||||
| 					} | ||||
| 
 | ||||
| 					if !found { | ||||
| 						e.logger.Error( | ||||
| 							"could not resolve discontinuity, rewinding consensus head", | ||||
| 						) | ||||
| 
 | ||||
| 						if err = iter.Close(); err != nil { | ||||
| 							panic(err) | ||||
| 						} | ||||
| 
 | ||||
| 						return prior | ||||
| 					} | ||||
| 				} | ||||
| 
 | ||||
| 				idx = v.FrameNumber | ||||
| 			} else { | ||||
| 				prior = v | ||||
| 			} | ||||
| 
 | ||||
| 			idx++ | ||||
| 		} | ||||
| 
 | ||||
| 		if err = iter.Close(); err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		start += 16 | ||||
| 	} | ||||
| 
 | ||||
| 	return frame | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) GetDifficulty() uint32 { | ||||
| 	return e.difficulty | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) GetFrame() *protobufs.ClockFrame { | ||||
| 	return e.frame | ||||
| 	frame, err := e.dataTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) GetState() consensus.EngineState { | ||||
|  | ||||
										
											
												File diff suppressed because it is too large
												Load Diff
											
										
									
								
							| @ -15,13 +15,18 @@ func (e *CeremonyDataClockConsensusEngine) RegisterExecutor( | ||||
| 
 | ||||
| 	go func() { | ||||
| 		for { | ||||
| 			masterFrame, err := e.masterTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			logger.Info( | ||||
| 				"awaiting frame", | ||||
| 				zap.Uint64("current_frame", e.frame.FrameNumber), | ||||
| 				zap.Uint64("current_frame", masterFrame.FrameNumber), | ||||
| 				zap.Uint64("target_frame", frame), | ||||
| 			) | ||||
| 
 | ||||
| 			newFrame := e.frame.FrameNumber | ||||
| 			newFrame := masterFrame.FrameNumber | ||||
| 			if newFrame >= frame { | ||||
| 				logger.Info( | ||||
| 					"injecting execution engine at frame", | ||||
| @ -52,13 +57,18 @@ func (e *CeremonyDataClockConsensusEngine) UnregisterExecutor( | ||||
| 
 | ||||
| 	go func() { | ||||
| 		for { | ||||
| 			masterFrame, err := e.masterTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			logger.Info( | ||||
| 				"awaiting frame", | ||||
| 				zap.Uint64("current_frame", e.frame.FrameNumber), | ||||
| 				zap.Uint64("current_frame", masterFrame.FrameNumber), | ||||
| 				zap.Uint64("target_frame", frame), | ||||
| 			) | ||||
| 
 | ||||
| 			newFrame := e.frame.FrameNumber | ||||
| 			newFrame := masterFrame.FrameNumber | ||||
| 			if newFrame >= frame { | ||||
| 				logger.Info( | ||||
| 					"removing execution engine at frame", | ||||
|  | ||||
| @ -1,85 +0,0 @@ | ||||
| package ceremony | ||||
| 
 | ||||
| import ( | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"google.golang.org/protobuf/types/known/anypb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) handlePendingCommits( | ||||
| 	workerId int64, | ||||
| ) { | ||||
| 	for { | ||||
| 		msg := <-e.pendingCommits | ||||
| 		switch msg.TypeUrl { | ||||
| 		case protobufs.KeyBundleAnnouncementType: | ||||
| 			if err := e.includeKeyBundle(msg); err != nil { | ||||
| 				e.logger.Error( | ||||
| 					"failed to include key bundle", | ||||
| 					zap.Error(errors.Wrap(err, "handle pending commits")), | ||||
| 					zap.Int64("worker_id", workerId), | ||||
| 				) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (e *CeremonyDataClockConsensusEngine) includeKeyBundle( | ||||
| 	any *anypb.Any, | ||||
| ) error { | ||||
| 	poly, err := e.prover.BytesToPolynomial(any.Value) | ||||
| 	if err != nil { | ||||
| 		e.logger.Error( | ||||
| 			"error converting key bundle to polynomial", | ||||
| 			zap.Error(err), | ||||
| 		) | ||||
| 		return errors.Wrap(err, "include key bundle") | ||||
| 	} | ||||
| 
 | ||||
| 	for i := 0; i < 128-len(poly); i++ { | ||||
| 		poly = append( | ||||
| 			poly, | ||||
| 			curves.BLS48581G1().Scalar.Zero().(curves.PairingScalar), | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	evalPoly, err := crypto.FFT( | ||||
| 		poly, | ||||
| 		*curves.BLS48581( | ||||
| 			curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 		), | ||||
| 		128, | ||||
| 		false, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		e.logger.Error( | ||||
| 			"error performing fast fourier transform on key bundle", | ||||
| 			zap.Error(err), | ||||
| 		) | ||||
| 		return errors.Wrap(err, "include key bundle") | ||||
| 	} | ||||
| 
 | ||||
| 	commitment, err := e.prover.Commit(evalPoly) | ||||
| 	if err != nil { | ||||
| 		e.logger.Error( | ||||
| 			"error creating kzg commitment", | ||||
| 			zap.Error(err), | ||||
| 		) | ||||
| 		return errors.Wrap(err, "include key bundle") | ||||
| 	} | ||||
| 
 | ||||
| 	e.stagedKeyCommitsMx.Lock() | ||||
| 	e.stagedKeyCommits[commitment] = &protobufs.InclusionCommitment{ | ||||
| 		Filter:     e.filter, | ||||
| 		TypeUrl:    any.TypeUrl, | ||||
| 		Data:       any.Value, | ||||
| 		Commitment: commitment.ToAffineCompressed(), | ||||
| 	} | ||||
| 	e.stagedKeyPolynomials[commitment] = evalPoly | ||||
| 	e.stagedKeyCommitsMx.Unlock() | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| @ -11,7 +11,7 @@ import ( | ||||
| 	"google.golang.org/grpc" | ||||
| 	"google.golang.org/protobuf/proto" | ||||
| 	"google.golang.org/protobuf/types/known/anypb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| @ -106,7 +106,12 @@ func (e *CeremonyDataClockConsensusEngine) GetCompressedSyncFrames( | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	max := e.frame.FrameNumber | ||||
| 	head, err := e.dataTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	max := head.FrameNumber | ||||
| 	to := request.ToFrameNumber | ||||
| 
 | ||||
| 	// We need to slightly rewind, to compensate for unconfirmed frame heads on a
 | ||||
|  | ||||
| @ -51,13 +51,13 @@ type DataConsensusEngine interface { | ||||
| } | ||||
| 
 | ||||
| func GetMinimumVersionCutoff() time.Time { | ||||
| 	return time.Date(2024, time.January, 14, 7, 0, 0, 0, time.UTC) | ||||
| 	return time.Date(2024, time.February, 13, 7, 0, 0, 0, time.UTC) | ||||
| } | ||||
| 
 | ||||
| func GetMinimumVersion() []byte { | ||||
| 	return []byte{0x01, 0x02, 0x03} | ||||
| 	return []byte{0x01, 0x02, 0x04} | ||||
| } | ||||
| 
 | ||||
| func GetVersion() []byte { | ||||
| 	return []byte{0x01, 0x02, 0x03} | ||||
| 	return []byte{0x01, 0x02, 0x04} | ||||
| } | ||||
|  | ||||
| @ -1,9 +1,7 @@ | ||||
| package master | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"strings" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/iden3/go-iden3-crypto/poseidon" | ||||
| 	"github.com/pkg/errors" | ||||
| @ -92,17 +90,6 @@ func (e *MasterClockConsensusEngine) handleClockFrameData( | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if e.frame.FrameNumber > frame.FrameNumber { | ||||
| 		e.logger.Debug( | ||||
| 			"received anachronistic frame", | ||||
| 			zap.Binary("sender", peerID), | ||||
| 			zap.Binary("filter", frame.Filter), | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 			zap.Int("proof_count", len(frame.AggregateProofs)), | ||||
| 		) | ||||
| 		return nil | ||||
| 	} | ||||
| 
 | ||||
| 	if e.difficulty != frame.Difficulty { | ||||
| 		e.logger.Debug( | ||||
| 			"frame difficulty mismatched", | ||||
| @ -122,65 +109,35 @@ func (e *MasterClockConsensusEngine) handleClockFrameData( | ||||
| 		zap.Int("proof_count", len(frame.AggregateProofs)), | ||||
| 	) | ||||
| 
 | ||||
| 	if err := frame.VerifyMasterClockFrame(); err != nil { | ||||
| 	if err := e.frameProver.VerifyMasterClockFrame(frame); err != nil { | ||||
| 		e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 		return errors.Wrap(err, "handle clock frame data") | ||||
| 	} | ||||
| 
 | ||||
| 	if e.frame.FrameNumber < frame.FrameNumber { | ||||
| 		if err := e.enqueueSeenFrame(frame); err != nil { | ||||
| 			e.logger.Error("could not enqueue seen clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "handle clock frame data") | ||||
| 		} | ||||
| 	} | ||||
| 	e.masterTimeReel.Insert(frame) | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) enqueueSeenFrame( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	e.seenFramesMx.Lock() | ||||
| 	found := false | ||||
| 	for _, f := range e.seenFrames { | ||||
| 		if f.FrameNumber == frame.FrameNumber && | ||||
| 			bytes.Equal(f.Input, frame.Input) && | ||||
| 			f.Difficulty == frame.Difficulty && | ||||
| 			bytes.Equal(f.Output, frame.Output) { | ||||
| 			found = true | ||||
| 		} | ||||
| 	} | ||||
| 	if !found { | ||||
| 		e.logger.Debug( | ||||
| 			"enqueuing frame for consensus", | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 		) | ||||
| 		e.seenFrames = append(e.seenFrames, frame) | ||||
| 		e.lastFrameReceivedAt = time.Now().UTC() | ||||
| 	} | ||||
| 	e.seenFramesMx.Unlock() | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) publishProof( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	if e.state == consensus.EngineStatePublishing { | ||||
| 		e.logger.Debug( | ||||
| 			"publishing frame", | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 		) | ||||
| 		e.enqueueSeenFrame(frame) | ||||
| 		if err := e.publishMessage(e.filter, frame); err != nil { | ||||
| 			return errors.Wrap( | ||||
| 				err, | ||||
| 				"publish proof", | ||||
| 			) | ||||
| 		} | ||||
| 	e.logger.Debug( | ||||
| 		"publishing frame", | ||||
| 		zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 	) | ||||
| 
 | ||||
| 		e.state = consensus.EngineStateCollecting | ||||
| 	e.masterTimeReel.Insert(frame) | ||||
| 
 | ||||
| 	if err := e.publishMessage(e.filter, frame); err != nil { | ||||
| 		return errors.Wrap( | ||||
| 			err, | ||||
| 			"publish proof", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	e.state = consensus.EngineStateCollecting | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
|  | ||||
| @ -1,16 +1,11 @@ | ||||
| package master | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"encoding/binary" | ||||
| 	"sort" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/go-libp2p-blossomsub/pb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/vdf" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| @ -19,250 +14,92 @@ import ( | ||||
| func (e *MasterClockConsensusEngine) prove( | ||||
| 	previousFrame *protobufs.ClockFrame, | ||||
| ) (*protobufs.ClockFrame, error) { | ||||
| 	if e.state == consensus.EngineStateProving { | ||||
| 		e.logger.Debug("proving new frame") | ||||
| 	e.logger.Debug("proving new frame") | ||||
| 
 | ||||
| 		frame, err := protobufs.ProveMasterClockFrame( | ||||
| 			previousFrame, | ||||
| 			e.difficulty, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.Wrap(err, "prove") | ||||
| 		} | ||||
| 
 | ||||
| 		e.state = consensus.EngineStatePublishing | ||||
| 		e.logger.Debug("returning new proven frame") | ||||
| 		return frame, nil | ||||
| 	frame, err := e.frameProver.ProveMasterClockFrame( | ||||
| 		previousFrame, | ||||
| 		time.Now().UnixMilli(), | ||||
| 		e.difficulty, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove") | ||||
| 	} | ||||
| 
 | ||||
| 	return nil, nil | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) setFrame(frame *protobufs.ClockFrame) { | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], frame.Output[:516]) | ||||
| 
 | ||||
| 	e.logger.Debug("set frame", zap.Uint64("frame_number", frame.FrameNumber)) | ||||
| 	e.frame = frame | ||||
| 
 | ||||
| 	go func() { | ||||
| 		e.frameChan <- e.frame | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| func ( | ||||
| 	e *MasterClockConsensusEngine, | ||||
| ) CreateGenesisFrame() *protobufs.ClockFrame { | ||||
| 	e.logger.Debug("creating genesis frame") | ||||
| 	b := sha3.Sum256(e.input) | ||||
| 	v := vdf.New(e.difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 	inputMessage := o[:] | ||||
| 
 | ||||
| 	e.logger.Debug("proving genesis frame") | ||||
| 	input := []byte{} | ||||
| 	input = append(input, e.filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, 0) | ||||
| 	input = binary.BigEndian.AppendUint32(input, e.difficulty) | ||||
| 	if bytes.Equal(e.input, []byte{0x00}) { | ||||
| 		value := [516]byte{} | ||||
| 		input = append(input, value[:]...) | ||||
| 	} else { | ||||
| 		input = append(input, e.input...) | ||||
| 	} | ||||
| 
 | ||||
| 	b = sha3.Sum256(input) | ||||
| 	v = vdf.New(e.difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o = v.GetOutput() | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:      e.filter, | ||||
| 		FrameNumber: 0, | ||||
| 		Timestamp:   0, | ||||
| 		Difficulty:  e.difficulty, | ||||
| 		Input:       inputMessage, | ||||
| 		Output:      o[:], | ||||
| 		ParentSelector: []byte{ | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 		}, | ||||
| 		AggregateProofs:    []*protobufs.InclusionAggregateProof{}, | ||||
| 		PublicKeySignature: nil, | ||||
| 	} | ||||
| 
 | ||||
| 	e.setFrame(frame) | ||||
| 	return frame | ||||
| 	e.state = consensus.EngineStatePublishing | ||||
| 	e.logger.Debug("returning new proven frame") | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) collect( | ||||
| 	currentFramePublished *protobufs.ClockFrame, | ||||
| ) (*protobufs.ClockFrame, error) { | ||||
| 	if e.state == consensus.EngineStateCollecting { | ||||
| 		e.logger.Debug("collecting vdf proofs") | ||||
| 	e.logger.Debug("collecting vdf proofs") | ||||
| 
 | ||||
| 		latest := e.frame | ||||
| 
 | ||||
| 		if e.syncingStatus == SyncStatusNotSyncing { | ||||
| 			peer, err := e.pubSub.GetRandomPeer(e.filter) | ||||
| 			if err != nil { | ||||
| 				if errors.Is(err, p2p.ErrNoPeersAvailable) { | ||||
| 					e.logger.Debug("no peers available, skipping sync") | ||||
| 				} else { | ||||
| 					e.logger.Error("error while fetching random peer", zap.Error(err)) | ||||
| 				} | ||||
| 			} else { | ||||
| 				e.syncingStatus = SyncStatusAwaitingResponse | ||||
| 				e.logger.Debug("setting syncing target", zap.Binary("peer_id", peer)) | ||||
| 				e.syncingTarget = peer | ||||
| 
 | ||||
| 				channel := e.createPeerReceiveChannel(peer) | ||||
| 				e.logger.Debug( | ||||
| 					"listening on peer receive channel", | ||||
| 					zap.Binary("channel", channel), | ||||
| 				) | ||||
| 				e.pubSub.Subscribe(channel, e.handleSync, true) | ||||
| 				e.pubSub.Subscribe( | ||||
| 					peer, | ||||
| 					func(message *pb.Message) error { return nil }, | ||||
| 					true, | ||||
| 				) | ||||
| 
 | ||||
| 				go func() { | ||||
| 					time.Sleep(2 * time.Second) | ||||
| 					if err := e.publishMessage(peer, &protobufs.ClockFramesRequest{ | ||||
| 						Filter:          e.filter, | ||||
| 						FromFrameNumber: latest.FrameNumber + 1, | ||||
| 					}); err != nil { | ||||
| 						e.logger.Error( | ||||
| 							"could not publish clock frame request", | ||||
| 							zap.Error(err), | ||||
| 						) | ||||
| 					} | ||||
| 				}() | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		waitDecay := time.Duration(2000) | ||||
| 		for e.syncingStatus != SyncStatusNotSyncing { | ||||
| 			e.logger.Debug( | ||||
| 				"waiting for sync to complete...", | ||||
| 				zap.Duration("wait_decay", waitDecay), | ||||
| 			) | ||||
| 
 | ||||
| 			time.Sleep(waitDecay * time.Millisecond) | ||||
| 
 | ||||
| 			waitDecay = waitDecay * 2 | ||||
| 			if waitDecay >= (100 * (2 << 6)) { | ||||
| 				if e.syncingStatus == SyncStatusAwaitingResponse { | ||||
| 					e.logger.Debug("maximum wait for sync response, skipping sync") | ||||
| 					e.syncingStatus = SyncStatusNotSyncing | ||||
| 					break | ||||
| 				} else { | ||||
| 					waitDecay = 100 * (2 << 6) | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		e.logger.Debug("selecting leader") | ||||
| 		latestFrame, err := e.confirmLatestFrame() | ||||
| 		if err != nil { | ||||
| 			e.logger.Error("could not confirm latest frame", zap.Error(err)) | ||||
| 			return nil, errors.Wrap(err, "collect") | ||||
| 		} | ||||
| 
 | ||||
| 		e.logger.Debug( | ||||
| 			"returning leader frame", | ||||
| 			zap.Uint64("frame_number", latestFrame.FrameNumber), | ||||
| 		) | ||||
| 
 | ||||
| 		e.state = consensus.EngineStateProving | ||||
| 		return latestFrame, nil | ||||
| 	} | ||||
| 
 | ||||
| 	return nil, nil | ||||
| } | ||||
| 
 | ||||
| func ( | ||||
| 	e *MasterClockConsensusEngine, | ||||
| ) confirmLatestFrame() (*protobufs.ClockFrame, error) { | ||||
| 	e.seenFramesMx.Lock() | ||||
| 	defer e.seenFramesMx.Unlock() | ||||
| 
 | ||||
| 	sort.Slice(e.seenFrames, func(i, j int) bool { | ||||
| 		return e.seenFrames[i].FrameNumber < e.seenFrames[j].FrameNumber | ||||
| 	}) | ||||
| 
 | ||||
| 	if len(e.seenFrames) == 0 { | ||||
| 		return e.frame, nil | ||||
| 	} | ||||
| 
 | ||||
| 	prev := e.frame | ||||
| 	committedSet := []*protobufs.ClockFrame{} | ||||
| 
 | ||||
| 	for len(e.seenFrames) > 0 { | ||||
| 		curr := e.seenFrames[0] | ||||
| 		e.seenFrames = e.seenFrames[1:] | ||||
| 
 | ||||
| 		e.logger.Debug( | ||||
| 			"checking continuity for frame", | ||||
| 			zap.Uint64("frame_number", curr.FrameNumber), | ||||
| 		) | ||||
| 
 | ||||
| 		if prev.FrameNumber+1 < curr.FrameNumber || | ||||
| 			prev.FrameNumber > curr.FrameNumber { | ||||
| 			e.logger.Debug( | ||||
| 				"continuity break found", | ||||
| 				zap.Uint64("prev_frame_number", prev.FrameNumber), | ||||
| 				zap.Uint64("curr_frame_number", curr.FrameNumber), | ||||
| 			) | ||||
| 			break | ||||
| 		} | ||||
| 
 | ||||
| 		if bytes.Equal(prev.Output, curr.Input[:516]) { | ||||
| 			prev = curr | ||||
| 			committedSet = append(committedSet, prev) | ||||
| 		} else { | ||||
| 			e.logger.Debug("frame mismatch on input/output") | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	txn, err := e.clockStore.NewTransaction() | ||||
| 	latest, err := e.masterTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		e.logger.Error("error while creating transaction", zap.Error(err)) | ||||
| 		return nil, errors.Wrap(err, "confirm latest frame") | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	for _, frame := range committedSet { | ||||
| 		frame := frame | ||||
| 		if err = e.clockStore.PutMasterClockFrame(frame, txn); err != nil { | ||||
| 			e.logger.Error("error while committing frame", zap.Error(err)) | ||||
| 			return nil, errors.Wrap(err, "confirm latest frame") | ||||
| 	if e.syncingStatus == SyncStatusNotSyncing { | ||||
| 		peer, err := e.pubSub.GetRandomPeer(e.filter) | ||||
| 		if err != nil { | ||||
| 			if errors.Is(err, p2p.ErrNoPeersAvailable) { | ||||
| 				e.logger.Debug("no peers available, skipping sync") | ||||
| 			} else { | ||||
| 				e.logger.Error("error while fetching random peer", zap.Error(err)) | ||||
| 			} | ||||
| 		} else { | ||||
| 			e.syncingStatus = SyncStatusAwaitingResponse | ||||
| 			e.logger.Debug("setting syncing target", zap.Binary("peer_id", peer)) | ||||
| 			e.syncingTarget = peer | ||||
| 
 | ||||
| 			channel := e.createPeerReceiveChannel(peer) | ||||
| 			e.logger.Debug( | ||||
| 				"listening on peer receive channel", | ||||
| 				zap.Binary("channel", channel), | ||||
| 			) | ||||
| 			e.pubSub.Subscribe(channel, e.handleSync, true) | ||||
| 			e.pubSub.Subscribe( | ||||
| 				peer, | ||||
| 				func(message *pb.Message) error { return nil }, | ||||
| 				true, | ||||
| 			) | ||||
| 
 | ||||
| 			go func() { | ||||
| 				time.Sleep(2 * time.Second) | ||||
| 				if err := e.publishMessage(peer, &protobufs.ClockFramesRequest{ | ||||
| 					Filter:          e.filter, | ||||
| 					FromFrameNumber: latest.FrameNumber + 1, | ||||
| 				}); err != nil { | ||||
| 					e.logger.Error( | ||||
| 						"could not publish clock frame request", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 				} | ||||
| 			}() | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	if err = txn.Commit(); err != nil { | ||||
| 		e.logger.Error("error while committing transaction", zap.Error(err)) | ||||
| 		return nil, errors.Wrap(err, "confirm latest frame") | ||||
| 	waitDecay := time.Duration(2000) | ||||
| 	for e.syncingStatus != SyncStatusNotSyncing { | ||||
| 		e.logger.Debug( | ||||
| 			"waiting for sync to complete...", | ||||
| 			zap.Duration("wait_decay", waitDecay), | ||||
| 		) | ||||
| 
 | ||||
| 		time.Sleep(waitDecay * time.Millisecond) | ||||
| 
 | ||||
| 		waitDecay = waitDecay * 2 | ||||
| 		if waitDecay >= (100 * (2 << 6)) { | ||||
| 			if e.syncingStatus == SyncStatusAwaitingResponse { | ||||
| 				e.logger.Debug("maximum wait for sync response, skipping sync") | ||||
| 				e.syncingStatus = SyncStatusNotSyncing | ||||
| 				break | ||||
| 			} else { | ||||
| 				waitDecay = 100 * (2 << 6) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	e.logger.Debug("stored frames", zap.Int("frame_count", len(committedSet))) | ||||
| 
 | ||||
| 	e.historicFramesMx.Lock() | ||||
| 
 | ||||
| 	e.historicFrames = append(e.historicFrames, committedSet...) | ||||
| 	if len(e.historicFrames) > 256 { | ||||
| 		e.historicFrames = e.historicFrames[len(e.historicFrames)-256:] | ||||
| 	} | ||||
| 
 | ||||
| 	e.historicFramesMx.Unlock() | ||||
| 
 | ||||
| 	e.setFrame(prev) | ||||
| 
 | ||||
| 	return prev, nil | ||||
| 	return latest, nil | ||||
| } | ||||
|  | ||||
| @ -15,11 +15,16 @@ func (e *MasterClockConsensusEngine) RegisterExecutor( | ||||
| 	errChan := make(chan error) | ||||
| 
 | ||||
| 	go func() { | ||||
| 		masterFrame, err := e.masterTimeReel.Head() | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		logger.Info( | ||||
| 			"starting execution engine at frame", | ||||
| 			zap.Uint64("current_frame", e.frame.FrameNumber), | ||||
| 			zap.Uint64("current_frame", masterFrame.FrameNumber), | ||||
| 		) | ||||
| 		err := <-exec.Start() | ||||
| 		err = <-exec.Start() | ||||
| 		if err != nil { | ||||
| 			logger.Error("could not start execution engine", zap.Error(err)) | ||||
| 			errChan <- err | ||||
| @ -27,13 +32,18 @@ func (e *MasterClockConsensusEngine) RegisterExecutor( | ||||
| 		} | ||||
| 
 | ||||
| 		for { | ||||
| 			masterFrame, err = e.masterTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			logger.Info( | ||||
| 				"awaiting frame", | ||||
| 				zap.Uint64("current_frame", e.frame.FrameNumber), | ||||
| 				zap.Uint64("current_frame", masterFrame.FrameNumber), | ||||
| 				zap.Uint64("target_frame", frame), | ||||
| 			) | ||||
| 
 | ||||
| 			newFrame := e.frame.FrameNumber | ||||
| 			newFrame := masterFrame.FrameNumber | ||||
| 			if newFrame >= frame { | ||||
| 				logger.Info( | ||||
| 					"injecting execution engine at frame", | ||||
| @ -74,13 +84,18 @@ func (e *MasterClockConsensusEngine) UnregisterExecutor( | ||||
| 
 | ||||
| 	go func() { | ||||
| 		for { | ||||
| 			masterFrame, err := e.masterTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			logger.Info( | ||||
| 				"awaiting frame", | ||||
| 				zap.Uint64("current_frame", e.frame.FrameNumber), | ||||
| 				zap.Uint64("current_frame", masterFrame.FrameNumber), | ||||
| 				zap.Uint64("target_frame", frame), | ||||
| 			) | ||||
| 
 | ||||
| 			newFrame := e.frame.FrameNumber | ||||
| 			newFrame := masterFrame.FrameNumber | ||||
| 			if newFrame >= frame { | ||||
| 				logger.Info( | ||||
| 					"removing execution engine at frame", | ||||
|  | ||||
| @ -9,6 +9,8 @@ import ( | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus" | ||||
| 	qtime "source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| @ -25,12 +27,12 @@ const ( | ||||
| ) | ||||
| 
 | ||||
| type MasterClockConsensusEngine struct { | ||||
| 	frame               *protobufs.ClockFrame | ||||
| 	difficulty          uint32 | ||||
| 	logger              *zap.Logger | ||||
| 	state               consensus.EngineState | ||||
| 	pubSub              p2p.PubSub | ||||
| 	keyManager          keys.KeyManager | ||||
| 	frameProver         crypto.FrameProver | ||||
| 	lastFrameReceivedAt time.Time | ||||
| 
 | ||||
| 	frameChan        chan *protobufs.ClockFrame | ||||
| @ -45,6 +47,7 @@ type MasterClockConsensusEngine struct { | ||||
| 	seenFrames       []*protobufs.ClockFrame | ||||
| 	historicFrames   []*protobufs.ClockFrame | ||||
| 	clockStore       store.ClockStore | ||||
| 	masterTimeReel   *qtime.MasterTimeReel | ||||
| } | ||||
| 
 | ||||
| var _ consensus.ConsensusEngine = (*MasterClockConsensusEngine)(nil) | ||||
| @ -55,6 +58,8 @@ func NewMasterClockConsensusEngine( | ||||
| 	clockStore store.ClockStore, | ||||
| 	keyManager keys.KeyManager, | ||||
| 	pubSub p2p.PubSub, | ||||
| 	frameProver crypto.FrameProver, | ||||
| 	masterTimeReel *qtime.MasterTimeReel, | ||||
| ) *MasterClockConsensusEngine { | ||||
| 	if logger == nil { | ||||
| 		panic(errors.New("logger is nil")) | ||||
| @ -72,13 +77,20 @@ func NewMasterClockConsensusEngine( | ||||
| 		panic(errors.New("pubsub is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if frameProver == nil { | ||||
| 		panic(errors.New("frame prover is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	if masterTimeReel == nil { | ||||
| 		panic(errors.New("master time reel is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	seed, err := hex.DecodeString(engineConfig.GenesisSeed) | ||||
| 	if err != nil { | ||||
| 		panic(errors.New("genesis seed is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	e := &MasterClockConsensusEngine{ | ||||
| 		frame:               nil, | ||||
| 		difficulty:          10000, | ||||
| 		logger:              logger, | ||||
| 		state:               consensus.EngineStateStopped, | ||||
| @ -90,15 +102,8 @@ func NewMasterClockConsensusEngine( | ||||
| 		lastFrameReceivedAt: time.Time{}, | ||||
| 		syncingStatus:       SyncStatusNotSyncing, | ||||
| 		clockStore:          clockStore, | ||||
| 	} | ||||
| 
 | ||||
| 	latestFrame, err := e.clockStore.GetLatestMasterClockFrame(e.filter) | ||||
| 	if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if latestFrame != nil { | ||||
| 		e.frame = latestFrame | ||||
| 		frameProver:         frameProver, | ||||
| 		masterTimeReel:      masterTimeReel, | ||||
| 	} | ||||
| 
 | ||||
| 	if e.filter, err = hex.DecodeString(engineConfig.Filter); err != nil { | ||||
| @ -118,28 +123,17 @@ func (e *MasterClockConsensusEngine) Start() <-chan error { | ||||
| 	e.state = consensus.EngineStateLoading | ||||
| 	e.logger.Info("syncing last seen state") | ||||
| 
 | ||||
| 	latestFrame, err := e.clockStore.GetLatestMasterClockFrame(e.filter) | ||||
| 	if err != nil && errors.Is(err, store.ErrNotFound) { | ||||
| 		latestFrame = e.CreateGenesisFrame() | ||||
| 		txn, err := e.clockStore.NewTransaction() | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		if err = e.clockStore.PutMasterClockFrame(latestFrame, txn); err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		if err = txn.Commit(); err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 	} else if err != nil { | ||||
| 	err := e.masterTimeReel.Start() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} else { | ||||
| 		e.setFrame(latestFrame) | ||||
| 	} | ||||
| 
 | ||||
| 	e.buildHistoricFrameCache(latestFrame) | ||||
| 	frame, err := e.masterTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	e.buildHistoricFrameCache(frame) | ||||
| 
 | ||||
| 	e.logger.Info("subscribing to pubsub messages") | ||||
| 	e.pubSub.Subscribe(e.filter, e.handleMessage, true) | ||||
| @ -159,25 +153,39 @@ func (e *MasterClockConsensusEngine) Start() <-chan error { | ||||
| 	}() | ||||
| 
 | ||||
| 	go func() { | ||||
| 		newFrameCh := e.masterTimeReel.NewFrameCh() | ||||
| 
 | ||||
| 		for e.state < consensus.EngineStateStopping { | ||||
| 			var err error | ||||
| 			switch e.state { | ||||
| 			case consensus.EngineStateCollecting: | ||||
| 				currentFrame := latestFrame | ||||
| 				if latestFrame, err = e.collect(latestFrame); err != nil { | ||||
| 			select { | ||||
| 			case frame := <-newFrameCh: | ||||
| 				currentFrame := frame | ||||
| 				latestFrame := frame | ||||
| 				if latestFrame, err = e.collect(currentFrame); err != nil { | ||||
| 					e.logger.Error("could not collect", zap.Error(err)) | ||||
| 					latestFrame = currentFrame | ||||
| 					continue | ||||
| 				} | ||||
| 			case consensus.EngineStateProving: | ||||
| 				currentFrame := latestFrame | ||||
| 				if latestFrame, err = e.prove(latestFrame); err != nil { | ||||
| 					e.logger.Error("could not prove", zap.Error(err)) | ||||
| 					latestFrame = currentFrame | ||||
| 				} | ||||
| 			case consensus.EngineStatePublishing: | ||||
| 				if err = e.publishProof(latestFrame); err != nil { | ||||
| 					e.logger.Error("could not publish", zap.Error(err)) | ||||
| 				} | ||||
| 			case <-time.After(20 * time.Second): | ||||
| 				frame, err := e.masterTimeReel.Head() | ||||
| 				if err != nil { | ||||
| 					panic(err) | ||||
| 				} | ||||
| 
 | ||||
| 				if frame, err = e.prove(frame); err != nil { | ||||
| 					e.logger.Error("could not prove", zap.Error(err)) | ||||
| 					continue | ||||
| 				} | ||||
| 				if err = e.publishProof(frame); err != nil { | ||||
| 					e.logger.Error("could not publish", zap.Error(err)) | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 	}() | ||||
| @ -199,7 +207,13 @@ func (e *MasterClockConsensusEngine) Stop(force bool) <-chan error { | ||||
| 	for name := range e.executionEngines { | ||||
| 		name := name | ||||
| 		go func(name string) { | ||||
| 			err := <-e.UnregisterExecutor(name, e.frame.FrameNumber, force) | ||||
| 			frame, err := e.masterTimeReel.Head() | ||||
| 			if err != nil { | ||||
| 				errChan <- err | ||||
| 				return | ||||
| 			} | ||||
| 
 | ||||
| 			err = <-e.UnregisterExecutor(name, frame.FrameNumber, force) | ||||
| 			if err != nil { | ||||
| 				errChan <- err | ||||
| 			} | ||||
| @ -210,6 +224,7 @@ func (e *MasterClockConsensusEngine) Stop(force bool) <-chan error { | ||||
| 	e.logger.Info("waiting for execution engines to stop") | ||||
| 	wg.Wait() | ||||
| 	e.logger.Info("execution engines stopped") | ||||
| 	e.masterTimeReel.Stop() | ||||
| 
 | ||||
| 	e.state = consensus.EngineStateStopped | ||||
| 	go func() { | ||||
| @ -223,7 +238,12 @@ func (e *MasterClockConsensusEngine) GetDifficulty() uint32 { | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) GetFrame() *protobufs.ClockFrame { | ||||
| 	return e.frame | ||||
| 	frame, err := e.masterTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame | ||||
| } | ||||
| 
 | ||||
| func (e *MasterClockConsensusEngine) GetState() consensus.EngineState { | ||||
|  | ||||
| @ -96,7 +96,7 @@ func (e *MasterClockConsensusEngine) handleClockFramesResponse( | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 		) | ||||
| 
 | ||||
| 		if err := frame.VerifyMasterClockFrame(); err != nil { | ||||
| 		if err := e.frameProver.VerifyMasterClockFrame(frame); err != nil { | ||||
| 			e.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "handle clock frame response") | ||||
| 		} | ||||
| @ -108,12 +108,7 @@ func (e *MasterClockConsensusEngine) handleClockFramesResponse( | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 		) | ||||
| 
 | ||||
| 		if e.frame.FrameNumber < frame.FrameNumber { | ||||
| 			if err := e.enqueueSeenFrame(frame); err != nil { | ||||
| 				e.logger.Error("could not enqueue seen clock frame", zap.Error(err)) | ||||
| 				return errors.Wrap(err, "handle clock frame response") | ||||
| 			} | ||||
| 		} | ||||
| 		e.masterTimeReel.Insert(frame) | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| @ -145,7 +140,12 @@ func (e *MasterClockConsensusEngine) handleClockFramesRequest( | ||||
| 
 | ||||
| 	from := request.FromFrameNumber | ||||
| 
 | ||||
| 	if e.frame.FrameNumber < from || len(e.historicFrames) == 0 { | ||||
| 	masterFrame, err := e.masterTimeReel.Head() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if masterFrame.FrameNumber < from || len(e.historicFrames) == 0 { | ||||
| 		e.logger.Debug( | ||||
| 			"peer asked for undiscovered frame", | ||||
| 			zap.Binary("peer_id", peerID), | ||||
| @ -169,8 +169,8 @@ func (e *MasterClockConsensusEngine) handleClockFramesRequest( | ||||
| 		to = request.FromFrameNumber + 127 | ||||
| 	} | ||||
| 
 | ||||
| 	if int(to) > int(e.frame.FrameNumber) { | ||||
| 		to = e.frame.FrameNumber | ||||
| 	if int(to) > int(masterFrame.FrameNumber) { | ||||
| 		to = masterFrame.FrameNumber | ||||
| 	} | ||||
| 
 | ||||
| 	e.logger.Debug( | ||||
|  | ||||
							
								
								
									
										607
									
								
								node/consensus/time/data_time_reel.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										607
									
								
								node/consensus/time/data_time_reel.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,607 @@ | ||||
| package time | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"math/big" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| ) | ||||
| 
 | ||||
| var allBitmaskFilter = []byte{ | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| } | ||||
| 
 | ||||
| var unknownDistance = new(big.Int).SetBytes([]byte{ | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| }) | ||||
| 
 | ||||
| type pendingFrame struct { | ||||
| 	parentSelector *big.Int | ||||
| 	distance       *big.Int | ||||
| } | ||||
| 
 | ||||
| type DataTimeReel struct { | ||||
| 	rwMutex sync.RWMutex | ||||
| 
 | ||||
| 	filter         []byte | ||||
| 	engineConfig   *config.EngineConfig | ||||
| 	logger         *zap.Logger | ||||
| 	clockStore     store.ClockStore | ||||
| 	frameProver    crypto.FrameProver | ||||
| 	parentTimeReel TimeReel | ||||
| 
 | ||||
| 	origin                []byte | ||||
| 	initialInclusionProof *crypto.InclusionAggregateProof | ||||
| 	initialProverKeys     [][]byte | ||||
| 	head                  *protobufs.ClockFrame | ||||
| 	totalDistance         *big.Int | ||||
| 	headDistance          *big.Int | ||||
| 	proverTrie            *tries.RollingFrecencyCritbitTrie | ||||
| 	pending               map[uint64][]*pendingFrame | ||||
| 	incompleteForks       map[uint64][]*pendingFrame | ||||
| 	frames                chan *protobufs.ClockFrame | ||||
| 	newFrameCh            chan *protobufs.ClockFrame | ||||
| 	badFrameCh            chan *protobufs.ClockFrame | ||||
| 	done                  chan bool | ||||
| } | ||||
| 
 | ||||
| func NewDataTimeReel( | ||||
| 	filter []byte, | ||||
| 	logger *zap.Logger, | ||||
| 	clockStore store.ClockStore, | ||||
| 	engineConfig *config.EngineConfig, | ||||
| 	frameProver crypto.FrameProver, | ||||
| 	origin []byte, | ||||
| 	initialInclusionProof *crypto.InclusionAggregateProof, | ||||
| 	initialProverKeys [][]byte, | ||||
| ) *DataTimeReel { | ||||
| 	if filter == nil { | ||||
| 		panic("filter is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if logger == nil { | ||||
| 		panic("logger is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if clockStore == nil { | ||||
| 		panic("clock store is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if engineConfig == nil { | ||||
| 		panic("engine config is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if frameProver == nil { | ||||
| 		panic("frame prover is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	return &DataTimeReel{ | ||||
| 		logger:                logger, | ||||
| 		filter:                filter, | ||||
| 		engineConfig:          engineConfig, | ||||
| 		clockStore:            clockStore, | ||||
| 		frameProver:           frameProver, | ||||
| 		origin:                origin, | ||||
| 		initialInclusionProof: initialInclusionProof, | ||||
| 		initialProverKeys:     initialProverKeys, | ||||
| 		pending:               make(map[uint64][]*pendingFrame), | ||||
| 		incompleteForks:       make(map[uint64][]*pendingFrame), | ||||
| 		frames:                make(chan *protobufs.ClockFrame), | ||||
| 		newFrameCh:            make(chan *protobufs.ClockFrame), | ||||
| 		badFrameCh:            make(chan *protobufs.ClockFrame), | ||||
| 		done:                  make(chan bool), | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) Start() error { | ||||
| 	trie := &tries.RollingFrecencyCritbitTrie{} | ||||
| 	frame, err := d.clockStore.GetLatestDataClockFrame(d.filter, trie) | ||||
| 	if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if frame == nil { | ||||
| 		d.head, d.proverTrie = d.createGenesisFrame() | ||||
| 		d.totalDistance = big.NewInt(0) | ||||
| 	} else { | ||||
| 		d.head = frame | ||||
| 		d.proverTrie = trie | ||||
| 		d.totalDistance = d.getTotalDistance(frame) | ||||
| 	} | ||||
| 
 | ||||
| 	go d.runLoop() | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) Head() (*protobufs.ClockFrame, error) { | ||||
| 	return d.head, nil | ||||
| } | ||||
| 
 | ||||
| // Insert enqueues a structurally valid frame into the time reel. If the frame
 | ||||
| // is the next one in sequence, it advances the reel head forward and emits a
 | ||||
| // new frame on the new frame channel.
 | ||||
| func (d *DataTimeReel) Insert(frame *protobufs.ClockFrame) error { | ||||
| 	go func() { | ||||
| 		d.frames <- frame | ||||
| 	}() | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) GetFrameProverTrie() *tries.RollingFrecencyCritbitTrie { | ||||
| 	return d.proverTrie | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) NewFrameCh() <-chan *protobufs.ClockFrame { | ||||
| 	return d.newFrameCh | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) BadFrameCh() <-chan *protobufs.ClockFrame { | ||||
| 	return d.badFrameCh | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) Stop() { | ||||
| 	d.done <- true | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) createGenesisFrame() ( | ||||
| 	*protobufs.ClockFrame, | ||||
| 	*tries.RollingFrecencyCritbitTrie, | ||||
| ) { | ||||
| 	if d.origin == nil { | ||||
| 		panic("origin is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if d.initialInclusionProof == nil { | ||||
| 		panic("initial inclusion proof is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if d.initialProverKeys == nil { | ||||
| 		panic("initial prover keys is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	frame, trie, err := d.frameProver.CreateDataGenesisFrame( | ||||
| 		d.filter, | ||||
| 		d.origin, | ||||
| 		d.engineConfig.Difficulty, | ||||
| 		d.initialInclusionProof, | ||||
| 		d.initialProverKeys, | ||||
| 		true, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	txn, err := d.clockStore.NewTransaction() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err := d.clockStore.PutDataClockFrame( | ||||
| 		frame, | ||||
| 		trie, | ||||
| 		txn, | ||||
| 		false, | ||||
| 	); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err := txn.Commit(); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, trie | ||||
| } | ||||
| 
 | ||||
| // Main data consensus loop
 | ||||
| func (d *DataTimeReel) runLoop() { | ||||
| 	for { | ||||
| 		select { | ||||
| 		case frame := <-d.frames: | ||||
| 			// Most common scenario: in order – new frame is higher number
 | ||||
| 			if d.head.FrameNumber < frame.FrameNumber { | ||||
| 				parent := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 				selector, err := frame.GetSelector() | ||||
| 				if err != nil { | ||||
| 					panic(err) | ||||
| 				} | ||||
| 
 | ||||
| 				distance, err := d.GetDistance(frame) | ||||
| 				if err != nil { | ||||
| 					// If the frame arrived ahead of a master, e.g. the master data is not
 | ||||
| 					// synced, we'll go ahead and mark it as pending and process it when
 | ||||
| 					// we can, but if we had a general fault, panic:
 | ||||
| 					if !errors.Is(err, store.ErrNotFound) { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					d.addPending(selector, parent, unknownDistance, frame) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				// If the frame has a gap from the head, mark it as pending:
 | ||||
| 				if frame.FrameNumber-d.head.FrameNumber != 1 { | ||||
| 					d.addPending(selector, parent, distance, frame) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				// Otherwise set it as the next and process all pending
 | ||||
| 				d.setHead(frame, distance) | ||||
| 				d.processPending(frame) | ||||
| 			} else if d.head.FrameNumber == frame.FrameNumber { | ||||
| 				// frames are equivalent, no need to act
 | ||||
| 				if bytes.Equal(d.head.Output, frame.Output) { | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				distance, err := d.GetDistance(frame) | ||||
| 				if err != nil { | ||||
| 					panic(err) | ||||
| 				} | ||||
| 
 | ||||
| 				// Optimization: if competing frames share a parent we can short-circuit
 | ||||
| 				// fork choice
 | ||||
| 				if bytes.Equal(d.head.ParentSelector, frame.ParentSelector) && | ||||
| 					distance.Cmp(d.headDistance) < 0 { | ||||
| 					d.totalDistance.Sub(d.totalDistance, d.headDistance) | ||||
| 					d.setHead(frame, distance) | ||||
| 				} | ||||
| 
 | ||||
| 				// Choose fork
 | ||||
| 				d.forkChoice(frame, distance) | ||||
| 			} else { | ||||
| 				// tag: dusk – we should have some kind of check here to avoid brutal
 | ||||
| 				// thrashing
 | ||||
| 				existing, _, err := d.clockStore.GetDataClockFrame( | ||||
| 					d.filter, | ||||
| 					frame.FrameNumber, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					// if this returns an error it's either not found (which shouldn't
 | ||||
| 					// happen without corruption) or pebble is borked, either way, panic
 | ||||
| 					panic(err) | ||||
| 				} | ||||
| 
 | ||||
| 				// It's a fork, but it's behind. We need to stash it until it catches
 | ||||
| 				// up (or dies off)
 | ||||
| 				if !bytes.Equal(existing.Output, frame.Output) { | ||||
| 					distance, err := d.GetDistance(frame) | ||||
| 					if err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					parent, selector, err := frame.GetParentAndSelector() | ||||
| 					if err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					d.addPending(selector, parent, distance, frame) | ||||
| 					d.processPending(d.head) | ||||
| 				} | ||||
| 			} | ||||
| 		case <-d.done: | ||||
| 			return | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) addPending( | ||||
| 	selector *big.Int, | ||||
| 	parent *big.Int, | ||||
| 	distance *big.Int, | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) { | ||||
| 	if _, ok := d.pending[frame.FrameNumber]; !ok { | ||||
| 		d.pending[frame.FrameNumber] = []*pendingFrame{} | ||||
| 	} | ||||
| 
 | ||||
| 	txn, err := d.clockStore.NewTransaction() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if distance.Cmp(unknownDistance) == 0 { | ||||
| 		distance = new(big.Int).Set(unknownDistance) | ||||
| 		distance.Sub(distance, big.NewInt(int64(len(d.pending[frame.FrameNumber])))) | ||||
| 	} | ||||
| 
 | ||||
| 	err = d.clockStore.PutCandidateDataClockFrame( | ||||
| 		parent.FillBytes(make([]byte, 32)), | ||||
| 		distance.FillBytes(make([]byte, 32)), | ||||
| 		selector.FillBytes(make([]byte, 32)), | ||||
| 		frame, | ||||
| 		txn, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		txn.Abort() | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err = txn.Commit(); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	d.pending[frame.FrameNumber] = append( | ||||
| 		d.pending[frame.FrameNumber], | ||||
| 		&pendingFrame{ | ||||
| 			parentSelector: parent, | ||||
| 			distance:       distance, | ||||
| 		}, | ||||
| 	) | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) processPending(frame *protobufs.ClockFrame) { | ||||
| 	neighbors := false | ||||
| 	// Flush the current pending frames
 | ||||
| 	neighborPending, ok := d.pending[frame.FrameNumber] | ||||
| 	for ok && neighborPending != nil { | ||||
| 		next := neighborPending[0] | ||||
| 		d.pending[frame.FrameNumber] = | ||||
| 			d.pending[frame.FrameNumber][1:] | ||||
| 		if len(d.pending[frame.FrameNumber]) == 0 { | ||||
| 			delete(d.pending, frame.FrameNumber) | ||||
| 			break | ||||
| 		} | ||||
| 
 | ||||
| 		nextFrame, err := d.clockStore.GetCandidateDataClockFrame( | ||||
| 			d.filter, | ||||
| 			frame.FrameNumber, | ||||
| 			next.parentSelector.FillBytes(make([]byte, 32)), | ||||
| 			next.distance.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 		if nextFrame != nil { | ||||
| 			neighbors = true | ||||
| 			go func() { | ||||
| 				d.frames <- nextFrame | ||||
| 			}() | ||||
| 		} | ||||
| 		neighborPending, ok = d.pending[frame.FrameNumber] | ||||
| 	} | ||||
| 
 | ||||
| 	if !neighbors { | ||||
| 		// Pull the next
 | ||||
| 		nextPending, ok := d.pending[frame.FrameNumber+1] | ||||
| 		if ok { | ||||
| 			next := nextPending[0] | ||||
| 			d.pending[frame.FrameNumber+1] = | ||||
| 				d.pending[frame.FrameNumber+1][1:] | ||||
| 			if len(d.pending[frame.FrameNumber+1]) == 0 { | ||||
| 				delete(d.pending, frame.FrameNumber+1) | ||||
| 			} | ||||
| 
 | ||||
| 			nextFrame, err := d.clockStore.GetCandidateDataClockFrame( | ||||
| 				d.filter, | ||||
| 				frame.FrameNumber+1, | ||||
| 				next.parentSelector.FillBytes(make([]byte, 32)), | ||||
| 				next.distance.FillBytes(make([]byte, 32)), | ||||
| 			) | ||||
| 			if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 			if nextFrame != nil { | ||||
| 				go func() { | ||||
| 					d.frames <- nextFrame | ||||
| 				}() | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) setHead(frame *protobufs.ClockFrame, distance *big.Int) { | ||||
| 	txn, err := d.clockStore.NewTransaction() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err := d.clockStore.PutDataClockFrame( | ||||
| 		frame, | ||||
| 		d.proverTrie, | ||||
| 		txn, | ||||
| 		false, | ||||
| 	); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err = txn.Commit(); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	d.head = frame | ||||
| 	d.totalDistance.Add(d.totalDistance, distance) | ||||
| 	d.headDistance = distance | ||||
| 	go func() { | ||||
| 		d.newFrameCh <- frame | ||||
| 	}() | ||||
| } | ||||
| 
 | ||||
| // tag: dusk – store the distance with the frame
 | ||||
| func (d *DataTimeReel) getTotalDistance(frame *protobufs.ClockFrame) *big.Int { | ||||
| 	total, err := d.GetDistance(frame) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	for index := frame; err == nil && | ||||
| 		index.FrameNumber > 0; index, err = d.clockStore.GetParentDataClockFrame( | ||||
| 		d.filter, | ||||
| 		index.FrameNumber-1, | ||||
| 		index.ParentSelector, | ||||
| 	) { | ||||
| 		distance, err := d.GetDistance(index) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		total.Add(total, distance) | ||||
| 	} | ||||
| 
 | ||||
| 	return total | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) GetDistance(frame *protobufs.ClockFrame) ( | ||||
| 	*big.Int, | ||||
| 	error, | ||||
| ) { | ||||
| 	// tag: equinox – master filter changes
 | ||||
| 	master, err := d.clockStore.GetMasterClockFrame( | ||||
| 		allBitmaskFilter, | ||||
| 		frame.FrameNumber) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "get distance") | ||||
| 	} | ||||
| 
 | ||||
| 	masterSelector, err := master.GetSelector() | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "get distance") | ||||
| 	} | ||||
| 
 | ||||
| 	discriminatorNode := | ||||
| 		d.proverTrie.FindNearest(masterSelector.FillBytes(make([]byte, 32))) | ||||
| 	discriminator := discriminatorNode.External.Key | ||||
| 	addr, err := frame.GetAddress() | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "get distance") | ||||
| 	} | ||||
| 	distance := new(big.Int).Sub( | ||||
| 		new(big.Int).SetBytes(discriminator), | ||||
| 		new(big.Int).SetBytes(addr), | ||||
| 	) | ||||
| 	distance.Abs(distance) | ||||
| 
 | ||||
| 	return distance, nil | ||||
| } | ||||
| 
 | ||||
| func (d *DataTimeReel) forkChoice( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| 	distance *big.Int, | ||||
| ) { | ||||
| 	parentSelector, selector, err := frame.GetParentAndSelector() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	leftIndex := d.head | ||||
| 	rightIndex := frame | ||||
| 	leftTotal := new(big.Int).Set(d.headDistance) | ||||
| 	rightTotal := new(big.Int).Set(distance) | ||||
| 	left := d.head.ParentSelector | ||||
| 	right := frame.ParentSelector | ||||
| 
 | ||||
| 	rightReplaySelectors := [][]byte{selector.FillBytes(make([]byte, 32))} | ||||
| 
 | ||||
| 	// Walk backwards through the parents, until we find a matching parent
 | ||||
| 	// selector:
 | ||||
| 	for !bytes.Equal(left, right) { | ||||
| 		rightReplaySelectors = append( | ||||
| 			append( | ||||
| 				[][]byte{}, | ||||
| 				right, | ||||
| 			), | ||||
| 			rightReplaySelectors..., | ||||
| 		) | ||||
| 		leftIndex, err = d.clockStore.GetParentDataClockFrame( | ||||
| 			d.filter, | ||||
| 			leftIndex.FrameNumber-1, | ||||
| 			leftIndex.ParentSelector, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		rightIndex, err = d.clockStore.GetParentDataClockFrame( | ||||
| 			d.filter, | ||||
| 			rightIndex.FrameNumber-1, | ||||
| 			rightIndex.ParentSelector, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			// If lineage cannot be verified, set it for later
 | ||||
| 			if errors.Is(err, store.ErrNotFound) { | ||||
| 				d.addPending(selector, parentSelector, distance, frame) | ||||
| 				return | ||||
| 			} else { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		left = leftIndex.ParentSelector | ||||
| 		right = rightIndex.ParentSelector | ||||
| 		leftIndexDistance, err := d.GetDistance(leftIndex) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		rightIndexDistance, err := d.GetDistance(rightIndex) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		leftTotal.Add(leftTotal, leftIndexDistance) | ||||
| 		rightTotal.Add(rightTotal, rightIndexDistance) | ||||
| 	} | ||||
| 
 | ||||
| 	// Choose new fork based on lightest distance sub-tree
 | ||||
| 	if rightTotal.Cmp(leftTotal) < 0 { | ||||
| 		for { | ||||
| 			if len(rightReplaySelectors) == 0 { | ||||
| 				break | ||||
| 			} | ||||
| 			next := rightReplaySelectors[0] | ||||
| 			rightReplaySelectors[frame.FrameNumber] = | ||||
| 				rightReplaySelectors[frame.FrameNumber][1:] | ||||
| 
 | ||||
| 			rightIndex, err = d.clockStore.GetParentDataClockFrame( | ||||
| 				d.filter, | ||||
| 				rightIndex.FrameNumber+1, | ||||
| 				next, | ||||
| 			) | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			txn, err := d.clockStore.NewTransaction() | ||||
| 			if err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			if err := d.clockStore.PutDataClockFrame( | ||||
| 				rightIndex, | ||||
| 				d.proverTrie, | ||||
| 				txn, | ||||
| 				false, | ||||
| 			); err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 
 | ||||
| 			if err = txn.Commit(); err != nil { | ||||
| 				panic(err) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		d.head = frame | ||||
| 		d.totalDistance.Sub(d.totalDistance, leftTotal) | ||||
| 		d.totalDistance.Add(d.totalDistance, rightTotal) | ||||
| 		d.headDistance = distance | ||||
| 		go func() { | ||||
| 			d.newFrameCh <- frame | ||||
| 		}() | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| var _ TimeReel = (*DataTimeReel)(nil) | ||||
							
								
								
									
										407
									
								
								node/consensus/time/data_time_reel_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										407
									
								
								node/consensus/time/data_time_reel_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,407 @@ | ||||
| package time_test | ||||
| 
 | ||||
| import ( | ||||
| 	"encoding/hex" | ||||
| 	"fmt" | ||||
| 	"strings" | ||||
| 	"sync" | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/cloudflare/circl/sign/ed448" | ||||
| 	"github.com/iden3/go-iden3-crypto/poseidon" | ||||
| 	"github.com/libp2p/go-libp2p/core/crypto" | ||||
| 	"github.com/libp2p/go-libp2p/core/peer" | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	qcrypto "source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| ) | ||||
| 
 | ||||
| func generateTestProvers() ( | ||||
| 	keys.KeyManager, | ||||
| 	[]peer.ID, | ||||
| 	[][]byte, | ||||
| 	[][]byte, | ||||
| 	map[string]string, | ||||
| 	*tries.RollingFrecencyCritbitTrie, | ||||
| ) { | ||||
| 	keyManager := keys.NewInMemoryKeyManager() | ||||
| 	peers := []peer.ID{} | ||||
| 	pubKeys := [][]byte{} | ||||
| 	privKeys := [][]byte{} | ||||
| 	addrMap := map[string]string{} | ||||
| 	for i := 0; i < 1000; i++ { | ||||
| 		keyManager.CreateSigningKey( | ||||
| 			fmt.Sprintf("test-key-%d", i), | ||||
| 			keys.KeyTypeEd448, | ||||
| 		) | ||||
| 		k, err := keyManager.GetRawKey(fmt.Sprintf("test-key-%d", i)) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		privKey, err := crypto.UnmarshalEd448PrivateKey([]byte(k.PrivateKey)) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		privKeys = append(privKeys, []byte(k.PrivateKey)) | ||||
| 
 | ||||
| 		pub := privKey.GetPublic() | ||||
| 		id, err := peer.IDFromPublicKey(pub) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		peers = append(peers, id) | ||||
| 
 | ||||
| 		keyManager.CreateSigningKey( | ||||
| 			fmt.Sprintf("proving-key-%d", i), | ||||
| 			keys.KeyTypeEd448, | ||||
| 		) | ||||
| 		pk, err := keyManager.GetRawKey(fmt.Sprintf("proving-key-%d", i)) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		pprivKey, err := crypto.UnmarshalEd448PrivateKey([]byte(pk.PrivateKey)) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		ppub := pprivKey.GetPublic() | ||||
| 		ppubKey, err := ppub.Raw() | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		pubKeys = append(pubKeys, ppubKey) | ||||
| 	} | ||||
| 
 | ||||
| 	proverTrie := &tries.RollingFrecencyCritbitTrie{} | ||||
| 
 | ||||
| 	for i, s := range pubKeys { | ||||
| 		addr, err := poseidon.HashBytes(s) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		addrBytes := addr.Bytes() | ||||
| 		addrBytes = append(make([]byte, 32-len(addrBytes)), addrBytes...) | ||||
| 		proverTrie.Add(addrBytes, 0) | ||||
| 		addrMap[string(addrBytes)] = fmt.Sprintf("proving-key-%d", i) | ||||
| 	} | ||||
| 
 | ||||
| 	return keyManager, | ||||
| 		peers, | ||||
| 		pubKeys, | ||||
| 		privKeys, | ||||
| 		addrMap, | ||||
| 		proverTrie | ||||
| } | ||||
| 
 | ||||
| func TestDataTimeReel(t *testing.T) { | ||||
| 	logger, _ := zap.NewProduction() | ||||
| 	db := store.NewInMemKVDB() | ||||
| 	clockStore := store.NewPebbleClockStore(db, logger) | ||||
| 	prover := qcrypto.NewWesolowskiFrameProver(logger) | ||||
| 	filter := "ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff" | ||||
| 	keyManager, | ||||
| 		_, | ||||
| 		pubKeys, | ||||
| 		_, | ||||
| 		addrMap, | ||||
| 		proverTrie := generateTestProvers() | ||||
| 
 | ||||
| 	// We're going to set this up by churning 100 master frames so we don't
 | ||||
| 	// have to zig zag on master and data frames to confirm data time reel
 | ||||
| 	// behaviors
 | ||||
| 	m := time.NewMasterTimeReel( | ||||
| 		logger, | ||||
| 		clockStore, | ||||
| 		&config.EngineConfig{ | ||||
| 			Filter:      filter, | ||||
| 			GenesisSeed: strings.Repeat("00", 516), | ||||
| 			Difficulty:  10, | ||||
| 		}, | ||||
| 		prover, | ||||
| 	) | ||||
| 
 | ||||
| 	err := m.Start() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frame, err := m.Head() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frames := []*protobufs.ClockFrame{} | ||||
| 	wg := sync.WaitGroup{} | ||||
| 	wg.Add(1) | ||||
| 	frameCh := m.NewFrameCh() | ||||
| 	go func() { | ||||
| 		for i := 0; i < 100; i++ { | ||||
| 			frames = append(frames, <-frameCh) | ||||
| 		} | ||||
| 		wg.Done() | ||||
| 	}() | ||||
| 
 | ||||
| 	// in order
 | ||||
| 	for i := int64(0); i < 100; i++ { | ||||
| 		frame, err = prover.ProveMasterClockFrame(frame, i+1, 10) | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		err := m.Insert(frame) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	for i := 0; i < 100; i++ { | ||||
| 		assert.NotNil(t, frames[i]) | ||||
| 		assert.Equal(t, frames[i].FrameNumber, uint64(i+1)) | ||||
| 	} | ||||
| 
 | ||||
| 	filterBytes, _ := hex.DecodeString(filter) | ||||
| 
 | ||||
| 	// Ok, now onto the data time reel. We're going to test the following
 | ||||
| 	// scenarios:
 | ||||
| 	// 1. Z-dist optimal, in order
 | ||||
| 	// 2. Z-dist optimal, out of order
 | ||||
| 	// 3. 90% optimal, out of order
 | ||||
| 	// 4. Malicious majority, out of order
 | ||||
| 	d := time.NewDataTimeReel( | ||||
| 		filterBytes, | ||||
| 		logger, | ||||
| 		clockStore, | ||||
| 		&config.EngineConfig{ | ||||
| 			Filter:      filter, | ||||
| 			GenesisSeed: strings.Repeat("00", 516), | ||||
| 			Difficulty:  10, | ||||
| 		}, | ||||
| 		prover, | ||||
| 		frames[0].Output, | ||||
| 		&qcrypto.InclusionAggregateProof{ | ||||
| 			InclusionCommitments: []*qcrypto.InclusionCommitment{}, | ||||
| 			AggregateCommitment:  []byte{}, | ||||
| 			Proof:                []byte{}, | ||||
| 		}, | ||||
| 		pubKeys, | ||||
| 	) | ||||
| 
 | ||||
| 	err = d.Start() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frame, err = d.Head() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	dataFrames := []*protobufs.ClockFrame{} | ||||
| 	datawg := sync.WaitGroup{} | ||||
| 	datawg.Add(1) | ||||
| 	dataFrameCh := d.NewFrameCh() | ||||
| 	go func() { | ||||
| 		for i := 0; i < 40; i++ { | ||||
| 			dataFrames = append(dataFrames, <-dataFrameCh) | ||||
| 		} | ||||
| 		datawg.Done() | ||||
| 	}() | ||||
| 
 | ||||
| 	// 1. z-dist optimal – proof submission is strictly master-frame evoked leader
 | ||||
| 	for i := int64(0); i < 10; i++ { | ||||
| 		masterSelector, err := frames[i+1].GetSelector() | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		proverSelection := proverTrie.FindNearest( | ||||
| 			masterSelector.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		optimalSigner, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelection.External.Key)], | ||||
| 		) | ||||
| 		frame, err = prover.ProveDataClockFrame( | ||||
| 			frame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			optimalSigner, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		d.Insert(frame) | ||||
| 	} | ||||
| 
 | ||||
| 	// 2. z-dist optimal, out of order – proof submission is strictly master-frame
 | ||||
| 	// evoked leader, but arrived completely backwards
 | ||||
| 	insertFrames := []*protobufs.ClockFrame{} | ||||
| 
 | ||||
| 	for i := int64(10); i < 20; i++ { | ||||
| 		masterSelector, err := frames[i+1].GetSelector() | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		proverSelection := proverTrie.FindNearest( | ||||
| 			masterSelector.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		optimalSigner, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelection.External.Key)], | ||||
| 		) | ||||
| 		frame, err = prover.ProveDataClockFrame( | ||||
| 			frame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			optimalSigner, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		insertFrames = append(insertFrames, frame) | ||||
| 	} | ||||
| 
 | ||||
| 	for i := 9; i >= 0; i-- { | ||||
| 		err := d.Insert(insertFrames[i]) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	// 3. 90% optimal, out of order
 | ||||
| 	insertFrames = []*protobufs.ClockFrame{} | ||||
| 
 | ||||
| 	for i := int64(20); i < 25; i++ { | ||||
| 		masterSelector, err := frames[i+1].GetSelector() | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		proverSelection := proverTrie.FindNearest( | ||||
| 			masterSelector.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		optimalSigner, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelection.External.Key)], | ||||
| 		) | ||||
| 		frame, err = prover.ProveDataClockFrame( | ||||
| 			frame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			optimalSigner, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		d.Insert(frame) | ||||
| 	} | ||||
| 
 | ||||
| 	masterSelector, err := frames[26].GetSelector() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	proverSelections := proverTrie.FindNearestAndApproximateNeighbors( | ||||
| 		masterSelector.FillBytes(make([]byte, 32)), | ||||
| 	) | ||||
| 	suboptimalSigner2, _ := keyManager.GetSigningKey( | ||||
| 		addrMap[string(proverSelections[2].External.Key)], | ||||
| 	) | ||||
| 	// What we're trying to simulate: consensus heads progressed on a slightly
 | ||||
| 	// less optimal prover.
 | ||||
| 	frame, err = prover.ProveDataClockFrame( | ||||
| 		frame, | ||||
| 		[][]byte{}, | ||||
| 		[]*protobufs.InclusionAggregateProof{}, | ||||
| 		suboptimalSigner2, | ||||
| 		26, | ||||
| 		10, | ||||
| 	) | ||||
| 	insertFrames = append(insertFrames, frame) | ||||
| 
 | ||||
| 	for i := int64(26); i < 30; i++ { | ||||
| 		masterSelector, err := frames[i+1].GetSelector() | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		proverSelection := proverTrie.FindNearest( | ||||
| 			masterSelector.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		optimalSigner, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelection.External.Key)], | ||||
| 		) | ||||
| 		frame, err = prover.ProveDataClockFrame( | ||||
| 			frame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			optimalSigner, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		insertFrames = append(insertFrames, frame) | ||||
| 	} | ||||
| 
 | ||||
| 	for i := 4; i >= 0; i-- { | ||||
| 		err := d.Insert(insertFrames[i]) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	// 4. Malicious majority, out of order – handle a suppressive majority and
 | ||||
| 	// force consensus on the lowest distance sub-tree:
 | ||||
| 	insertFrames = []*protobufs.ClockFrame{} | ||||
| 	conflictFrames := []*protobufs.ClockFrame{} | ||||
| 	optimalKeySet := [][]byte{} | ||||
| 	suppressedFrame := frame | ||||
| 	for i := int64(30); i < 40; i++ { | ||||
| 		masterSelector, err := frames[i+1].GetSelector() | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		proverSelections := proverTrie.FindNearestAndApproximateNeighbors( | ||||
| 			masterSelector.FillBytes(make([]byte, 32)), | ||||
| 		) | ||||
| 		optimalSigner, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelections[0].External.Key)], | ||||
| 		) | ||||
| 		suboptimalSigner2, _ := keyManager.GetSigningKey( | ||||
| 			addrMap[string(proverSelections[2].External.Key)], | ||||
| 		) | ||||
| 		optimalKeySet = append(optimalKeySet, []byte( | ||||
| 			(optimalSigner.Public()).(ed448.PublicKey), | ||||
| 		)) | ||||
| 
 | ||||
| 		// What we're trying to simulate: the majority is intentionally ignoring
 | ||||
| 		// the most optimal signer
 | ||||
| 		suppressedFrame, err := prover.ProveDataClockFrame( | ||||
| 			suppressedFrame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			optimalSigner, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		insertFrames = append(insertFrames, suppressedFrame) | ||||
| 		frame, err = prover.ProveDataClockFrame( | ||||
| 			frame, | ||||
| 			[][]byte{}, | ||||
| 			[]*protobufs.InclusionAggregateProof{}, | ||||
| 			suboptimalSigner2, | ||||
| 			i+1, | ||||
| 			10, | ||||
| 		) | ||||
| 		conflictFrames = append(conflictFrames, frame) | ||||
| 	} | ||||
| 
 | ||||
| 	for i := 9; i >= 0; i-- { | ||||
| 		err := d.Insert(conflictFrames[i]) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	// Someone is honest, but clumsy:
 | ||||
| 	for i := 9; i >= 0; i-- { | ||||
| 		err := d.Insert(insertFrames[i]) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	datawg.Wait() | ||||
| 
 | ||||
| 	for i := 0; i < 40; i++ { | ||||
| 		assert.NotNil(t, dataFrames[i]) | ||||
| 
 | ||||
| 		if i >= 40 { | ||||
| 			assert.Equal(t, uint64(i), dataFrames[i].FrameNumber) | ||||
| 			assert.Equal( | ||||
| 				t, | ||||
| 				dataFrames[i].GetPublicKeySignatureEd448().PublicKey.KeyValue, | ||||
| 				optimalKeySet[i-30], | ||||
| 			) | ||||
| 		} else { | ||||
| 			assert.Equal(t, uint64(i+1), dataFrames[i].FrameNumber) | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
							
								
								
									
										221
									
								
								node/consensus/time/master_time_reel.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										221
									
								
								node/consensus/time/master_time_reel.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,221 @@ | ||||
| package time | ||||
| 
 | ||||
| import ( | ||||
| 	"encoding/hex" | ||||
| 	"errors" | ||||
| 	"math/big" | ||||
| 	"sync" | ||||
| 
 | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| ) | ||||
| 
 | ||||
| type MasterTimeReel struct { | ||||
| 	rwMutex sync.RWMutex | ||||
| 
 | ||||
| 	filter       []byte | ||||
| 	engineConfig *config.EngineConfig | ||||
| 	logger       *zap.Logger | ||||
| 	clockStore   store.ClockStore | ||||
| 	frameProver  crypto.FrameProver | ||||
| 
 | ||||
| 	head       *protobufs.ClockFrame | ||||
| 	pending    map[uint64][]*big.Int | ||||
| 	frames     chan *protobufs.ClockFrame | ||||
| 	newFrameCh chan *protobufs.ClockFrame | ||||
| 	badFrameCh chan *protobufs.ClockFrame | ||||
| 	done       chan bool | ||||
| } | ||||
| 
 | ||||
| func NewMasterTimeReel( | ||||
| 	logger *zap.Logger, | ||||
| 	clockStore store.ClockStore, | ||||
| 	engineConfig *config.EngineConfig, | ||||
| 	frameProver crypto.FrameProver, | ||||
| ) *MasterTimeReel { | ||||
| 	if logger == nil { | ||||
| 		panic("logger is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if clockStore == nil { | ||||
| 		panic("clock store is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if engineConfig == nil { | ||||
| 		panic("engine config is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	if frameProver == nil { | ||||
| 		panic("frame prover is nil") | ||||
| 	} | ||||
| 
 | ||||
| 	filter, err := hex.DecodeString(engineConfig.Filter) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	return &MasterTimeReel{ | ||||
| 		logger:       logger, | ||||
| 		filter:       filter, | ||||
| 		engineConfig: engineConfig, | ||||
| 		clockStore:   clockStore, | ||||
| 		frameProver:  frameProver, | ||||
| 		pending:      make(map[uint64][]*big.Int), | ||||
| 		frames:       make(chan *protobufs.ClockFrame), | ||||
| 		newFrameCh:   make(chan *protobufs.ClockFrame), | ||||
| 		badFrameCh:   make(chan *protobufs.ClockFrame), | ||||
| 		done:         make(chan bool), | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Start implements TimeReel.
 | ||||
| func (m *MasterTimeReel) Start() error { | ||||
| 	frame, err := m.clockStore.GetLatestMasterClockFrame(m.filter) | ||||
| 	if err != nil && !errors.Is(err, store.ErrNotFound) { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if frame == nil { | ||||
| 		m.head = m.createGenesisFrame() | ||||
| 	} else { | ||||
| 		m.head = frame | ||||
| 	} | ||||
| 
 | ||||
| 	go m.runLoop() | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // Head implements TimeReel.
 | ||||
| func (m *MasterTimeReel) Head() (*protobufs.ClockFrame, error) { | ||||
| 	return m.head, nil | ||||
| } | ||||
| 
 | ||||
| // Insert enqueues a structurally valid frame into the time reel. If the frame
 | ||||
| // is the next one in sequence, it advances the reel head forward and emits a
 | ||||
| // new frame on the new frame channel.
 | ||||
| func (m *MasterTimeReel) Insert(frame *protobufs.ClockFrame) error { | ||||
| 	go func() { | ||||
| 		m.frames <- frame | ||||
| 	}() | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| // NewFrameCh implements TimeReel.
 | ||||
| func (m *MasterTimeReel) NewFrameCh() <-chan *protobufs.ClockFrame { | ||||
| 	return m.newFrameCh | ||||
| } | ||||
| 
 | ||||
| func (m *MasterTimeReel) BadFrameCh() <-chan *protobufs.ClockFrame { | ||||
| 	return m.badFrameCh | ||||
| } | ||||
| 
 | ||||
| // Stop implements TimeReel.
 | ||||
| func (m *MasterTimeReel) Stop() { | ||||
| 	m.done <- true | ||||
| } | ||||
| 
 | ||||
| func (m *MasterTimeReel) createGenesisFrame() *protobufs.ClockFrame { | ||||
| 	seed, err := hex.DecodeString(m.engineConfig.GenesisSeed) | ||||
| 	if err != nil { | ||||
| 		panic(errors.New("genesis seed is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	frame, err := m.frameProver.CreateMasterGenesisFrame( | ||||
| 		m.filter, | ||||
| 		seed, | ||||
| 		m.engineConfig.Difficulty, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	txn, err := m.clockStore.NewTransaction() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err = m.clockStore.PutMasterClockFrame(frame, txn); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	if err = txn.Commit(); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame | ||||
| } | ||||
| 
 | ||||
| func (m *MasterTimeReel) runLoop() { | ||||
| 	for { | ||||
| 		select { | ||||
| 		case frame := <-m.frames: | ||||
| 			if m.head.FrameNumber < frame.FrameNumber { | ||||
| 				m.logger.Debug( | ||||
| 					"new frame has higher number", | ||||
| 					zap.Uint32("new_frame_number", uint32(frame.FrameNumber)), | ||||
| 					zap.Uint32("frame_number", uint32(m.head.FrameNumber)), | ||||
| 				) | ||||
| 				if frame.FrameNumber-m.head.FrameNumber == 1 { | ||||
| 					parent := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 					selector, err := m.head.GetSelector() | ||||
| 					if err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					// master frames cannot fork, this is invalid
 | ||||
| 					if parent.Cmp(selector) != 0 { | ||||
| 						m.logger.Debug( | ||||
| 							"invalid parent selector for frame", | ||||
| 							zap.Binary("frame_parent_selector", frame.ParentSelector), | ||||
| 							zap.Binary("actual_parent_selector", selector.FillBytes( | ||||
| 								make([]byte, 32), | ||||
| 							)), | ||||
| 						) | ||||
| 						go func() { | ||||
| 							m.badFrameCh <- frame | ||||
| 						}() | ||||
| 						continue | ||||
| 					} | ||||
| 
 | ||||
| 					txn, err := m.clockStore.NewTransaction() | ||||
| 					if err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					if err := m.clockStore.PutMasterClockFrame(frame, txn); err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					if err = txn.Commit(); err != nil { | ||||
| 						panic(err) | ||||
| 					} | ||||
| 
 | ||||
| 					m.head = frame | ||||
| 					go func() { | ||||
| 						m.newFrameCh <- frame | ||||
| 					}() | ||||
| 				} else { | ||||
| 					go func() { | ||||
| 						m.frames <- frame | ||||
| 					}() | ||||
| 				} | ||||
| 			} else { | ||||
| 				m.logger.Debug( | ||||
| 					"new frame has same or lower frame number", | ||||
| 					zap.Uint32("new_frame_number", uint32(frame.FrameNumber)), | ||||
| 					zap.Uint32("frame_number", uint32(m.head.FrameNumber)), | ||||
| 				) | ||||
| 				continue | ||||
| 			} | ||||
| 		case <-m.done: | ||||
| 			return | ||||
| 		} | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| var _ TimeReel = (*MasterTimeReel)(nil) | ||||
							
								
								
									
										82
									
								
								node/consensus/time/master_time_reel_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										82
									
								
								node/consensus/time/master_time_reel_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,82 @@ | ||||
| package time_test | ||||
| 
 | ||||
| import ( | ||||
| 	"strings" | ||||
| 	"sync" | ||||
| 	"testing" | ||||
| 
 | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| ) | ||||
| 
 | ||||
| func TestMasterTimeReel(t *testing.T) { | ||||
| 	logger, _ := zap.NewProduction() | ||||
| 	db := store.NewInMemKVDB() | ||||
| 	clockStore := store.NewPebbleClockStore(db, logger) | ||||
| 	prover := crypto.NewWesolowskiFrameProver(logger) | ||||
| 	filter := "ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff" | ||||
| 
 | ||||
| 	m := time.NewMasterTimeReel( | ||||
| 		logger, | ||||
| 		clockStore, | ||||
| 		&config.EngineConfig{ | ||||
| 			Filter:      filter, | ||||
| 			GenesisSeed: strings.Repeat("00", 516), | ||||
| 			Difficulty:  10, | ||||
| 		}, | ||||
| 		prover, | ||||
| 	) | ||||
| 
 | ||||
| 	err := m.Start() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frame, err := m.Head() | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frames := []*protobufs.ClockFrame{} | ||||
| 	wg := sync.WaitGroup{} | ||||
| 	wg.Add(1) | ||||
| 	frameCh := m.NewFrameCh() | ||||
| 	go func() { | ||||
| 		for i := 0; i < 200; i++ { | ||||
| 			frames = append(frames, <-frameCh) | ||||
| 		} | ||||
| 		wg.Done() | ||||
| 	}() | ||||
| 
 | ||||
| 	// in order
 | ||||
| 	for i := int64(0); i < 100; i++ { | ||||
| 		frame, err = prover.ProveMasterClockFrame(frame, i+1, 10) | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		err := m.Insert(frame) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	insertFrames := []*protobufs.ClockFrame{} | ||||
| 
 | ||||
| 	// reverse order
 | ||||
| 	for i := int64(100); i < 200; i++ { | ||||
| 		frame, err = prover.ProveMasterClockFrame(frame, i+1, 10) | ||||
| 		assert.NoError(t, err) | ||||
| 
 | ||||
| 		insertFrames = append(insertFrames, frame) | ||||
| 	} | ||||
| 
 | ||||
| 	for i := 99; i >= 0; i-- { | ||||
| 		err := m.Insert(insertFrames[i]) | ||||
| 		assert.NoError(t, err) | ||||
| 	} | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	for i := 0; i < 200; i++ { | ||||
| 		assert.NotNil(t, frames[i]) | ||||
| 		assert.Equal(t, frames[i].FrameNumber, uint64(i+1)) | ||||
| 	} | ||||
| } | ||||
							
								
								
									
										14
									
								
								node/consensus/time/time_reel.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										14
									
								
								node/consensus/time/time_reel.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,14 @@ | ||||
| package time | ||||
| 
 | ||||
| import ( | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| 
 | ||||
| type TimeReel interface { | ||||
| 	Start() error | ||||
| 	Stop() | ||||
| 	Insert(frame *protobufs.ClockFrame) error | ||||
| 	Head() (*protobufs.ClockFrame, error) | ||||
| 	NewFrameCh() <-chan *protobufs.ClockFrame | ||||
| 	BadFrameCh() <-chan *protobufs.ClockFrame | ||||
| } | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto | ||||
| package channel | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/aes" | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto_test | ||||
| package channel_test | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| @ -7,7 +7,7 @@ import ( | ||||
| 	"github.com/stretchr/testify/require" | ||||
| 
 | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/channel" | ||||
| ) | ||||
| 
 | ||||
| func TestRatchetEncrypt(t *testing.T) { | ||||
| @ -20,7 +20,7 @@ func TestRatchetEncrypt(t *testing.T) { | ||||
| 	x448ReceivingIdentityKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingIdentityPrivateKey) | ||||
| 	x448ReceivingSignedPreKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingSignedPrePrivateKey) | ||||
| 
 | ||||
| 	senderResult := crypto.SenderX3DH( | ||||
| 	senderResult := channel.SenderX3DH( | ||||
| 		x448SendingIdentityPrivateKey, | ||||
| 		x448SendingEphemeralPrivateKey, | ||||
| 		x448ReceivingIdentityKey, | ||||
| @ -28,7 +28,7 @@ func TestRatchetEncrypt(t *testing.T) { | ||||
| 		96, | ||||
| 	) | ||||
| 
 | ||||
| 	receiverResult := crypto.ReceiverX3DH( | ||||
| 	receiverResult := channel.ReceiverX3DH( | ||||
| 		x448ReceivingIdentityPrivateKey, | ||||
| 		x448ReceivingSignedPrePrivateKey, | ||||
| 		x448SendingIdentityKey, | ||||
| @ -36,7 +36,7 @@ func TestRatchetEncrypt(t *testing.T) { | ||||
| 		96, | ||||
| 	) | ||||
| 
 | ||||
| 	sender, err := crypto.NewDoubleRatchetParticipant( | ||||
| 	sender, err := channel.NewDoubleRatchetParticipant( | ||||
| 		senderResult[:32], | ||||
| 		senderResult[32:64], | ||||
| 		senderResult[64:], | ||||
| @ -48,7 +48,7 @@ func TestRatchetEncrypt(t *testing.T) { | ||||
| 	) | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	receiver, err := crypto.NewDoubleRatchetParticipant( | ||||
| 	receiver, err := channel.NewDoubleRatchetParticipant( | ||||
| 		receiverResult[:32], | ||||
| 		receiverResult[32:64], | ||||
| 		receiverResult[64:], | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto | ||||
| package channel | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/sha512" | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto_test | ||||
| package channel_test | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| @ -7,7 +7,7 @@ import ( | ||||
| 	"github.com/stretchr/testify/require" | ||||
| 
 | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/channel" | ||||
| ) | ||||
| 
 | ||||
| func TestX3DHMatches(t *testing.T) { | ||||
| @ -20,7 +20,7 @@ func TestX3DHMatches(t *testing.T) { | ||||
| 	x448ReceivingIdentityKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingIdentityPrivateKey) | ||||
| 	x448ReceivingSignedPreKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingSignedPrePrivateKey) | ||||
| 
 | ||||
| 	result := crypto.SenderX3DH( | ||||
| 	result := channel.SenderX3DH( | ||||
| 		x448SendingIdentityPrivateKey, | ||||
| 		x448SendingEphemeralPrivateKey, | ||||
| 		x448ReceivingIdentityKey, | ||||
| @ -28,7 +28,7 @@ func TestX3DHMatches(t *testing.T) { | ||||
| 		32, | ||||
| 	) | ||||
| 
 | ||||
| 	compare := crypto.ReceiverX3DH( | ||||
| 	compare := channel.ReceiverX3DH( | ||||
| 		x448ReceivingIdentityPrivateKey, | ||||
| 		x448ReceivingSignedPrePrivateKey, | ||||
| 		x448SendingIdentityKey, | ||||
							
								
								
									
										43
									
								
								node/crypto/frame_prover.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										43
									
								
								node/crypto/frame_prover.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,43 @@ | ||||
| package crypto | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto" | ||||
| 
 | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| ) | ||||
| 
 | ||||
| type FrameProver interface { | ||||
| 	ProveMasterClockFrame( | ||||
| 		previousFrame *protobufs.ClockFrame, | ||||
| 		timestamp int64, | ||||
| 		difficulty uint32, | ||||
| 	) (*protobufs.ClockFrame, error) | ||||
| 	ProveDataClockFrame( | ||||
| 		previousFrame *protobufs.ClockFrame, | ||||
| 		commitments [][]byte, | ||||
| 		aggregateProofs []*protobufs.InclusionAggregateProof, | ||||
| 		provingKey crypto.Signer, | ||||
| 		timestamp int64, | ||||
| 		difficulty uint32, | ||||
| 	) (*protobufs.ClockFrame, error) | ||||
| 	CreateMasterGenesisFrame( | ||||
| 		filter []byte, | ||||
| 		seed []byte, | ||||
| 		difficulty uint32, | ||||
| 	) (*protobufs.ClockFrame, error) | ||||
| 	CreateDataGenesisFrame( | ||||
| 		filter []byte, | ||||
| 		origin []byte, | ||||
| 		difficulty uint32, | ||||
| 		inclusionProof *InclusionAggregateProof, | ||||
| 		proverKeys [][]byte, | ||||
| 		preDusk bool, | ||||
| 	) (*protobufs.ClockFrame, *tries.RollingFrecencyCritbitTrie, error) | ||||
| 	VerifyMasterClockFrame( | ||||
| 		frame *protobufs.ClockFrame, | ||||
| 	) error | ||||
| 	VerifyDataClockFrame( | ||||
| 		frame *protobufs.ClockFrame, | ||||
| 	) error | ||||
| } | ||||
							
								
								
									
										28
									
								
								node/crypto/inclusion_prover.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										28
									
								
								node/crypto/inclusion_prover.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,28 @@ | ||||
| package crypto | ||||
| 
 | ||||
| import "source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 
 | ||||
| type InclusionCommitment struct { | ||||
| 	TypeUrl    string | ||||
| 	Data       []byte | ||||
| 	Commitment []byte | ||||
| } | ||||
| 
 | ||||
| type InclusionAggregateProof struct { | ||||
| 	InclusionCommitments []*InclusionCommitment | ||||
| 	AggregateCommitment  []byte | ||||
| 	Proof                []byte | ||||
| } | ||||
| 
 | ||||
| type InclusionProver interface { | ||||
| 	Commit( | ||||
| 		data []byte, | ||||
| 		typeUrl string, | ||||
| 	) (*InclusionCommitment, error) | ||||
| 	ProveAggregate(commits []*InclusionCommitment) ( | ||||
| 		*InclusionAggregateProof, | ||||
| 		error, | ||||
| 	) | ||||
| 	VerifyAggregate(proof *InclusionAggregateProof) (bool, error) | ||||
| 	VerifyFrame(frame *protobufs.ClockFrame) error | ||||
| } | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto | ||||
| package kzg | ||||
| 
 | ||||
| import ( | ||||
| 	"math/big" | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto | ||||
| package kzg | ||||
| 
 | ||||
| import ( | ||||
| 	"encoding/binary" | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto_test | ||||
| package kzg_test | ||||
| 
 | ||||
| import ( | ||||
| 	"encoding/hex" | ||||
| @ -13,16 +13,16 @@ import ( | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves/native/bls48581" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/kzg" | ||||
| ) | ||||
| 
 | ||||
| func TestMain(m *testing.M) { | ||||
| 	csBytes, err := os.ReadFile("../ceremony.json") | ||||
| 	csBytes, err := os.ReadFile("../../ceremony.json") | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	cs := &crypto.CeremonyState{} | ||||
| 	cs := &kzg.CeremonyState{} | ||||
| 	if err := json.Unmarshal(csBytes, cs); err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| @ -75,8 +75,8 @@ func TestMain(m *testing.M) { | ||||
| 
 | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	crypto.CeremonyBLS48581G1 = g1s | ||||
| 	crypto.CeremonyBLS48581G2 = g2s | ||||
| 	kzg.CeremonyBLS48581G1 = g1s | ||||
| 	kzg.CeremonyBLS48581G2 = g2s | ||||
| 
 | ||||
| 	// Post-ceremony, precompute everything and put it in the finalized ceremony
 | ||||
| 	// state
 | ||||
| @ -135,16 +135,16 @@ func TestMain(m *testing.M) { | ||||
| 	wg.Add(len(sizes)) | ||||
| 	for i := range root { | ||||
| 		i := i | ||||
| 		crypto.RootOfUnityBLS48581[uint64(sizes[i])] = root[i] | ||||
| 		crypto.RootsOfUnityBLS48581[uint64(sizes[i])] = roots[i] | ||||
| 		crypto.ReverseRootsOfUnityBLS48581[uint64(sizes[i])] = reverseRoots[i] | ||||
| 		kzg.RootOfUnityBLS48581[uint64(sizes[i])] = root[i] | ||||
| 		kzg.RootsOfUnityBLS48581[uint64(sizes[i])] = roots[i] | ||||
| 		kzg.ReverseRootsOfUnityBLS48581[uint64(sizes[i])] = reverseRoots[i] | ||||
| 
 | ||||
| 		go func() { | ||||
| 			// We precomputed 65536, others are cheap and will be fully precomputed
 | ||||
| 			// post-ceremony
 | ||||
| 			if sizes[i] < 65536 { | ||||
| 				fftG1, err := crypto.FFTG1( | ||||
| 					crypto.CeremonyBLS48581G1[:sizes[i]], | ||||
| 				fftG1, err := kzg.FFTG1( | ||||
| 					kzg.CeremonyBLS48581G1[:sizes[i]], | ||||
| 					*curves.BLS48581( | ||||
| 						curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 					), | ||||
| @ -165,7 +165,7 @@ func TestMain(m *testing.M) { | ||||
| 	wg.Wait() | ||||
| 
 | ||||
| 	for i := range root { | ||||
| 		crypto.FFTBLS48581[uint64(sizes[i])] = ffts[i] | ||||
| 		kzg.FFTBLS48581[uint64(sizes[i])] = ffts[i] | ||||
| 	} | ||||
| 	code := m.Run() | ||||
| 	os.Exit(code) | ||||
| @ -175,7 +175,7 @@ func TestKzgBytesToPoly(t *testing.T) { | ||||
| 	modulus := make([]byte, 73) | ||||
| 	bls48581.NewBIGints(bls48581.CURVE_Order, nil).ToBytes(modulus) | ||||
| 	q := new(big.Int).SetBytes(modulus) | ||||
| 	p := crypto.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 	p := kzg.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 
 | ||||
| 	poly, err := p.BytesToPolynomial([]byte( | ||||
| 		"Did you ever hear the tragedy of Darth Plagueis The Wise? I thought not." + | ||||
| @ -217,7 +217,7 @@ func TestPolynomialCommitment(t *testing.T) { | ||||
| 	modulus := make([]byte, 73) | ||||
| 	bls48581.NewBIGints(bls48581.CURVE_Order, nil).ToBytes(modulus) | ||||
| 	q := new(big.Int).SetBytes(modulus) | ||||
| 	p := crypto.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 	p := kzg.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 
 | ||||
| 	poly, err := p.BytesToPolynomial([]byte( | ||||
| 		"Did you ever hear the tragedy of Darth Plagueis The Wise? I thought not." + | ||||
| @ -235,7 +235,7 @@ func TestPolynomialCommitment(t *testing.T) { | ||||
| 		poly = append(poly, curves.BLS48581G1().NewScalar().(curves.PairingScalar)) | ||||
| 	} | ||||
| 	require.NoError(t, err) | ||||
| 	evalPoly, err := crypto.FFT( | ||||
| 	evalPoly, err := kzg.FFT( | ||||
| 		poly, | ||||
| 		*curves.BLS48581( | ||||
| 			curves.BLS48581G1().NewGeneratorPoint(), | ||||
| @ -247,12 +247,12 @@ func TestPolynomialCommitment(t *testing.T) { | ||||
| 
 | ||||
| 	require.NoError(t, err) | ||||
| 	commitByCoeffs, err := p.PointLinearCombination( | ||||
| 		crypto.CeremonyBLS48581G1[:16], | ||||
| 		kzg.CeremonyBLS48581G1[:16], | ||||
| 		poly, | ||||
| 	) | ||||
| 	require.NoError(t, err) | ||||
| 	commitByEval, err := p.PointLinearCombination( | ||||
| 		crypto.FFTBLS48581[16], | ||||
| 		kzg.FFTBLS48581[16], | ||||
| 		evalPoly, | ||||
| 	) | ||||
| 	require.NoError(t, err) | ||||
| @ -265,7 +265,7 @@ func TestKZGProof(t *testing.T) { | ||||
| 	modulus := make([]byte, 73) | ||||
| 	bls48581.NewBIGints(bls48581.CURVE_Order, nil).ToBytes(modulus) | ||||
| 	q := new(big.Int).SetBytes(modulus) | ||||
| 	p := crypto.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 	p := kzg.NewKZGProver(curves.BLS48581(curves.BLS48581G1().Point), sha3.New256, q) | ||||
| 
 | ||||
| 	poly, err := p.BytesToPolynomial([]byte( | ||||
| 		"Did you ever hear the tragedy of Darth Plagueis The Wise? I thought not." + | ||||
| @ -284,7 +284,7 @@ func TestKZGProof(t *testing.T) { | ||||
| 		poly = append(poly, curves.BLS48581G1().NewScalar().(curves.PairingScalar)) | ||||
| 	} | ||||
| 
 | ||||
| 	evalPoly, err := crypto.FFT( | ||||
| 	evalPoly, err := kzg.FFT( | ||||
| 		poly, | ||||
| 		*curves.BLS48581( | ||||
| 			curves.BLS48581G1().NewGeneratorPoint(), | ||||
| @ -297,7 +297,7 @@ func TestKZGProof(t *testing.T) { | ||||
| 	commit, err := p.Commit(poly) | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	z := crypto.RootsOfUnityBLS48581[16][2] | ||||
| 	z := kzg.RootsOfUnityBLS48581[16][2] | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	checky := evalPoly[len(poly)-1] | ||||
| @ -331,7 +331,7 @@ func TestKZGProof(t *testing.T) { | ||||
| 		diff -= 1 | ||||
| 	} | ||||
| 
 | ||||
| 	proof, err := p.PointLinearCombination(crypto.CeremonyBLS48581G1[:15], out) | ||||
| 	proof, err := p.PointLinearCombination(kzg.CeremonyBLS48581G1[:15], out) | ||||
| 	// proof, err := p.Prove(evalPoly, commit, z.(curves.PairingScalar))
 | ||||
| 	require.NoError(t, err) | ||||
| 	require.True(t, p.Verify(commit, z, checky, proof)) | ||||
							
								
								
									
										464
									
								
								node/crypto/kzg_inclusion_prover.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										464
									
								
								node/crypto/kzg_inclusion_prover.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,464 @@ | ||||
| package crypto | ||||
| 
 | ||||
| import ( | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/kzg" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| 
 | ||||
| type KZGInclusionProver struct { | ||||
| 	prover *kzg.KZGProver | ||||
| 	logger *zap.Logger | ||||
| } | ||||
| 
 | ||||
| func NewKZGInclusionProver(logger *zap.Logger) *KZGInclusionProver { | ||||
| 	return &KZGInclusionProver{ | ||||
| 		prover: kzg.DefaultKZGProver(), | ||||
| 		logger: logger, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Commit implements InclusionProver.
 | ||||
| func (k *KZGInclusionProver) Commit( | ||||
| 	data []byte, | ||||
| 	typeUrl string, | ||||
| ) (*InclusionCommitment, error) { | ||||
| 	if typeUrl == protobufs.IntrinsicExecutionOutputType { | ||||
| 		digest := sha3.NewShake256() | ||||
| 		_, err := digest.Write(data) | ||||
| 		if err != nil { | ||||
| 			k.logger.Error( | ||||
| 				"error converting key bundle to polynomial", | ||||
| 				zap.Error(err), | ||||
| 			) | ||||
| 			return nil, errors.Wrap(err, "prove aggregate") | ||||
| 		} | ||||
| 
 | ||||
| 		expand := make([]byte, 1024) | ||||
| 		_, err = digest.Read(expand) | ||||
| 		if err != nil { | ||||
| 			k.logger.Error( | ||||
| 				"error converting key bundle to polynomial", | ||||
| 				zap.Error(err), | ||||
| 			) | ||||
| 			return nil, errors.Wrap(err, "prove aggregate") | ||||
| 		} | ||||
| 
 | ||||
| 		poly, err := k.prover.BytesToPolynomial(expand) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.Wrap(err, "commit") | ||||
| 		} | ||||
| 
 | ||||
| 		k.logger.Debug("proving execution output for inclusion") | ||||
| 		polys, err := kzg.FFT( | ||||
| 			poly, | ||||
| 			*curves.BLS48581( | ||||
| 				curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 			), | ||||
| 			16, | ||||
| 			false, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.Wrap(err, "prove") | ||||
| 		} | ||||
| 
 | ||||
| 		k.logger.Debug("converted execution output chunk to evaluation form") | ||||
| 
 | ||||
| 		k.logger.Debug("creating kzg commitment") | ||||
| 		points, err := k.prover.Commit(polys) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.Wrap(err, "prove") | ||||
| 		} | ||||
| 
 | ||||
| 		return &InclusionCommitment{ | ||||
| 			TypeUrl:    typeUrl, | ||||
| 			Data:       data, | ||||
| 			Commitment: points.ToAffineCompressed(), | ||||
| 		}, nil | ||||
| 	} | ||||
| 
 | ||||
| 	poly, err := k.prover.BytesToPolynomial(data) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "commit") | ||||
| 	} | ||||
| 
 | ||||
| 	points, err := k.prover.Commit(poly) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "commit") | ||||
| 	} | ||||
| 
 | ||||
| 	return &InclusionCommitment{ | ||||
| 		TypeUrl:    typeUrl, | ||||
| 		Data:       data, | ||||
| 		Commitment: points.ToAffineCompressed(), | ||||
| 	}, nil | ||||
| } | ||||
| 
 | ||||
| // ProveAggregate implements InclusionProver.
 | ||||
| func (k *KZGInclusionProver) ProveAggregate( | ||||
| 	commits []*InclusionCommitment, | ||||
| ) (*InclusionAggregateProof, error) { | ||||
| 	polys := [][]curves.PairingScalar{} | ||||
| 	commitPoints := []curves.PairingPoint{} | ||||
| 	for _, commit := range commits { | ||||
| 		switch commit.TypeUrl { | ||||
| 		case protobufs.IntrinsicExecutionOutputType: | ||||
| 			k.logger.Debug("confirming inclusion in aggregate") | ||||
| 			digest := sha3.NewShake256() | ||||
| 			_, err := digest.Write(commit.Data) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error converting key bundle to polynomial", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 
 | ||||
| 			expand := make([]byte, 1024) | ||||
| 			_, err = digest.Read(expand) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error converting key bundle to polynomial", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 
 | ||||
| 			poly, err := k.prover.BytesToPolynomial(expand) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error converting key bundle to polynomial", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 
 | ||||
| 			evalPoly, err := kzg.FFT( | ||||
| 				poly, | ||||
| 				*curves.BLS48581( | ||||
| 					curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 				), | ||||
| 				16, | ||||
| 				false, | ||||
| 			) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error performing fast fourier transform on key bundle", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 			k.logger.Debug( | ||||
| 				"created fft of polynomial", | ||||
| 				zap.Int("poly_size", len(evalPoly)), | ||||
| 			) | ||||
| 
 | ||||
| 			polys = append(polys, evalPoly) | ||||
| 
 | ||||
| 			c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 				commit.Commitment, | ||||
| 			) | ||||
| 			if err != nil { | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 			commitPoints = append(commitPoints, c.(curves.PairingPoint)) | ||||
| 		default: | ||||
| 			k.logger.Debug("confirming inclusion in aggregate") | ||||
| 			poly, err := k.prover.BytesToPolynomial(commit.Data) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error converting key bundle to polynomial", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 
 | ||||
| 			for i := 0; i < 128-len(poly); i++ { | ||||
| 				poly = append( | ||||
| 					poly, | ||||
| 					curves.BLS48581G1().Scalar.Zero().(curves.PairingScalar), | ||||
| 				) | ||||
| 			} | ||||
| 
 | ||||
| 			evalPoly, err := kzg.FFT( | ||||
| 				poly, | ||||
| 				*curves.BLS48581( | ||||
| 					curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 				), | ||||
| 				128, | ||||
| 				false, | ||||
| 			) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error( | ||||
| 					"error performing fast fourier transform on key bundle", | ||||
| 					zap.Error(err), | ||||
| 				) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 			k.logger.Debug( | ||||
| 				"created fft of polynomial", | ||||
| 				zap.Int("poly_size", len(evalPoly)), | ||||
| 			) | ||||
| 
 | ||||
| 			polys = append(polys, evalPoly) | ||||
| 
 | ||||
| 			c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 				commit.Commitment, | ||||
| 			) | ||||
| 			if err != nil { | ||||
| 				k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 				return nil, errors.Wrap(err, "prove aggregate") | ||||
| 			} | ||||
| 			commitPoints = append(commitPoints, c.(curves.PairingPoint)) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	proof, commitment, err := k.prover.ProveAggregate( | ||||
| 		polys, | ||||
| 		commitPoints, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove aggregate") | ||||
| 	} | ||||
| 
 | ||||
| 	if proof.IsIdentity() { | ||||
| 		return nil, errors.Wrap(errors.New("invalid proof"), "prove aggregate") | ||||
| 	} | ||||
| 
 | ||||
| 	return &InclusionAggregateProof{ | ||||
| 		InclusionCommitments: commits, | ||||
| 		AggregateCommitment:  commitment.ToAffineCompressed(), | ||||
| 		Proof:                proof.ToAffineCompressed(), | ||||
| 	}, nil | ||||
| } | ||||
| 
 | ||||
| // VerifyAggregate implements InclusionProver.
 | ||||
| func (k *KZGInclusionProver) VerifyAggregate( | ||||
| 	proof *InclusionAggregateProof, | ||||
| ) (bool, error) { | ||||
| 	polys := [][]curves.PairingScalar{} | ||||
| 	commitPoints := []curves.PairingPoint{} | ||||
| 	for _, commit := range proof.InclusionCommitments { | ||||
| 		poly, err := k.prover.BytesToPolynomial(commit.Data) | ||||
| 		if err != nil { | ||||
| 			return false, errors.Wrap(err, "verify aggregate") | ||||
| 		} | ||||
| 
 | ||||
| 		polys = append(polys, poly) | ||||
| 
 | ||||
| 		point, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 			commit.Commitment, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			return false, errors.Wrap(err, "verify aggregate") | ||||
| 		} | ||||
| 
 | ||||
| 		commitPoints = append(commitPoints, point.(curves.PairingPoint)) | ||||
| 	} | ||||
| 
 | ||||
| 	aggregate, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 		proof.AggregateCommitment, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return false, errors.Wrap(err, "verify aggregate") | ||||
| 	} | ||||
| 
 | ||||
| 	proofPoint, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 		proof.Proof, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return false, errors.Wrap(err, "verify aggregate") | ||||
| 	} | ||||
| 
 | ||||
| 	verify, err := k.prover.VerifyAggregateProof( | ||||
| 		polys, | ||||
| 		commitPoints, | ||||
| 		aggregate.(curves.PairingPoint), | ||||
| 		proofPoint.(curves.PairingPoint), | ||||
| 	) | ||||
| 	return verify, errors.Wrap(err, "verify aggregate") | ||||
| } | ||||
| 
 | ||||
| func (k *KZGInclusionProver) VerifyFrame( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	aggregateCommitments := []curves.PairingPoint{} | ||||
| 	for i := 0; i < (len(frame.Input)-516)/74; i++ { | ||||
| 		c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 			frame.Input[516+(i*74) : 516+(i*74)+74], | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "verify frame") | ||||
| 		} | ||||
| 		aggregateCommitments = append(aggregateCommitments, c.(curves.PairingPoint)) | ||||
| 	} | ||||
| 
 | ||||
| 	if len(aggregateCommitments) != len(frame.AggregateProofs) { | ||||
| 		k.logger.Error( | ||||
| 			"commit length mismatched proof for frame", | ||||
| 			zap.Int("commit_length", len(aggregateCommitments)), | ||||
| 			zap.Int("proof_length", len(frame.AggregateProofs)), | ||||
| 		) | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("commit length mismatched proof for frame"), | ||||
| 			"verify frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	for i, proof := range frame.AggregateProofs { | ||||
| 		aggregatePoly := [][]curves.PairingScalar{} | ||||
| 		commitments := []curves.PairingPoint{} | ||||
| 
 | ||||
| 		for _, commit := range proof.GetInclusionCommitments() { | ||||
| 			switch commit.TypeUrl { | ||||
| 			case protobufs.IntrinsicExecutionOutputType: | ||||
| 				k.logger.Debug("confirming inclusion in aggregate") | ||||
| 				digest := sha3.NewShake256() | ||||
| 				_, err := digest.Write(commit.Data) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 
 | ||||
| 				expand := make([]byte, 1024) | ||||
| 				_, err = digest.Read(expand) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 
 | ||||
| 				poly, err := k.prover.BytesToPolynomial(expand) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 
 | ||||
| 				evalPoly, err := kzg.FFT( | ||||
| 					poly, | ||||
| 					*curves.BLS48581( | ||||
| 						curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 					), | ||||
| 					16, | ||||
| 					false, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error performing fast fourier transform on key bundle", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 				k.logger.Debug( | ||||
| 					"created fft of polynomial", | ||||
| 					zap.Int("poly_size", len(evalPoly)), | ||||
| 				) | ||||
| 
 | ||||
| 				aggregatePoly = append(aggregatePoly, evalPoly) | ||||
| 
 | ||||
| 				c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 					commit.Commitment, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 				commitments = append(commitments, c.(curves.PairingPoint)) | ||||
| 			default: | ||||
| 				k.logger.Debug("confirming inclusion in aggregate") | ||||
| 				poly, err := k.prover.BytesToPolynomial(commit.Data) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error converting key bundle to polynomial", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 
 | ||||
| 				for i := 0; i < 128-len(poly); i++ { | ||||
| 					poly = append( | ||||
| 						poly, | ||||
| 						curves.BLS48581G1().Scalar.Zero().(curves.PairingScalar), | ||||
| 					) | ||||
| 				} | ||||
| 
 | ||||
| 				evalPoly, err := kzg.FFT( | ||||
| 					poly, | ||||
| 					*curves.BLS48581( | ||||
| 						curves.BLS48581G1().NewGeneratorPoint(), | ||||
| 					), | ||||
| 					128, | ||||
| 					false, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error( | ||||
| 						"error performing fast fourier transform on key bundle", | ||||
| 						zap.Error(err), | ||||
| 					) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 				k.logger.Debug( | ||||
| 					"created fft of polynomial", | ||||
| 					zap.Int("poly_size", len(evalPoly)), | ||||
| 				) | ||||
| 
 | ||||
| 				aggregatePoly = append(aggregatePoly, evalPoly) | ||||
| 
 | ||||
| 				c, err := curves.BLS48581G1().NewGeneratorPoint().FromAffineCompressed( | ||||
| 					commit.Commitment, | ||||
| 				) | ||||
| 				if err != nil { | ||||
| 					k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 					return errors.Wrap(err, "verify frame") | ||||
| 				} | ||||
| 				commitments = append(commitments, c.(curves.PairingPoint)) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		p, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 			proof.Proof, | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap(err, "verify frame") | ||||
| 		} | ||||
| 
 | ||||
| 		result, err := k.prover.VerifyAggregateProof( | ||||
| 			aggregatePoly, | ||||
| 			commitments, | ||||
| 			aggregateCommitments[i], | ||||
| 			p.(curves.PairingPoint), | ||||
| 		) | ||||
| 		if err != nil { | ||||
| 			k.logger.Error( | ||||
| 				"could not verify clock frame", | ||||
| 				zap.Error(err), | ||||
| 			) | ||||
| 			return errors.Wrap(err, "verify frame") | ||||
| 		} | ||||
| 
 | ||||
| 		if !result { | ||||
| 			k.logger.Error("could not verify clock frame", zap.Error(err)) | ||||
| 			return errors.Wrap( | ||||
| 				errors.New("invalid proof"), | ||||
| 				"verify frame", | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| var _ InclusionProver = (*KZGInclusionProver)(nil) | ||||
							
								
								
									
										72
									
								
								node/crypto/kzg_inclusion_prover_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										72
									
								
								node/crypto/kzg_inclusion_prover_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,72 @@ | ||||
| package crypto_test | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"crypto/rand" | ||||
| 	"testing" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/cloudflare/circl/sign/ed448" | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/kzg" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| 
 | ||||
| func TestKZGVerifyFrame(t *testing.T) { | ||||
| 	kzg.TestInit("../ceremony.json") | ||||
| 	data := make([]byte, 1024) | ||||
| 	rand.Read(data) | ||||
| 
 | ||||
| 	l, _ := zap.NewProduction() | ||||
| 	inclusionProver := crypto.NewKZGInclusionProver(l) | ||||
| 
 | ||||
| 	commitment, err := inclusionProver.Commit( | ||||
| 		data, | ||||
| 		protobufs.IntrinsicExecutionOutputType, | ||||
| 	) | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	proof, err := inclusionProver.ProveAggregate( | ||||
| 		[]*crypto.InclusionCommitment{commitment}, | ||||
| 	) | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:      []byte{0x00}, | ||||
| 		FrameNumber: 1, | ||||
| 		Input:       bytes.Repeat([]byte{0x00}, 516), | ||||
| 		Output:      bytes.Repeat([]byte{0x00}, 516), | ||||
| 	} | ||||
| 
 | ||||
| 	_, priv, _ := ed448.GenerateKey(rand.Reader) | ||||
| 	w := crypto.NewWesolowskiFrameProver(l) | ||||
| 	frame, err = w.ProveDataClockFrame( | ||||
| 		frame, | ||||
| 		[][]byte{proof.AggregateCommitment}, | ||||
| 		[]*protobufs.InclusionAggregateProof{ | ||||
| 			{ | ||||
| 				Filter:      []byte{0x00}, | ||||
| 				FrameNumber: 1, | ||||
| 				InclusionCommitments: []*protobufs.InclusionCommitment{ | ||||
| 					{ | ||||
| 						Filter:      []byte{0x00}, | ||||
| 						FrameNumber: 1, | ||||
| 						TypeUrl:     proof.InclusionCommitments[0].TypeUrl, | ||||
| 						Commitment:  proof.InclusionCommitments[0].Commitment, | ||||
| 						Data:        data, | ||||
| 						Position:    0, | ||||
| 					}, | ||||
| 				}, | ||||
| 				Proof: proof.Proof, | ||||
| 			}, | ||||
| 		}, | ||||
| 		priv, | ||||
| 		time.Now().UnixMilli(), | ||||
| 		100, | ||||
| 	) | ||||
| 
 | ||||
| 	err = inclusionProver.VerifyFrame(frame) | ||||
| 	assert.NoError(t, err) | ||||
| } | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto | ||||
| package shuffle | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto/rand" | ||||
| @ -7,10 +7,16 @@ import ( | ||||
| 	"filippo.io/edwards25519" | ||||
| ) | ||||
| 
 | ||||
| var lBE = []byte{16, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 222, 249, 222, 162, 247, 156, 214, 88, 18, 99, 26, 92, 245, 211, 236} | ||||
| var lBE = []byte{ | ||||
| 	16, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 20, 222, 249, 222, 162, 247, | ||||
| 	156, 214, 88, 18, 99, 26, 92, 245, 211, 236, | ||||
| } | ||||
| var lBigInt = big.NewInt(0).SetBytes(lBE) | ||||
| 
 | ||||
| func genPolyFrags(secret *edwards25519.Scalar, total, threshold int) []*edwards25519.Scalar { | ||||
| func genPolyFrags( | ||||
| 	secret *edwards25519.Scalar, | ||||
| 	total, threshold int, | ||||
| ) []*edwards25519.Scalar { | ||||
| 	coeffs := []*edwards25519.Scalar{} | ||||
| 	coeffs = append(coeffs, secret) | ||||
| 
 | ||||
| @ -48,7 +54,10 @@ func genPolyFrags(secret *edwards25519.Scalar, total, threshold int) []*edwards2 | ||||
| 	return frags | ||||
| } | ||||
| 
 | ||||
| func ShamirSplitMatrix(matrix [][]*edwards25519.Scalar, total, threshold int) [][][]*edwards25519.Scalar { | ||||
| func ShamirSplitMatrix( | ||||
| 	matrix [][]*edwards25519.Scalar, | ||||
| 	total, threshold int, | ||||
| ) [][][]*edwards25519.Scalar { | ||||
| 	shamirMatrix := make([][][]*edwards25519.Scalar, len(matrix)) | ||||
| 
 | ||||
| 	for x := 0; x < len(matrix); x++ { | ||||
| @ -79,7 +88,9 @@ func AddMatrices(matrices ...[][]*edwards25519.Scalar) [][]*edwards25519.Scalar | ||||
| 	return result | ||||
| } | ||||
| 
 | ||||
| func GenerateRandomVectorShares(length, total, threshold int) [][]*edwards25519.Scalar { | ||||
| func GenerateRandomVectorShares( | ||||
| 	length, total, threshold int, | ||||
| ) [][]*edwards25519.Scalar { | ||||
| 	result := make([][]*edwards25519.Scalar, length) | ||||
| 
 | ||||
| 	for i := 0; i < length; i++ { | ||||
| @ -93,7 +104,10 @@ func GenerateRandomVectorShares(length, total, threshold int) [][]*edwards25519. | ||||
| 	return result | ||||
| } | ||||
| 
 | ||||
| func InterpolatePolynomialShares(shares []*edwards25519.Scalar, ids []int) *edwards25519.Scalar { | ||||
| func InterpolatePolynomialShares( | ||||
| 	shares []*edwards25519.Scalar, | ||||
| 	ids []int, | ||||
| ) *edwards25519.Scalar { | ||||
| 	var reconstructedSum *edwards25519.Scalar | ||||
| 
 | ||||
| 	for j := 0; j < len(ids); j++ { | ||||
| @ -116,7 +130,10 @@ func InterpolatePolynomialShares(shares []*edwards25519.Scalar, ids []int) *edwa | ||||
| 
 | ||||
| 		coeffDenom.Invert(coeffDenom) | ||||
| 		coeffNum.Multiply(coeffNum, coeffDenom) | ||||
| 		reconstructedFrag := edwards25519.NewScalar().Multiply(coeffNum, shares[ids[j]-1]) | ||||
| 		reconstructedFrag := edwards25519.NewScalar().Multiply( | ||||
| 			coeffNum, | ||||
| 			shares[ids[j]-1], | ||||
| 		) | ||||
| 
 | ||||
| 		if reconstructedSum == nil { | ||||
| 			reconstructedSum = reconstructedFrag | ||||
| @ -128,7 +145,9 @@ func InterpolatePolynomialShares(shares []*edwards25519.Scalar, ids []int) *edwa | ||||
| 	return reconstructedSum | ||||
| } | ||||
| 
 | ||||
| func LUDecompose(matrix [][]*edwards25519.Scalar) ([][]*edwards25519.Scalar, [][]*edwards25519.Scalar) { | ||||
| func LUDecompose( | ||||
| 	matrix [][]*edwards25519.Scalar, | ||||
| ) ([][]*edwards25519.Scalar, [][]*edwards25519.Scalar) { | ||||
| 	imax := 0 | ||||
| 	maxA := edwards25519.NewScalar() | ||||
| 	N := len(matrix) | ||||
| @ -141,18 +160,24 @@ func LUDecompose(matrix [][]*edwards25519.Scalar) ([][]*edwards25519.Scalar, [][ | ||||
| 		pm[i] = make([]*edwards25519.Scalar, N) | ||||
| 		p[i] = i | ||||
| 		for j := 0; j < N; j++ { | ||||
| 			newA[i][j], _ = edwards25519.NewScalar().SetCanonicalBytes(matrix[i][j].Bytes()) | ||||
| 			newA[i][j], _ = edwards25519.NewScalar().SetCanonicalBytes( | ||||
| 				matrix[i][j].Bytes(), | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	scalarOne, _ := edwards25519.NewScalar().SetCanonicalBytes(BigIntToLEBytes(big.NewInt(int64(1)))) | ||||
| 	scalarOne, _ := edwards25519.NewScalar().SetCanonicalBytes( | ||||
| 		BigIntToLEBytes(big.NewInt(int64(1))), | ||||
| 	) | ||||
| 
 | ||||
| 	for i := 0; i < N; i++ { | ||||
| 		maxA = edwards25519.NewScalar() | ||||
| 		imax = i | ||||
| 
 | ||||
| 		for k := i; k < N; k++ { | ||||
| 			if LEBytesToBigInt(newA[k][i].Bytes()).Cmp(LEBytesToBigInt(maxA.Bytes())) > 0 { | ||||
| 			if LEBytesToBigInt(newA[k][i].Bytes()).Cmp( | ||||
| 				LEBytesToBigInt(maxA.Bytes()), | ||||
| 			) > 0 { | ||||
| 				maxA = newA[k][i] | ||||
| 				imax = k | ||||
| 			} | ||||
| @ -171,10 +196,16 @@ func LUDecompose(matrix [][]*edwards25519.Scalar) ([][]*edwards25519.Scalar, [][ | ||||
| 		} | ||||
| 
 | ||||
| 		for j := i + 1; j < N; j++ { | ||||
| 			newA[j][i].Multiply(newA[j][i], edwards25519.NewScalar().Invert(newA[i][i])) | ||||
| 			newA[j][i].Multiply( | ||||
| 				newA[j][i], | ||||
| 				edwards25519.NewScalar().Invert(newA[i][i]), | ||||
| 			) | ||||
| 
 | ||||
| 			for k := i + 1; k < N; k++ { | ||||
| 				newA[j][k].Subtract(newA[j][k], edwards25519.NewScalar().Multiply(newA[j][i], newA[i][k])) | ||||
| 				newA[j][k].Subtract(newA[j][k], edwards25519.NewScalar().Multiply( | ||||
| 					newA[j][i], | ||||
| 					newA[i][k], | ||||
| 				)) | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| @ -205,13 +236,19 @@ func Invert(matrix [][]*edwards25519.Scalar) [][]*edwards25519.Scalar { | ||||
| 			ia[i][j] = edwards25519.NewScalar().Set(p[i][j]) | ||||
| 
 | ||||
| 			for k := 0; k < i; k++ { | ||||
| 				ia[i][j].Subtract(ia[i][j], edwards25519.NewScalar().Multiply(a[i][k], ia[k][j])) | ||||
| 				ia[i][j].Subtract(ia[i][j], edwards25519.NewScalar().Multiply( | ||||
| 					a[i][k], | ||||
| 					ia[k][j], | ||||
| 				)) | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		for i := len(matrix) - 1; i >= 0; i-- { | ||||
| 			for k := i + 1; k < len(matrix); k++ { | ||||
| 				ia[i][j].Subtract(ia[i][j], edwards25519.NewScalar().Multiply(a[i][k], ia[k][j])) | ||||
| 				ia[i][j].Subtract(ia[i][j], edwards25519.NewScalar().Multiply( | ||||
| 					a[i][k], | ||||
| 					ia[k][j], | ||||
| 				)) | ||||
| 			} | ||||
| 
 | ||||
| 			ia[i][j].Multiply(ia[i][j], edwards25519.NewScalar().Invert(a[i][i])) | ||||
| @ -221,7 +258,10 @@ func Invert(matrix [][]*edwards25519.Scalar) [][]*edwards25519.Scalar { | ||||
| 	return ia | ||||
| } | ||||
| 
 | ||||
| func InterpolateMatrixShares(matrixShares [][][]*edwards25519.Scalar, ids []int) [][]*edwards25519.Scalar { | ||||
| func InterpolateMatrixShares( | ||||
| 	matrixShares [][][]*edwards25519.Scalar, | ||||
| 	ids []int, | ||||
| ) [][]*edwards25519.Scalar { | ||||
| 	matrix := make([][]*edwards25519.Scalar, len(matrixShares)) | ||||
| 
 | ||||
| 	for x := 0; x < len(matrix); x++ { | ||||
| @ -241,10 +281,17 @@ func ScalarMult(a int, b [][]*edwards25519.Scalar) [][]*edwards25519.Scalar { | ||||
| 
 | ||||
| 		for y := 0; y < len(b[0]); y++ { | ||||
| 			if a >= 0 { | ||||
| 				prod[x][y], _ = edwards25519.NewScalar().SetCanonicalBytes(BigIntToLEBytes(big.NewInt(int64(a)))) | ||||
| 				prod[x][y], _ = edwards25519.NewScalar().SetCanonicalBytes( | ||||
| 					BigIntToLEBytes(big.NewInt(int64(a))), | ||||
| 				) | ||||
| 			} else { | ||||
| 				negA, _ := edwards25519.NewScalar().SetCanonicalBytes(BigIntToLEBytes(big.NewInt(int64(-a)))) | ||||
| 				prod[x][y] = edwards25519.NewScalar().Subtract(edwards25519.NewScalar(), negA) | ||||
| 				negA, _ := edwards25519.NewScalar().SetCanonicalBytes( | ||||
| 					BigIntToLEBytes(big.NewInt(int64(-a))), | ||||
| 				) | ||||
| 				prod[x][y] = edwards25519.NewScalar().Subtract( | ||||
| 					edwards25519.NewScalar(), | ||||
| 					negA, | ||||
| 				) | ||||
| 			} | ||||
| 
 | ||||
| 			prod[x][y] = prod[x][y].Multiply(prod[x][y], b[x][y]) | ||||
| @ -254,7 +301,9 @@ func ScalarMult(a int, b [][]*edwards25519.Scalar) [][]*edwards25519.Scalar { | ||||
| 	return prod | ||||
| } | ||||
| 
 | ||||
| func GenerateDotProduct(a, b [][]*edwards25519.Scalar) [][]*edwards25519.Scalar { | ||||
| func GenerateDotProduct( | ||||
| 	a, b [][]*edwards25519.Scalar, | ||||
| ) [][]*edwards25519.Scalar { | ||||
| 	if len(a[0]) != len(b) { | ||||
| 		panic("cannot generate dot product of a and b - mismatched length") | ||||
| 	} | ||||
| @ -276,7 +325,9 @@ func GenerateDotProduct(a, b [][]*edwards25519.Scalar) [][]*edwards25519.Scalar | ||||
| 	return abMatrix | ||||
| } | ||||
| 
 | ||||
| func GenerateRandomMatrixAndInverseShares(size, total, threshold int) [2][][][]*edwards25519.Scalar { | ||||
| func GenerateRandomMatrixAndInverseShares( | ||||
| 	size, total, threshold int, | ||||
| ) [2][][][]*edwards25519.Scalar { | ||||
| 	output := make([][]*edwards25519.Scalar, size) | ||||
| 	for x := 0; x < size; x++ { | ||||
| 		output[x] = make([]*edwards25519.Scalar, size) | ||||
| @ -294,7 +345,9 @@ func GenerateRandomMatrixAndInverseShares(size, total, threshold int) [2][][][]* | ||||
| 	return [2][][][]*edwards25519.Scalar{splitOutput, splitInverse} | ||||
| } | ||||
| 
 | ||||
| func GenerateRandomBeaverTripleMatrixShares(size, total, threshold int) [3][][][]*edwards25519.Scalar { | ||||
| func GenerateRandomBeaverTripleMatrixShares( | ||||
| 	size, total, threshold int, | ||||
| ) [3][][][]*edwards25519.Scalar { | ||||
| 	uMatrix := make([][]*edwards25519.Scalar, size) | ||||
| 	vMatrix := make([][]*edwards25519.Scalar, size) | ||||
| 
 | ||||
| @ -321,7 +374,9 @@ func GenerateRandomBeaverTripleMatrixShares(size, total, threshold int) [3][][][ | ||||
| 	vMatrixShares := ShamirSplitMatrix(vMatrix, total, threshold) | ||||
| 	uvMatrixShares := ShamirSplitMatrix(uvMatrix, total, threshold) | ||||
| 
 | ||||
| 	return [3][][][]*edwards25519.Scalar{uMatrixShares, vMatrixShares, uvMatrixShares} | ||||
| 	return [3][][][]*edwards25519.Scalar{ | ||||
| 		uMatrixShares, vMatrixShares, uvMatrixShares, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func GeneratePermutationMatrix(size int) [][]*edwards25519.Scalar { | ||||
| @ -1,4 +1,4 @@ | ||||
| package crypto_test | ||||
| package shuffle_test | ||||
| 
 | ||||
| import ( | ||||
| 	"fmt" | ||||
| @ -7,11 +7,11 @@ import ( | ||||
| 
 | ||||
| 	"filippo.io/edwards25519" | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/shuffle" | ||||
| ) | ||||
| 
 | ||||
| func TestGeneratePermutationMatrix(t *testing.T) { | ||||
| 	m := crypto.GeneratePermutationMatrix(6) | ||||
| 	m := shuffle.GeneratePermutationMatrix(6) | ||||
| 	for _, x := range m { | ||||
| 		ySum := byte(0x00) | ||||
| 		for _, y := range x { | ||||
| @ -39,14 +39,14 @@ func verifyLagrange(t *testing.T, shares []*edwards25519.Scalar, expected *edwar | ||||
| 		var reconstructedSum *edwards25519.Scalar | ||||
| 
 | ||||
| 		for j := 0; j < threshold; j++ { | ||||
| 			oneLENumBytes := crypto.BigIntToLEBytes(big.NewInt(1)) | ||||
| 			oneLENumBytes := shuffle.BigIntToLEBytes(big.NewInt(1)) | ||||
| 			coeffNum, _ := edwards25519.NewScalar().SetCanonicalBytes(oneLENumBytes) | ||||
| 			coeffDenom, _ := edwards25519.NewScalar().SetCanonicalBytes(oneLENumBytes) | ||||
| 
 | ||||
| 			for k := 0; k < threshold; k++ { | ||||
| 				if j != k { | ||||
| 					ikBytes := crypto.BigIntToLEBytes(big.NewInt(int64(i + k))) | ||||
| 					ijBytes := crypto.BigIntToLEBytes(big.NewInt(int64(i + j))) | ||||
| 					ikBytes := shuffle.BigIntToLEBytes(big.NewInt(int64(i + k))) | ||||
| 					ijBytes := shuffle.BigIntToLEBytes(big.NewInt(int64(i + j))) | ||||
| 					ikScalar, _ := edwards25519.NewScalar().SetCanonicalBytes(ikBytes) | ||||
| 					ijScalar, _ := edwards25519.NewScalar().SetCanonicalBytes(ijBytes) | ||||
| 
 | ||||
| @ -78,8 +78,8 @@ func verifyLagrange(t *testing.T, shares []*edwards25519.Scalar, expected *edwar | ||||
| } | ||||
| 
 | ||||
| func TestGenerateShamirMatrix(t *testing.T) { | ||||
| 	m := crypto.GeneratePermutationMatrix(6) | ||||
| 	sm := crypto.ShamirSplitMatrix(m, 10, 3) | ||||
| 	m := shuffle.GeneratePermutationMatrix(6) | ||||
| 	sm := shuffle.ShamirSplitMatrix(m, 10, 3) | ||||
| 	for xi, x := range sm { | ||||
| 		for yi, y := range x { | ||||
| 			verifyLagrange(t, y, m[xi][yi], 10, 3) | ||||
| @ -88,11 +88,11 @@ func TestGenerateShamirMatrix(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| func TestMatrixDotProduct(t *testing.T) { | ||||
| 	zeroBytes := crypto.BigIntToLEBytes(big.NewInt(0)) | ||||
| 	oneBytes := crypto.BigIntToLEBytes(big.NewInt(1)) | ||||
| 	twoBytes := crypto.BigIntToLEBytes(big.NewInt(2)) | ||||
| 	threeBytes := crypto.BigIntToLEBytes(big.NewInt(3)) | ||||
| 	fourBytes := crypto.BigIntToLEBytes(big.NewInt(4)) | ||||
| 	zeroBytes := shuffle.BigIntToLEBytes(big.NewInt(0)) | ||||
| 	oneBytes := shuffle.BigIntToLEBytes(big.NewInt(1)) | ||||
| 	twoBytes := shuffle.BigIntToLEBytes(big.NewInt(2)) | ||||
| 	threeBytes := shuffle.BigIntToLEBytes(big.NewInt(3)) | ||||
| 	fourBytes := shuffle.BigIntToLEBytes(big.NewInt(4)) | ||||
| 
 | ||||
| 	zero, _ := edwards25519.NewScalar().SetCanonicalBytes(zeroBytes) | ||||
| 	one, _ := edwards25519.NewScalar().SetCanonicalBytes(oneBytes) | ||||
| @ -110,7 +110,7 @@ func TestMatrixDotProduct(t *testing.T) { | ||||
| 		{three, two, four}, | ||||
| 	} | ||||
| 
 | ||||
| 	abMatrix := crypto.GenerateDotProduct(aMatrix, bMatrix) | ||||
| 	abMatrix := shuffle.GenerateDotProduct(aMatrix, bMatrix) | ||||
| 	assert.Equal(t, byte(0x0a), abMatrix[0][0].Bytes()[0]) | ||||
| 	assert.Equal(t, byte(0x06), abMatrix[0][1].Bytes()[0]) | ||||
| 	assert.Equal(t, byte(0x0c), abMatrix[0][2].Bytes()[0]) | ||||
| @ -123,15 +123,15 @@ func TestMatrixDotProduct(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| func TestGenerateRandomBeaverTripleMatrixShares(t *testing.T) { | ||||
| 	beaverTripleShares := crypto.GenerateRandomBeaverTripleMatrixShares(6, 10, 3) | ||||
| 	beaverTripleShares := shuffle.GenerateRandomBeaverTripleMatrixShares(6, 10, 3) | ||||
| 
 | ||||
| 	uMatrixShares := beaverTripleShares[0] | ||||
| 	vMatrixShares := beaverTripleShares[1] | ||||
| 	uvMatrixShares := beaverTripleShares[2] | ||||
| 
 | ||||
| 	uMatrix := crypto.InterpolateMatrixShares(uMatrixShares, []int{1, 2, 3}) | ||||
| 	vMatrix := crypto.InterpolateMatrixShares(vMatrixShares, []int{1, 2, 3}) | ||||
| 	uvMatrix := crypto.InterpolateMatrixShares(uvMatrixShares, []int{1, 2, 3}) | ||||
| 	uMatrix := shuffle.InterpolateMatrixShares(uMatrixShares, []int{1, 2, 3}) | ||||
| 	vMatrix := shuffle.InterpolateMatrixShares(vMatrixShares, []int{1, 2, 3}) | ||||
| 	uvMatrix := shuffle.InterpolateMatrixShares(uvMatrixShares, []int{1, 2, 3}) | ||||
| 
 | ||||
| 	for x := 0; x < len(uMatrixShares); x++ { | ||||
| 		for y := 0; y < len(uMatrixShares[0]); y++ { | ||||
| @ -141,21 +141,21 @@ func TestGenerateRandomBeaverTripleMatrixShares(t *testing.T) { | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	uvCheck := crypto.GenerateDotProduct(uMatrix, vMatrix) | ||||
| 	uvCheck := shuffle.GenerateDotProduct(uMatrix, vMatrix) | ||||
| 	assert.Equal(t, uvMatrix, uvCheck) | ||||
| } | ||||
| 
 | ||||
| func TestPermutationMatrix(t *testing.T) { | ||||
| 	permutationMatrix1 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix2 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix3 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix4 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix1 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix2 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix3 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix4 := shuffle.GeneratePermutationMatrix(6) | ||||
| 
 | ||||
| 	permutationMatrix := crypto.GenerateDotProduct(permutationMatrix1, permutationMatrix2) | ||||
| 	permutationMatrix = crypto.GenerateDotProduct(permutationMatrix, permutationMatrix3) | ||||
| 	permutationMatrix = crypto.GenerateDotProduct(permutationMatrix, permutationMatrix4) | ||||
| 	permutationMatrix := shuffle.GenerateDotProduct(permutationMatrix1, permutationMatrix2) | ||||
| 	permutationMatrix = shuffle.GenerateDotProduct(permutationMatrix, permutationMatrix3) | ||||
| 	permutationMatrix = shuffle.GenerateDotProduct(permutationMatrix, permutationMatrix4) | ||||
| 
 | ||||
| 	one, _ := edwards25519.NewScalar().SetCanonicalBytes(crypto.BigIntToLEBytes(big.NewInt(1))) | ||||
| 	one, _ := edwards25519.NewScalar().SetCanonicalBytes(shuffle.BigIntToLEBytes(big.NewInt(1))) | ||||
| 	for x := 0; x < 6; x++ { | ||||
| 		sumX := edwards25519.NewScalar() | ||||
| 
 | ||||
| @ -178,14 +178,14 @@ func TestPermutationMatrix(t *testing.T) { | ||||
| } | ||||
| 
 | ||||
| func TestPermutationSharing(t *testing.T) { | ||||
| 	permutationMatrix1 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix2 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix3 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix4 := crypto.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrixShares1 := crypto.ShamirSplitMatrix(permutationMatrix1, 4, 3) | ||||
| 	permutationMatrixShares2 := crypto.ShamirSplitMatrix(permutationMatrix2, 4, 3) | ||||
| 	permutationMatrixShares3 := crypto.ShamirSplitMatrix(permutationMatrix3, 4, 3) | ||||
| 	permutationMatrixShares4 := crypto.ShamirSplitMatrix(permutationMatrix4, 4, 3) | ||||
| 	permutationMatrix1 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix2 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix3 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrix4 := shuffle.GeneratePermutationMatrix(6) | ||||
| 	permutationMatrixShares1 := shuffle.ShamirSplitMatrix(permutationMatrix1, 4, 3) | ||||
| 	permutationMatrixShares2 := shuffle.ShamirSplitMatrix(permutationMatrix2, 4, 3) | ||||
| 	permutationMatrixShares3 := shuffle.ShamirSplitMatrix(permutationMatrix3, 4, 3) | ||||
| 	permutationMatrixShares4 := shuffle.ShamirSplitMatrix(permutationMatrix4, 4, 3) | ||||
| 
 | ||||
| 	inverseShareMatrix1 := make([][][]*edwards25519.Scalar, 4) | ||||
| 	inverseShareMatrix2 := make([][][]*edwards25519.Scalar, 4) | ||||
| @ -213,9 +213,9 @@ func TestPermutationSharing(t *testing.T) { | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	beaverTripleShares1 := crypto.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 	beaverTripleShares2 := crypto.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 	beaverTripleShares3 := crypto.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 	beaverTripleShares1 := shuffle.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 	beaverTripleShares2 := shuffle.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 	beaverTripleShares3 := shuffle.GenerateRandomBeaverTripleMatrixShares(6, 4, 3) | ||||
| 
 | ||||
| 	beaverTriplesAShares1 := beaverTripleShares1[0] | ||||
| 	beaverTriplesBShares1 := beaverTripleShares1[1] | ||||
| @ -310,18 +310,18 @@ func TestPermutationSharing(t *testing.T) { | ||||
| 	// f = b - v
 | ||||
| 	// (a - u)(b - v) = -ab + ub + av - uv + (ab-av) + (ab - ub) + uv
 | ||||
| 
 | ||||
| 	e1 := crypto.InterpolateMatrixShares(es1, []int{1, 2, 3, 4}) | ||||
| 	f1 := crypto.InterpolateMatrixShares(fs1, []int{1, 2, 3, 4}) | ||||
| 	e2 := crypto.InterpolateMatrixShares(es2, []int{1, 2, 3, 4}) | ||||
| 	f2 := crypto.InterpolateMatrixShares(fs2, []int{1, 2, 3, 4}) | ||||
| 	e3 := crypto.InterpolateMatrixShares(es3, []int{1, 2, 3, 4}) | ||||
| 	f3 := crypto.InterpolateMatrixShares(fs3, []int{1, 2, 3, 4}) | ||||
| 	e1 := shuffle.InterpolateMatrixShares(es1, []int{1, 2, 3, 4}) | ||||
| 	f1 := shuffle.InterpolateMatrixShares(fs1, []int{1, 2, 3, 4}) | ||||
| 	e2 := shuffle.InterpolateMatrixShares(es2, []int{1, 2, 3, 4}) | ||||
| 	f2 := shuffle.InterpolateMatrixShares(fs2, []int{1, 2, 3, 4}) | ||||
| 	e3 := shuffle.InterpolateMatrixShares(es3, []int{1, 2, 3, 4}) | ||||
| 	f3 := shuffle.InterpolateMatrixShares(fs3, []int{1, 2, 3, 4}) | ||||
| 
 | ||||
| 	// mul(a, b) => <e> = <a> - <u>, <f> = <b> - <v>, <c> = -i * e * f + f * <a> + e * <b> + <z>
 | ||||
| 
 | ||||
| 	ef1 := crypto.GenerateDotProduct(e1, f1) | ||||
| 	ef2 := crypto.GenerateDotProduct(e2, f2) | ||||
| 	ef3 := crypto.GenerateDotProduct(e3, f3) | ||||
| 	ef1 := shuffle.GenerateDotProduct(e1, f1) | ||||
| 	ef2 := shuffle.GenerateDotProduct(e2, f2) | ||||
| 	ef3 := shuffle.GenerateDotProduct(e3, f3) | ||||
| 	fa1 := make([][][]*edwards25519.Scalar, 4) | ||||
| 	fa2 := make([][][]*edwards25519.Scalar, 4) | ||||
| 	fa3 := make([][][]*edwards25519.Scalar, 4) | ||||
| @ -336,15 +336,15 @@ func TestPermutationSharing(t *testing.T) { | ||||
| 	inverseCS3 := make([][][]*edwards25519.Scalar, 6) | ||||
| 
 | ||||
| 	for i := 0; i < 4; i++ { | ||||
| 		fa1[i] = crypto.GenerateDotProduct(inverseShareMatrix1[i], f1) | ||||
| 		eb1[i] = crypto.GenerateDotProduct(e1, inverseShareMatrix2[i]) | ||||
| 		fa2[i] = crypto.GenerateDotProduct(inverseShareMatrix2[i], f2) | ||||
| 		eb2[i] = crypto.GenerateDotProduct(e2, inverseShareMatrix3[i]) | ||||
| 		fa3[i] = crypto.GenerateDotProduct(inverseShareMatrix3[i], f3) | ||||
| 		eb3[i] = crypto.GenerateDotProduct(e3, inverseShareMatrix4[i]) | ||||
| 		cs1[i] = crypto.AddMatrices(crypto.ScalarMult(-1, ef1), fa1[i], eb1[i], inverseBeaverTriplesABShares1[i]) | ||||
| 		cs2[i] = crypto.AddMatrices(crypto.ScalarMult(-1, ef2), fa2[i], eb2[i], inverseBeaverTriplesABShares2[i]) | ||||
| 		cs3[i] = crypto.AddMatrices(crypto.ScalarMult(-1, ef3), fa3[i], eb3[i], inverseBeaverTriplesABShares3[i]) | ||||
| 		fa1[i] = shuffle.GenerateDotProduct(inverseShareMatrix1[i], f1) | ||||
| 		eb1[i] = shuffle.GenerateDotProduct(e1, inverseShareMatrix2[i]) | ||||
| 		fa2[i] = shuffle.GenerateDotProduct(inverseShareMatrix2[i], f2) | ||||
| 		eb2[i] = shuffle.GenerateDotProduct(e2, inverseShareMatrix3[i]) | ||||
| 		fa3[i] = shuffle.GenerateDotProduct(inverseShareMatrix3[i], f3) | ||||
| 		eb3[i] = shuffle.GenerateDotProduct(e3, inverseShareMatrix4[i]) | ||||
| 		cs1[i] = shuffle.AddMatrices(shuffle.ScalarMult(-1, ef1), fa1[i], eb1[i], inverseBeaverTriplesABShares1[i]) | ||||
| 		cs2[i] = shuffle.AddMatrices(shuffle.ScalarMult(-1, ef2), fa2[i], eb2[i], inverseBeaverTriplesABShares2[i]) | ||||
| 		cs3[i] = shuffle.AddMatrices(shuffle.ScalarMult(-1, ef3), fa3[i], eb3[i], inverseBeaverTriplesABShares3[i]) | ||||
| 	} | ||||
| 
 | ||||
| 	for x := 0; x < 6; x++ { | ||||
| @ -360,12 +360,12 @@ func TestPermutationSharing(t *testing.T) { | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	c1 := crypto.InterpolateMatrixShares(inverseCS1, []int{1, 2, 3, 4}) | ||||
| 	c3 := crypto.InterpolateMatrixShares(inverseCS3, []int{1, 2, 3, 4}) | ||||
| 	c := crypto.GenerateDotProduct(c1, c3) | ||||
| 	ab := crypto.GenerateDotProduct(permutationMatrix1, permutationMatrix2) | ||||
| 	abc := crypto.GenerateDotProduct(ab, permutationMatrix3) | ||||
| 	abcd := crypto.GenerateDotProduct(abc, permutationMatrix4) | ||||
| 	c1 := shuffle.InterpolateMatrixShares(inverseCS1, []int{1, 2, 3, 4}) | ||||
| 	c3 := shuffle.InterpolateMatrixShares(inverseCS3, []int{1, 2, 3, 4}) | ||||
| 	c := shuffle.GenerateDotProduct(c1, c3) | ||||
| 	ab := shuffle.GenerateDotProduct(permutationMatrix1, permutationMatrix2) | ||||
| 	abc := shuffle.GenerateDotProduct(ab, permutationMatrix3) | ||||
| 	abcd := shuffle.GenerateDotProduct(abc, permutationMatrix4) | ||||
| 
 | ||||
| 	for x := 0; x < 6; x++ { | ||||
| 		for y := 0; y < 6; y++ { | ||||
							
								
								
									
										469
									
								
								node/crypto/wesolowski_frame_prover.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										469
									
								
								node/crypto/wesolowski_frame_prover.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,469 @@ | ||||
| package crypto | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"crypto" | ||||
| 	"crypto/rand" | ||||
| 	"encoding/binary" | ||||
| 	"math/big" | ||||
| 
 | ||||
| 	"github.com/cloudflare/circl/sign/ed448" | ||||
| 	"github.com/iden3/go-iden3-crypto/poseidon" | ||||
| 	"github.com/pkg/errors" | ||||
| 	"go.uber.org/zap" | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/vdf" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| ) | ||||
| 
 | ||||
| type WesolowskiFrameProver struct { | ||||
| 	logger *zap.Logger | ||||
| } | ||||
| 
 | ||||
| func NewWesolowskiFrameProver(logger *zap.Logger) *WesolowskiFrameProver { | ||||
| 	return &WesolowskiFrameProver{ | ||||
| 		logger, | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) ProveMasterClockFrame( | ||||
| 	previousFrame *protobufs.ClockFrame, | ||||
| 	timestamp int64, | ||||
| 	difficulty uint32, | ||||
| ) (*protobufs.ClockFrame, error) { | ||||
| 	input := []byte{} | ||||
| 	input = append(input, previousFrame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, previousFrame.FrameNumber+1) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	input = append(input, previousFrame.Output[:]...) | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], previousFrame.Output[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:          previousFrame.Filter, | ||||
| 		FrameNumber:     previousFrame.FrameNumber + 1, | ||||
| 		Timestamp:       timestamp, | ||||
| 		Difficulty:      difficulty, | ||||
| 		ParentSelector:  parent.FillBytes(make([]byte, 32)), | ||||
| 		Input:           previousFrame.Output, | ||||
| 		AggregateProofs: []*protobufs.InclusionAggregateProof{}, | ||||
| 		Output:          o[:], | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) VerifyMasterClockFrame( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	input := []byte{} | ||||
| 	input = append(input, frame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, frame.FrameNumber) | ||||
| 	input = binary.BigEndian.AppendUint32(input, frame.Difficulty) | ||||
| 	input = append(input, frame.Input...) | ||||
| 
 | ||||
| 	if len(frame.Input) < 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if len(frame.AggregateProofs) > 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if frame.PublicKeySignature != nil { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if len(frame.Input) != 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(frame.Difficulty, b) | ||||
| 	proof := [516]byte{} | ||||
| 	copy(proof[:], frame.Output) | ||||
| 
 | ||||
| 	if !v.Verify(proof) { | ||||
| 		w.logger.Error("invalid proof", | ||||
| 			zap.Binary("filter", frame.Filter), | ||||
| 			zap.Uint64("frame_number", frame.FrameNumber), | ||||
| 			zap.Uint32("difficulty", frame.Difficulty), | ||||
| 			zap.Binary("frame_input", frame.Input), | ||||
| 			zap.Binary("frame_output", frame.Output), | ||||
| 		) | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid proof"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], frame.Input[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "verify clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	selector := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 	if parent.Cmp(selector) != 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("selector did not match input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) CreateMasterGenesisFrame( | ||||
| 	filter []byte, | ||||
| 	seed []byte, | ||||
| 	difficulty uint32, | ||||
| ) ( | ||||
| 	*protobufs.ClockFrame, | ||||
| 	error, | ||||
| ) { | ||||
| 	b := sha3.Sum256(seed) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 	inputMessage := o[:] | ||||
| 
 | ||||
| 	w.logger.Debug("proving genesis frame") | ||||
| 	input := []byte{} | ||||
| 	input = append(input, filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, 0) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	if bytes.Equal(seed, []byte{0x00}) { | ||||
| 		value := [516]byte{} | ||||
| 		input = append(input, value[:]...) | ||||
| 	} else { | ||||
| 		input = append(input, seed...) | ||||
| 	} | ||||
| 
 | ||||
| 	b = sha3.Sum256(input) | ||||
| 	v = vdf.New(difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o = v.GetOutput() | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:      filter, | ||||
| 		FrameNumber: 0, | ||||
| 		Timestamp:   0, | ||||
| 		Difficulty:  difficulty, | ||||
| 		Input:       inputMessage, | ||||
| 		Output:      o[:], | ||||
| 		ParentSelector: []byte{ | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 		}, | ||||
| 		AggregateProofs:    []*protobufs.InclusionAggregateProof{}, | ||||
| 		PublicKeySignature: nil, | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) ProveDataClockFrame( | ||||
| 	previousFrame *protobufs.ClockFrame, | ||||
| 	commitments [][]byte, | ||||
| 	aggregateProofs []*protobufs.InclusionAggregateProof, | ||||
| 	provingKey crypto.Signer, | ||||
| 	timestamp int64, | ||||
| 	difficulty uint32, | ||||
| ) (*protobufs.ClockFrame, error) { | ||||
| 	var pubkey []byte | ||||
| 	pubkeyType := keys.KeyTypeEd448 | ||||
| 	ed448PublicKey, ok := provingKey.Public().(ed448.PublicKey) | ||||
| 	if ok { | ||||
| 		pubkey = []byte(ed448PublicKey) | ||||
| 	} else { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("no valid signature provided"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	h, err := poseidon.HashBytes(pubkey) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("could not hash proving key"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	address := h.Bytes() | ||||
| 	input := []byte{} | ||||
| 	input = append(input, previousFrame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, previousFrame.FrameNumber+1) | ||||
| 	input = binary.BigEndian.AppendUint64(input, uint64(timestamp)) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	input = append(input, address...) | ||||
| 	input = append(input, previousFrame.Output[:]...) | ||||
| 
 | ||||
| 	commitmentInput := []byte{} | ||||
| 	for _, commitment := range commitments { | ||||
| 		commitmentInput = append(commitmentInput, commitment...) | ||||
| 	} | ||||
| 
 | ||||
| 	input = append(input, commitmentInput...) | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 
 | ||||
| 	// TODO: make this configurable for signing algorithms that allow
 | ||||
| 	// user-supplied hash functions
 | ||||
| 	signature, err := provingKey.Sign( | ||||
| 		rand.Reader, | ||||
| 		append(append([]byte{}, b[:]...), o[:]...), | ||||
| 		crypto.Hash(0), | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			err, | ||||
| 			"prove", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], previousFrame.Output[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:         previousFrame.Filter, | ||||
| 		FrameNumber:    previousFrame.FrameNumber + 1, | ||||
| 		Timestamp:      timestamp, | ||||
| 		Difficulty:     difficulty, | ||||
| 		ParentSelector: parent.Bytes(), | ||||
| 		Input: append( | ||||
| 			append([]byte{}, previousFrame.Output...), | ||||
| 			commitmentInput..., | ||||
| 		), | ||||
| 		AggregateProofs: aggregateProofs, | ||||
| 		Output:          o[:], | ||||
| 	} | ||||
| 
 | ||||
| 	switch pubkeyType { | ||||
| 	case keys.KeyTypeEd448: | ||||
| 		frame.PublicKeySignature = &protobufs.ClockFrame_PublicKeySignatureEd448{ | ||||
| 			PublicKeySignatureEd448: &protobufs.Ed448Signature{ | ||||
| 				Signature: signature, | ||||
| 				PublicKey: &protobufs.Ed448PublicKey{ | ||||
| 					KeyValue: pubkey, | ||||
| 				}, | ||||
| 			}, | ||||
| 		} | ||||
| 	default: | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("unsupported proving key"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) CreateDataGenesisFrame( | ||||
| 	filter []byte, | ||||
| 	origin []byte, | ||||
| 	difficulty uint32, | ||||
| 	inclusionProof *InclusionAggregateProof, | ||||
| 	proverKeys [][]byte, | ||||
| 	preDusk bool, | ||||
| ) (*protobufs.ClockFrame, *tries.RollingFrecencyCritbitTrie, error) { | ||||
| 	frameProverTrie := &tries.RollingFrecencyCritbitTrie{} | ||||
| 	for _, s := range proverKeys { | ||||
| 		addr, err := poseidon.HashBytes(s) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		addrBytes := addr.Bytes() | ||||
| 		addrBytes = append(make([]byte, 32-len(addrBytes)), addrBytes...) | ||||
| 		frameProverTrie.Add(addrBytes, 0) | ||||
| 	} | ||||
| 
 | ||||
| 	w.logger.Info("proving genesis frame") | ||||
| 	input := []byte{} | ||||
| 	input = append(input, filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, 0) | ||||
| 	input = binary.BigEndian.AppendUint64(input, 0) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	input = append(input, origin...) | ||||
| 	if !preDusk { | ||||
| 		input = append(input, inclusionProof.AggregateCommitment...) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 
 | ||||
| 	commitments := []*protobufs.InclusionCommitment{} | ||||
| 	for i, commit := range inclusionProof.InclusionCommitments { | ||||
| 		commitments = append(commitments, &protobufs.InclusionCommitment{ | ||||
| 			Filter:      filter, | ||||
| 			FrameNumber: 0, | ||||
| 			Position:    uint32(i), | ||||
| 			TypeUrl:     commit.TypeUrl, | ||||
| 			Data:        commit.Data, | ||||
| 			Commitment:  commit.Commitment, | ||||
| 		}) | ||||
| 	} | ||||
| 
 | ||||
| 	frame := &protobufs.ClockFrame{ | ||||
| 		Filter:      filter, | ||||
| 		FrameNumber: 0, | ||||
| 		Timestamp:   0, | ||||
| 		Difficulty:  difficulty, | ||||
| 		Input: append( | ||||
| 			append([]byte{}, origin...), | ||||
| 			inclusionProof.AggregateCommitment..., | ||||
| 		), | ||||
| 		Output: o[:], | ||||
| 		ParentSelector: []byte{ | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 		}, | ||||
| 		AggregateProofs: []*protobufs.InclusionAggregateProof{ | ||||
| 			{ | ||||
| 				Filter:               filter, | ||||
| 				FrameNumber:          0, | ||||
| 				InclusionCommitments: commitments, | ||||
| 				Proof:                inclusionProof.Proof, | ||||
| 			}, | ||||
| 		}, | ||||
| 		PublicKeySignature: nil, | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, frameProverTrie, nil | ||||
| } | ||||
| 
 | ||||
| func (w *WesolowskiFrameProver) VerifyDataClockFrame( | ||||
| 	frame *protobufs.ClockFrame, | ||||
| ) error { | ||||
| 	var pubkey []byte | ||||
| 	var signature []byte | ||||
| 	pubkeyType := keys.KeyTypeEd448 | ||||
| 	ed448PublicKey := frame.GetPublicKeySignatureEd448() | ||||
| 	if ed448PublicKey != nil { | ||||
| 		pubkey = ed448PublicKey.PublicKey.KeyValue | ||||
| 		signature = ed448PublicKey.Signature | ||||
| 	} else { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("no valid signature provided"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	h, err := poseidon.HashBytes(pubkey) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("could not hash proving key"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	address := h.Bytes() | ||||
| 
 | ||||
| 	input := []byte{} | ||||
| 	input = append(input, frame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, frame.FrameNumber) | ||||
| 	input = binary.BigEndian.AppendUint64(input, uint64(frame.Timestamp)) | ||||
| 	input = binary.BigEndian.AppendUint32(input, frame.Difficulty) | ||||
| 	input = append(input, address...) | ||||
| 	input = append(input, frame.Input...) | ||||
| 
 | ||||
| 	if len(frame.Input) < 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(frame.Difficulty, b) | ||||
| 	proof := [516]byte{} | ||||
| 	copy(proof[:], frame.Output) | ||||
| 
 | ||||
| 	// TODO: make this configurable for signing algorithms that allow
 | ||||
| 	// user-supplied hash functions
 | ||||
| 	switch pubkeyType { | ||||
| 	case keys.KeyTypeEd448: | ||||
| 		if len(pubkey) != 57 || len(signature) != 114 || !ed448.VerifyAny( | ||||
| 			pubkey, | ||||
| 			append(append([]byte{}, b[:]...), frame.Output...), | ||||
| 			signature, | ||||
| 			crypto.Hash(0), | ||||
| 		) { | ||||
| 			return errors.Wrap( | ||||
| 				errors.New("invalid signature for issuer"), | ||||
| 				"verify clock frame", | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 	if !v.Verify(proof) { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid proof"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], frame.Input[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "verify clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	selector := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 	if parent.Cmp(selector) != 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("selector did not match input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
							
								
								
									
										28
									
								
								node/crypto/wesolowski_frame_prover_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										28
									
								
								node/crypto/wesolowski_frame_prover_test.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,28 @@ | ||||
| package crypto_test | ||||
| 
 | ||||
| import ( | ||||
| 	"bytes" | ||||
| 	"testing" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/stretchr/testify/assert" | ||||
| 	"go.uber.org/zap" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| ) | ||||
| 
 | ||||
| func TestMasterProve(t *testing.T) { | ||||
| 	l, _ := zap.NewProduction() | ||||
| 	w := crypto.NewWesolowskiFrameProver(l) | ||||
| 	m, err := w.CreateMasterGenesisFrame([]byte{ | ||||
| 		0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 		0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 		0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 		0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 	}, bytes.Repeat([]byte{0x00}, 516), 10000) | ||||
| 	assert.NoError(t, err) | ||||
| 
 | ||||
| 	next, err := w.ProveMasterClockFrame(m, time.Now().UnixMilli(), 10000) | ||||
| 	assert.NoError(t, err) | ||||
| 	err = w.VerifyMasterClockFrame(next) | ||||
| 	assert.NoError(t, err) | ||||
| } | ||||
| @ -667,7 +667,10 @@ func MaterializeApplicationFromFrame( | ||||
| 			lastStateCount := frame.FrameNumber - | ||||
| 				inProgress.ActiveParticipants[len(inProgress.ActiveParticipants)-1].FrameNumber | ||||
| 			if !setCount || (lastStateCount < stateCount && lastStateCount > 20) { | ||||
| 				stateCount = lastStateCount - 20 | ||||
| 				stateCount = lastStateCount | ||||
| 				if stateCount >= 10 { | ||||
| 					stateCount -= 10 | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| @ -760,9 +763,26 @@ func MaterializeApplicationFromFrame( | ||||
| func (a *CeremonyApplication) ApplyTransition( | ||||
| 	currentFrameNumber uint64, | ||||
| 	transition *protobufs.CeremonyLobbyStateTransition, | ||||
| ) (*CeremonyApplication, error) { | ||||
| 	skipFailures bool, | ||||
| ) ( | ||||
| 	*CeremonyApplication, | ||||
| 	*protobufs.CeremonyLobbyStateTransition, | ||||
| 	*protobufs.CeremonyLobbyStateTransition, | ||||
| 	error, | ||||
| ) { | ||||
| 	finalizedTransition := &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{}, | ||||
| 		TransitionInputs: [][]byte{}, | ||||
| 	} | ||||
| 	skippedTransition := &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{}, | ||||
| 		TransitionInputs: [][]byte{}, | ||||
| 	} | ||||
| 	switch a.LobbyState { | ||||
| 	case CEREMONY_APPLICATION_STATE_OPEN: | ||||
| 		if a.StateCount > currentFrameNumber { | ||||
| 			a.StateCount = 0 | ||||
| 		} | ||||
| 		a.StateCount++ | ||||
| 
 | ||||
| 		for i, url := range transition.TypeUrls { | ||||
| @ -771,34 +791,75 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				join := &protobufs.CeremonyLobbyJoin{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], join) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber < join.FrameNumber { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("too recent join"), "apply transition", | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("too recent join"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					skippedTransition.TypeUrls = append( | ||||
| 						skippedTransition.TypeUrls, | ||||
| 						url, | ||||
| 					) | ||||
| 					skippedTransition.TransitionInputs = append( | ||||
| 						skippedTransition.TransitionInputs, | ||||
| 						transition.TransitionInputs[i], | ||||
| 					) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber-join.FrameNumber > 10 { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("outdated join"), "apply transition", | ||||
| 					) | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("outdated join"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyLobbyJoin(join); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			default: | ||||
| 				return a, nil | ||||
| 				if !skipFailures { | ||||
| 					return a, nil, nil, nil | ||||
| 				} | ||||
| 
 | ||||
| 				skippedTransition.TypeUrls = append( | ||||
| 					skippedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				skippedTransition.TransitionInputs = append( | ||||
| 					skippedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 				continue | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		if a.StateCount > 10 { | ||||
| 			if len(a.LobbyJoins) == 0 { | ||||
| 				return a, nil | ||||
| 				return a, finalizedTransition, skippedTransition, nil | ||||
| 			} | ||||
| 
 | ||||
| 			a.LobbyState = CEREMONY_APPLICATION_STATE_IN_PROGRESS | ||||
| @ -806,14 +867,17 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 			a.RoundCount = 1 | ||||
| 
 | ||||
| 			if err := a.finalizeParticipantSet(); err != nil { | ||||
| 				return nil, errors.Wrap(err, "apply transition") | ||||
| 				return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 			} | ||||
| 
 | ||||
| 			a.LobbyJoins = []*protobufs.CeremonyLobbyJoin{} | ||||
| 		} | ||||
| 
 | ||||
| 		return a, nil | ||||
| 		return a, finalizedTransition, skippedTransition, nil | ||||
| 	case CEREMONY_APPLICATION_STATE_IN_PROGRESS: | ||||
| 		if a.StateCount > currentFrameNumber { | ||||
| 			a.StateCount = 0 | ||||
| 		} | ||||
| 		a.StateCount++ | ||||
| 		for i, url := range transition.TypeUrls { | ||||
| 			switch url { | ||||
| @ -821,61 +885,149 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				seenProverAtt := &protobufs.CeremonySeenProverAttestation{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], seenProverAtt) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber < seenProverAtt.LastSeenFrame { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("too recent attestation"), "apply transition", | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("too recent attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					skippedTransition.TypeUrls = append( | ||||
| 						skippedTransition.TypeUrls, | ||||
| 						url, | ||||
| 					) | ||||
| 					skippedTransition.TransitionInputs = append( | ||||
| 						skippedTransition.TransitionInputs, | ||||
| 						transition.TransitionInputs[i], | ||||
| 					) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber-seenProverAtt.LastSeenFrame > 10 { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("outdated attestation"), "apply transition", | ||||
| 					) | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("outdated attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applySeenProverAttestation(seenProverAtt); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			case protobufs.CeremonyDroppedProverAttestationType: | ||||
| 				droppedProverAtt := &protobufs.CeremonyDroppedProverAttestation{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], droppedProverAtt) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber < droppedProverAtt.LastSeenFrame { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("too recent attestation"), "apply transition", | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("too recent attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					skippedTransition.TypeUrls = append( | ||||
| 						skippedTransition.TypeUrls, | ||||
| 						url, | ||||
| 					) | ||||
| 					skippedTransition.TransitionInputs = append( | ||||
| 						skippedTransition.TransitionInputs, | ||||
| 						transition.TransitionInputs[i], | ||||
| 					) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber-droppedProverAtt.LastSeenFrame > 10 { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("outdated attestation"), "apply transition", | ||||
| 					) | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("outdated attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyDroppedProverAttestation(droppedProverAtt); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			case protobufs.CeremonyTranscriptCommitType: | ||||
| 				transcriptCommit := &protobufs.CeremonyTranscriptCommit{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], transcriptCommit) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyTranscriptCommit(transcriptCommit); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			default: | ||||
| 				return a, nil | ||||
| 				if !skipFailures { | ||||
| 					return a, nil, nil, nil | ||||
| 				} | ||||
| 
 | ||||
| 				skippedTransition.TypeUrls = append( | ||||
| 					skippedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				skippedTransition.TransitionInputs = append( | ||||
| 					skippedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 				continue | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| @ -898,7 +1050,7 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				[]*protobufs.CeremonySeenProverAttestation{} | ||||
| 			a.TranscriptRoundAdvanceCommits = | ||||
| 				[]*protobufs.CeremonyAdvanceRound{} | ||||
| 			return a, nil | ||||
| 			return a, finalizedTransition, skippedTransition, nil | ||||
| 		} | ||||
| 
 | ||||
| 		attLimit := 1<<a.RoundCount - 1 | ||||
| @ -949,8 +1101,11 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				[]*protobufs.CeremonyAdvanceRound{} | ||||
| 		} | ||||
| 
 | ||||
| 		return a, nil | ||||
| 		return a, finalizedTransition, skippedTransition, nil | ||||
| 	case CEREMONY_APPLICATION_STATE_FINALIZING: | ||||
| 		if a.StateCount > currentFrameNumber { | ||||
| 			a.StateCount = 0 | ||||
| 		} | ||||
| 		a.StateCount++ | ||||
| 		for i, url := range transition.TypeUrls { | ||||
| 			switch url { | ||||
| @ -958,67 +1113,155 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				seenProverAtt := &protobufs.CeremonySeenProverAttestation{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], seenProverAtt) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber < seenProverAtt.LastSeenFrame { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("too recent attestation"), "apply transition", | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("too recent attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					skippedTransition.TypeUrls = append( | ||||
| 						skippedTransition.TypeUrls, | ||||
| 						url, | ||||
| 					) | ||||
| 					skippedTransition.TransitionInputs = append( | ||||
| 						skippedTransition.TransitionInputs, | ||||
| 						transition.TransitionInputs[i], | ||||
| 					) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber-seenProverAtt.LastSeenFrame > 10 { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("outdated attestation"), "apply transition", | ||||
| 					) | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("outdated attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applySeenProverAttestation(seenProverAtt); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			case protobufs.CeremonyDroppedProverAttestationType: | ||||
| 				droppedProverAtt := &protobufs.CeremonyDroppedProverAttestation{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], droppedProverAtt) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber < droppedProverAtt.LastSeenFrame { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("too recent attestation"), "apply transition", | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("too recent attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					skippedTransition.TypeUrls = append( | ||||
| 						skippedTransition.TypeUrls, | ||||
| 						url, | ||||
| 					) | ||||
| 					skippedTransition.TransitionInputs = append( | ||||
| 						skippedTransition.TransitionInputs, | ||||
| 						transition.TransitionInputs[i], | ||||
| 					) | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if currentFrameNumber-droppedProverAtt.LastSeenFrame > 10 { | ||||
| 					return nil, errors.Wrap( | ||||
| 						errors.New("outdated attestation"), "apply transition", | ||||
| 					) | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap( | ||||
| 							errors.New("outdated attestation"), "apply transition", | ||||
| 						) | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyDroppedProverAttestation(droppedProverAtt); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			case protobufs.CeremonyTranscriptShareType: | ||||
| 				transcriptShare := &protobufs.CeremonyTranscriptShare{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], transcriptShare) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyTranscriptShare(transcriptShare); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			default: | ||||
| 				return a, nil | ||||
| 				if !skipFailures { | ||||
| 					return a, nil, nil, nil | ||||
| 				} | ||||
| 
 | ||||
| 				skippedTransition.TypeUrls = append( | ||||
| 					skippedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				skippedTransition.TransitionInputs = append( | ||||
| 					skippedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 				continue | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		if len(a.TranscriptShares) == len(a.ActiveParticipants) { | ||||
| 			if err := a.finalizeTranscript(); err != nil { | ||||
| 				return nil, errors.Wrap(err, "apply transition") | ||||
| 				return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 			} | ||||
| 
 | ||||
| 			a.LobbyState = CEREMONY_APPLICATION_STATE_VALIDATING | ||||
| @ -1027,7 +1270,7 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				[]*protobufs.CeremonyDroppedProverAttestation{} | ||||
| 			a.LatestSeenProverAttestations = | ||||
| 				[]*protobufs.CeremonySeenProverAttestation{} | ||||
| 
 | ||||
| 			return a, finalizedTransition, skippedTransition, nil | ||||
| 		} | ||||
| 
 | ||||
| 		shouldReset := false | ||||
| @ -1048,7 +1291,7 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		if a.StateCount > 10 { | ||||
| 		if a.StateCount > 100 { | ||||
| 			shouldReset = true | ||||
| 		} | ||||
| 
 | ||||
| @ -1081,8 +1324,11 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				[]*protobufs.CeremonyTranscriptShare{} | ||||
| 		} | ||||
| 
 | ||||
| 		return a, nil | ||||
| 		return a, finalizedTransition, skippedTransition, nil | ||||
| 	case CEREMONY_APPLICATION_STATE_VALIDATING: | ||||
| 		if a.StateCount > currentFrameNumber { | ||||
| 			a.StateCount = 0 | ||||
| 		} | ||||
| 		a.StateCount++ | ||||
| 		for i, url := range transition.TypeUrls { | ||||
| 			switch url { | ||||
| @ -1090,20 +1336,47 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 				transcript := &protobufs.CeremonyTranscript{} | ||||
| 				err := proto.Unmarshal(transition.TransitionInputs[i], transcript) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				if err = a.applyTranscript(transcript); err != nil { | ||||
| 					return nil, errors.Wrap(err, "apply transition") | ||||
| 					if !skipFailures { | ||||
| 						return nil, nil, nil, errors.Wrap(err, "apply transition") | ||||
| 					} | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				finalizedTransition.TypeUrls = append( | ||||
| 					finalizedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				finalizedTransition.TransitionInputs = append( | ||||
| 					finalizedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 
 | ||||
| 				a.StateCount = 0 | ||||
| 			default: | ||||
| 				return a, nil | ||||
| 				if !skipFailures { | ||||
| 					return a, nil, nil, nil | ||||
| 				} | ||||
| 				skippedTransition.TypeUrls = append( | ||||
| 					skippedTransition.TypeUrls, | ||||
| 					url, | ||||
| 				) | ||||
| 				skippedTransition.TransitionInputs = append( | ||||
| 					skippedTransition.TransitionInputs, | ||||
| 					transition.TransitionInputs[i], | ||||
| 				) | ||||
| 				continue | ||||
| 			} | ||||
| 		} | ||||
| 
 | ||||
| 		shouldReset := false | ||||
| 		if a.StateCount > 100 { | ||||
| 		if a.StateCount > 300 { | ||||
| 			shouldReset = true | ||||
| 		} | ||||
| 
 | ||||
| @ -1160,9 +1433,9 @@ func (a *CeremonyApplication) ApplyTransition( | ||||
| 			a.FinalCommits = []*protobufs.CeremonyTranscriptCommit{} | ||||
| 		} | ||||
| 
 | ||||
| 		return a, nil | ||||
| 		return a, finalizedTransition, skippedTransition, nil | ||||
| 	default: | ||||
| 		return nil, errors.Wrap(ErrInvalidStateTransition, "apply transition") | ||||
| 		return nil, nil, nil, errors.Wrap(ErrInvalidStateTransition, "apply transition") | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| @ -5,7 +5,6 @@ import ( | ||||
| 	"fmt" | ||||
| 
 | ||||
| 	"github.com/pkg/errors" | ||||
| 	"golang.org/x/sync/errgroup" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| @ -79,43 +78,30 @@ func (a *CeremonyApplication) applyTranscriptShare( | ||||
| 		return errors.Wrap(err, "apply transcript share") | ||||
| 	} | ||||
| 
 | ||||
| 	eg := errgroup.Group{} | ||||
| 	eg.SetLimit(100) | ||||
| 
 | ||||
| 	for i, g1 := range a.LatestTranscript.G1Powers { | ||||
| 		i := i | ||||
| 		g1 := g1 | ||||
| 		eg.Go(func() error { | ||||
| 			if _, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 				g1.KeyValue, | ||||
| 			); err != nil { | ||||
| 				return errors.Wrap( | ||||
| 					errors.Wrap(err, fmt.Sprintf("invalid g1 at position %d", i)), | ||||
| 					"apply transcript share", | ||||
| 				) | ||||
| 			} | ||||
| 			return nil | ||||
| 		}) | ||||
| 		if _, err := curves.BLS48581G1().Point.FromAffineCompressed( | ||||
| 			g1.KeyValue, | ||||
| 		); err != nil { | ||||
| 			return errors.Wrap( | ||||
| 				errors.Wrap(err, fmt.Sprintf("invalid g1 at position %d", i)), | ||||
| 				"apply transcript share", | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	for i, g2 := range a.LatestTranscript.G2Powers { | ||||
| 		i := i | ||||
| 		g2 := g2 | ||||
| 		eg.Go(func() error { | ||||
| 			if _, err := curves.BLS48581G2().Point.FromAffineCompressed( | ||||
| 				g2.KeyValue, | ||||
| 			); err != nil { | ||||
| 				return errors.Wrap( | ||||
| 					errors.Wrap(err, fmt.Sprintf("invalid g2 at position %d", i)), | ||||
| 					"apply transcript share", | ||||
| 				) | ||||
| 			} | ||||
| 			return nil | ||||
| 		}) | ||||
| 	} | ||||
| 
 | ||||
| 	if err := eg.Wait(); err != nil { | ||||
| 		return err | ||||
| 		if _, err := curves.BLS48581G2().Point.FromAffineCompressed( | ||||
| 			g2.KeyValue, | ||||
| 		); err != nil { | ||||
| 			return errors.Wrap( | ||||
| 				errors.Wrap(err, fmt.Sprintf("invalid g2 at position %d", i)), | ||||
| 				"apply transcript share", | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	exists := false | ||||
| @ -100,26 +100,26 @@ func TestCeremonyTransitions(t *testing.T) { | ||||
| 	joinBytes, err := proto.Marshal(join) | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	a, err = a.ApplyTransition(0, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 	a, _, _, err = a.ApplyTransition(0, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{protobufs.CeremonyLobbyJoinType}, | ||||
| 		TransitionInputs: [][]byte{joinBytes}, | ||||
| 	}) | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_OPEN) | ||||
| 
 | ||||
| 	for i := uint64(0); i < 10; i++ { | ||||
| 		a, err = a.ApplyTransition(i+1, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		a, _, _, err = a.ApplyTransition(i+1, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 			TypeUrls:         []string{}, | ||||
| 			TransitionInputs: [][]byte{}, | ||||
| 		}) | ||||
| 		}, false) | ||||
| 		require.NoError(t, err) | ||||
| 		require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_OPEN) | ||||
| 	} | ||||
| 
 | ||||
| 	a, err = a.ApplyTransition(12, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 	a, _, _, err = a.ApplyTransition(12, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{}, | ||||
| 		TransitionInputs: [][]byte{}, | ||||
| 	}) | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_IN_PROGRESS) | ||||
| 	require.True(t, bytes.Equal( | ||||
| @ -165,10 +165,10 @@ func TestCeremonyTransitions(t *testing.T) { | ||||
| 	advanceRoundBytes, err := proto.Marshal(advanceRound) | ||||
| 
 | ||||
| 	require.NoError(t, err) | ||||
| 	a, err = a.ApplyTransition(13, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 	a, _, _, err = a.ApplyTransition(13, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{protobufs.CeremonyTranscriptCommitType}, | ||||
| 		TransitionInputs: [][]byte{advanceRoundBytes}, | ||||
| 	}) | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_FINALIZING) | ||||
| 
 | ||||
| @ -209,10 +209,10 @@ func TestCeremonyTransitions(t *testing.T) { | ||||
| 	shareBytes, err := proto.Marshal(transcriptShare) | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	a, err = a.ApplyTransition(14, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 	a, _, _, err = a.ApplyTransition(14, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{protobufs.CeremonyTranscriptShareType}, | ||||
| 		TransitionInputs: [][]byte{shareBytes}, | ||||
| 	}) | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_VALIDATING) | ||||
| 
 | ||||
| @ -272,10 +272,10 @@ func TestCeremonyTransitions(t *testing.T) { | ||||
| 	} | ||||
| 	transcriptBytes, err := proto.Marshal(updatedTranscript) | ||||
| 	require.NoError(t, err) | ||||
| 	a, err = a.ApplyTransition(15, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 	a, _, _, err = a.ApplyTransition(15, &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{protobufs.CeremonyTranscriptType}, | ||||
| 		TransitionInputs: [][]byte{transcriptBytes}, | ||||
| 	}) | ||||
| 	}, false) | ||||
| 	require.NoError(t, err) | ||||
| 	require.Equal(t, a.LobbyState, CEREMONY_APPLICATION_STATE_OPEN) | ||||
| 	bi, err := poseidon.HashBytes(proverPubKey) | ||||
| @ -12,8 +12,8 @@ import ( | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	bls48581 "source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves/native/bls48581" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/ot/base/simplest" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/channel" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| ) | ||||
| 
 | ||||
| func TestPairings(t *testing.T) { | ||||
| @ -329,7 +329,7 @@ func TestCompositeConstructionOfBLS(t *testing.T) { | ||||
| 	x448ReceivingIdentityKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingIdentityPrivateKey) | ||||
| 	x448ReceivingSignedPreKey := curves.ED448().NewGeneratorPoint().Mul(x448ReceivingSignedPrePrivateKey) | ||||
| 
 | ||||
| 	senderResult := crypto.SenderX3DH( | ||||
| 	senderResult := channel.SenderX3DH( | ||||
| 		x448SendingIdentityPrivateKey, | ||||
| 		x448SendingEphemeralPrivateKey, | ||||
| 		x448ReceivingIdentityKey, | ||||
| @ -337,7 +337,7 @@ func TestCompositeConstructionOfBLS(t *testing.T) { | ||||
| 		96, | ||||
| 	) | ||||
| 
 | ||||
| 	receiverResult := crypto.ReceiverX3DH( | ||||
| 	receiverResult := channel.ReceiverX3DH( | ||||
| 		x448ReceivingIdentityPrivateKey, | ||||
| 		x448ReceivingSignedPrePrivateKey, | ||||
| 		x448SendingIdentityKey, | ||||
| @ -345,7 +345,7 @@ func TestCompositeConstructionOfBLS(t *testing.T) { | ||||
| 		96, | ||||
| 	) | ||||
| 
 | ||||
| 	drSender, err := crypto.NewDoubleRatchetParticipant( | ||||
| 	drSender, err := channel.NewDoubleRatchetParticipant( | ||||
| 		senderResult[:32], | ||||
| 		senderResult[32:64], | ||||
| 		senderResult[64:], | ||||
| @ -357,7 +357,7 @@ func TestCompositeConstructionOfBLS(t *testing.T) { | ||||
| 	) | ||||
| 	require.NoError(t, err) | ||||
| 
 | ||||
| 	drReceiver, err := crypto.NewDoubleRatchetParticipant( | ||||
| 	drReceiver, err := channel.NewDoubleRatchetParticipant( | ||||
| 		receiverResult[:32], | ||||
| 		receiverResult[32:64], | ||||
| 		receiverResult[64:], | ||||
| @ -4,8 +4,12 @@ import ( | ||||
| 	"bytes" | ||||
| 	"crypto" | ||||
| 	"crypto/rand" | ||||
| 	"encoding/base64" | ||||
| 	"encoding/binary" | ||||
| 	"encoding/hex" | ||||
| 	"encoding/json" | ||||
| 	"fmt" | ||||
| 	"os" | ||||
| 	"strings" | ||||
| 	"sync" | ||||
| 
 | ||||
| @ -18,15 +22,19 @@ import ( | ||||
| 	"google.golang.org/protobuf/types/known/anypb" | ||||
| 
 | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/vdf" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/ceremony" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/consensus/time" | ||||
| 	qcrypto "source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/kzg" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/store" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/tries" | ||||
| ) | ||||
| 
 | ||||
| type CeremonyExecutionEngine struct { | ||||
| @ -41,6 +49,7 @@ type CeremonyExecutionEngine struct { | ||||
| 	provingKey                 crypto.Signer | ||||
| 	proverPublicKey            []byte | ||||
| 	provingKeyAddress          []byte | ||||
| 	inclusionProver            qcrypto.InclusionProver | ||||
| 	participantMx              sync.Mutex | ||||
| 	peerChannels               map[string]*p2p.PublicP2PChannel | ||||
| 	activeSecrets              []curves.Scalar | ||||
| @ -50,6 +59,7 @@ type CeremonyExecutionEngine struct { | ||||
| 	seenMessageMap             map[string]bool | ||||
| 	seenMessageMx              sync.Mutex | ||||
| 	intrinsicFilter            []byte | ||||
| 	frameProver                qcrypto.FrameProver | ||||
| } | ||||
| 
 | ||||
| func NewCeremonyExecutionEngine( | ||||
| @ -57,7 +67,10 @@ func NewCeremonyExecutionEngine( | ||||
| 	engineConfig *config.EngineConfig, | ||||
| 	keyManager keys.KeyManager, | ||||
| 	pubSub p2p.PubSub, | ||||
| 	frameProver qcrypto.FrameProver, | ||||
| 	inclusionProver qcrypto.InclusionProver, | ||||
| 	clockStore store.ClockStore, | ||||
| 	masterTimeReel *time.MasterTimeReel, | ||||
| 	keyStore store.KeyStore, | ||||
| ) *CeremonyExecutionEngine { | ||||
| 	if logger == nil { | ||||
| @ -74,6 +87,31 @@ func NewCeremonyExecutionEngine( | ||||
| 		p2p.GetBloomFilterIndices(application.CEREMONY_ADDRESS, 65536, 24)..., | ||||
| 	) | ||||
| 
 | ||||
| 	_, _, err = clockStore.GetDataClockFrame(intrinsicFilter, 0) | ||||
| 	var origin []byte | ||||
| 	var inclusionProof *qcrypto.InclusionAggregateProof | ||||
| 	var proverKeys [][]byte | ||||
| 
 | ||||
| 	if err != nil && errors.Is(err, store.ErrNotFound) { | ||||
| 		origin, inclusionProof, proverKeys = CreateGenesisState( | ||||
| 			logger, | ||||
| 			engineConfig, | ||||
| 			nil, | ||||
| 			inclusionProver, | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	dataTimeReel := time.NewDataTimeReel( | ||||
| 		intrinsicFilter, | ||||
| 		logger, | ||||
| 		clockStore, | ||||
| 		engineConfig, | ||||
| 		frameProver, | ||||
| 		origin, | ||||
| 		inclusionProof, | ||||
| 		proverKeys, | ||||
| 	) | ||||
| 
 | ||||
| 	clock := ceremony.NewCeremonyDataClockConsensusEngine( | ||||
| 		engineConfig, | ||||
| 		logger, | ||||
| @ -81,6 +119,10 @@ func NewCeremonyExecutionEngine( | ||||
| 		clockStore, | ||||
| 		keyStore, | ||||
| 		pubSub, | ||||
| 		frameProver, | ||||
| 		inclusionProver, | ||||
| 		masterTimeReel, | ||||
| 		dataTimeReel, | ||||
| 		intrinsicFilter, | ||||
| 		seed, | ||||
| 	) | ||||
| @ -93,6 +135,8 @@ func NewCeremonyExecutionEngine( | ||||
| 		clockStore:            clockStore, | ||||
| 		keyStore:              keyStore, | ||||
| 		pubSub:                pubSub, | ||||
| 		inclusionProver:       inclusionProver, | ||||
| 		frameProver:           frameProver, | ||||
| 		participantMx:         sync.Mutex{}, | ||||
| 		peerChannels:          map[string]*p2p.PublicP2PChannel{}, | ||||
| 		alreadyPublishedShare: false, | ||||
| @ -139,13 +183,234 @@ func ( | ||||
| 	} | ||||
| } | ||||
| 
 | ||||
| // Creates a genesis state for the intrinsic
 | ||||
| func CreateGenesisState( | ||||
| 	logger *zap.Logger, | ||||
| 	engineConfig *config.EngineConfig, | ||||
| 	testProverKeys [][]byte, | ||||
| 	inclusionProver qcrypto.InclusionProver, | ||||
| ) ( | ||||
| 	[]byte, | ||||
| 	*qcrypto.InclusionAggregateProof, | ||||
| 	[][]byte, | ||||
| ) { | ||||
| 	seed, err := hex.DecodeString(engineConfig.GenesisSeed) | ||||
| 
 | ||||
| 	if err != nil { | ||||
| 		panic(errors.New("genesis seed is nil")) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("creating genesis frame") | ||||
| 	for _, l := range strings.Split(string(seed), "\n") { | ||||
| 		logger.Info(l) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(seed) | ||||
| 	v := vdf.New(engineConfig.Difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 	inputMessage := o[:] | ||||
| 
 | ||||
| 	// Signatories are special, they don't have an inclusion proof because they
 | ||||
| 	// have not broadcasted communication keys, but they still get contribution
 | ||||
| 	// rights prior to PoMW, because they did produce meaningful work in the
 | ||||
| 	// first phase:
 | ||||
| 	logger.Info("encoding signatories to prover trie") | ||||
| 	proverKeys := [][]byte{} | ||||
| 	if len(testProverKeys) != 0 { | ||||
| 		logger.Warn( | ||||
| 			"TEST PROVER ENTRIES BEING ADDED, YOUR NODE WILL BE KICKED IF IN" + | ||||
| 				" PRODUCTION", | ||||
| 		) | ||||
| 		proverKeys = testProverKeys | ||||
| 	} else { | ||||
| 		for _, s := range kzg.CeremonySignatories { | ||||
| 			pubkey := s.ToAffineCompressed() | ||||
| 			logger.Info("0x" + hex.EncodeToString(pubkey)) | ||||
| 
 | ||||
| 			proverKeys = append(proverKeys, pubkey) | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("encoding ceremony and phase one signatories") | ||||
| 	transcript := &protobufs.CeremonyTranscript{} | ||||
| 	for p, s := range kzg.CeremonyBLS48581G1 { | ||||
| 		transcript.G1Powers = append( | ||||
| 			transcript.G1Powers, | ||||
| 			&protobufs.BLS48581G1PublicKey{ | ||||
| 				KeyValue: s.ToAffineCompressed(), | ||||
| 			}, | ||||
| 		) | ||||
| 		logger.Info(fmt.Sprintf("encoded G1 power %d", p)) | ||||
| 	} | ||||
| 	for p, s := range kzg.CeremonyBLS48581G2 { | ||||
| 		transcript.G2Powers = append( | ||||
| 			transcript.G2Powers, | ||||
| 			&protobufs.BLS48581G2PublicKey{ | ||||
| 				KeyValue: s.ToAffineCompressed(), | ||||
| 			}, | ||||
| 		) | ||||
| 		logger.Info(fmt.Sprintf("encoded G2 power %d", p)) | ||||
| 	} | ||||
| 
 | ||||
| 	transcript.RunningG1_256Witnesses = append( | ||||
| 		transcript.RunningG1_256Witnesses, | ||||
| 		&protobufs.BLS48581G1PublicKey{ | ||||
| 			KeyValue: kzg.CeremonyRunningProducts[0].ToAffineCompressed(), | ||||
| 		}, | ||||
| 	) | ||||
| 
 | ||||
| 	transcript.RunningG2_256Powers = append( | ||||
| 		transcript.RunningG2_256Powers, | ||||
| 		&protobufs.BLS48581G2PublicKey{ | ||||
| 			KeyValue: kzg.CeremonyBLS48581G2[len(kzg.CeremonyBLS48581G2)-1]. | ||||
| 				ToAffineCompressed(), | ||||
| 		}, | ||||
| 	) | ||||
| 
 | ||||
| 	outputProof := &protobufs.CeremonyLobbyStateTransition{ | ||||
| 		TypeUrls:         []string{}, | ||||
| 		TransitionInputs: [][]byte{}, | ||||
| 	} | ||||
| 
 | ||||
| 	proofBytes, err := proto.Marshal(outputProof) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("encoded transcript") | ||||
| 	logger.Info("encoding ceremony signatories into application state") | ||||
| 
 | ||||
| 	rewardTrie := &tries.RewardCritbitTrie{} | ||||
| 	for _, s := range kzg.CeremonySignatories { | ||||
| 		pubkey := s.ToAffineCompressed() | ||||
| 
 | ||||
| 		addr, err := poseidon.HashBytes(pubkey) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		addrBytes := addr.Bytes() | ||||
| 		addrBytes = append(make([]byte, 32-len(addrBytes)), addrBytes...) | ||||
| 		rewardTrie.Add(addrBytes, 0, 50) | ||||
| 	} | ||||
| 
 | ||||
| 	// 2024-01-03: 1.2.0
 | ||||
| 	d, err := os.ReadFile("./retroactive_peers.json") | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	type peerData struct { | ||||
| 		PeerId       string `json:"peer_id"` | ||||
| 		TokenBalance uint64 `json:"token_balance"` | ||||
| 	} | ||||
| 	type rewards struct { | ||||
| 		Rewards []peerData `json:"rewards"` | ||||
| 	} | ||||
| 
 | ||||
| 	retroEntries := &rewards{} | ||||
| 	err = json.Unmarshal(d, retroEntries) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("adding retroactive peer reward info") | ||||
| 	for _, s := range retroEntries.Rewards { | ||||
| 		peerId := s.PeerId | ||||
| 		peerBytes, err := base64.StdEncoding.DecodeString(peerId) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		addr, err := poseidon.HashBytes(peerBytes) | ||||
| 		if err != nil { | ||||
| 			panic(err) | ||||
| 		} | ||||
| 
 | ||||
| 		addrBytes := addr.Bytes() | ||||
| 		addrBytes = append(make([]byte, 32-len(addrBytes)), addrBytes...) | ||||
| 		rewardTrie.Add(addrBytes, 0, s.TokenBalance) | ||||
| 	} | ||||
| 
 | ||||
| 	trieBytes, err := rewardTrie.Serialize() | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	ceremonyLobbyState := &protobufs.CeremonyLobbyState{ | ||||
| 		LobbyState: 0, | ||||
| 		CeremonyState: &protobufs.CeremonyLobbyState_CeremonyOpenState{ | ||||
| 			CeremonyOpenState: &protobufs.CeremonyOpenState{ | ||||
| 				JoinedParticipants:    []*protobufs.CeremonyLobbyJoin{}, | ||||
| 				PreferredParticipants: []*protobufs.Ed448PublicKey{}, | ||||
| 			}, | ||||
| 		}, | ||||
| 		LatestTranscript: transcript, | ||||
| 		RewardTrie:       trieBytes, | ||||
| 	} | ||||
| 	outputBytes, err := proto.Marshal(ceremonyLobbyState) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	executionOutput := &protobufs.IntrinsicExecutionOutput{ | ||||
| 		Address: application.CEREMONY_ADDRESS, | ||||
| 		Output:  outputBytes, | ||||
| 		Proof:   proofBytes, | ||||
| 	} | ||||
| 
 | ||||
| 	data, err := proto.Marshal(executionOutput) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("encoded execution output") | ||||
| 
 | ||||
| 	digest := sha3.NewShake256() | ||||
| 	_, err = digest.Write(data) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	expand := make([]byte, 1024) | ||||
| 	_, err = digest.Read(expand) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("proving execution output for inclusion") | ||||
| 	commitment, err := inclusionProver.Commit( | ||||
| 		expand, | ||||
| 		protobufs.IntrinsicExecutionOutputType, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("creating kzg proof") | ||||
| 	proof, err := inclusionProver.ProveAggregate( | ||||
| 		[]*qcrypto.InclusionCommitment{ | ||||
| 			commitment, | ||||
| 		}, | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		panic(err) | ||||
| 	} | ||||
| 
 | ||||
| 	logger.Info("finalizing execution proof") | ||||
| 
 | ||||
| 	return inputMessage, proof, proverKeys | ||||
| } | ||||
| 
 | ||||
| // Start implements ExecutionEngine
 | ||||
| func (e *CeremonyExecutionEngine) Start() <-chan error { | ||||
| 	errChan := make(chan error) | ||||
| 
 | ||||
| 	e.logger.Info("ceremony data loaded", zap.Binary( | ||||
| 		"g2_power", | ||||
| 		qcrypto.CeremonyBLS48581G2[1].ToAffineCompressed(), | ||||
| 		kzg.CeremonyBLS48581G2[1].ToAffineCompressed(), | ||||
| 	)) | ||||
| 
 | ||||
| 	go func() { | ||||
| @ -203,7 +468,7 @@ func (e *CeremonyExecutionEngine) ProcessMessage( | ||||
| 				return nil, nil | ||||
| 			} | ||||
| 
 | ||||
| 			if err := frame.VerifyDataClockFrame(); err != nil { | ||||
| 			if err := e.frameProver.VerifyDataClockFrame(frame); err != nil { | ||||
| 				return nil, errors.Wrap(err, "process message") | ||||
| 			} | ||||
| 
 | ||||
| @ -221,7 +486,6 @@ func (e *CeremonyExecutionEngine) ProcessMessage( | ||||
| 		case protobufs.CeremonyTranscriptShareType: | ||||
| 			fallthrough | ||||
| 		case protobufs.CeremonyTranscriptType: | ||||
| 			frame := e.activeClockFrame | ||||
| 			hash := sha3.Sum256(any.Value) | ||||
| 			if any.TypeUrl == protobufs.CeremonyTranscriptType { | ||||
| 				e.seenMessageMx.Lock() | ||||
| @ -237,10 +501,6 @@ func (e *CeremonyExecutionEngine) ProcessMessage( | ||||
| 				e.seenMessageMx.Unlock() | ||||
| 			} | ||||
| 			if e.clock.IsInProverTrie(e.proverPublicKey) { | ||||
| 				app, err := application.MaterializeApplicationFromFrame(frame) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "process message") | ||||
| 				} | ||||
| 				proposedTransition := &protobufs.CeremonyLobbyStateTransition{ | ||||
| 					TypeUrls: []string{any.TypeUrl}, | ||||
| 					TransitionInputs: [][]byte{ | ||||
| @ -248,11 +508,6 @@ func (e *CeremonyExecutionEngine) ProcessMessage( | ||||
| 					}, | ||||
| 				} | ||||
| 
 | ||||
| 				_, err = app.ApplyTransition(frame.FrameNumber, proposedTransition) | ||||
| 				if err != nil { | ||||
| 					return nil, errors.Wrap(err, "process message") | ||||
| 				} | ||||
| 
 | ||||
| 				any := &anypb.Any{} | ||||
| 				if err := any.MarshalFrom(proposedTransition); err != nil { | ||||
| 					return nil, errors.Wrap(err, "process message") | ||||
| @ -293,14 +548,13 @@ func (e *CeremonyExecutionEngine) ProcessMessage( | ||||
| func (e *CeremonyExecutionEngine) RunWorker() { | ||||
| 	frameChan := e.clock.GetFrameChannel() | ||||
| 	for { | ||||
| 		frameFromBuffer := <-frameChan | ||||
| 		frame := e.clock.GetFrame() | ||||
| 		frame := <-frameChan | ||||
| 		e.activeClockFrame = frame | ||||
| 		e.logger.Info( | ||||
| 			"evaluating next frame", | ||||
| 			zap.Int( | ||||
| 				"last_run_took_frames", | ||||
| 				int(frame.FrameNumber)-int(frameFromBuffer.FrameNumber), | ||||
| 			zap.Uint64( | ||||
| 				"frame_number", | ||||
| 				frame.FrameNumber, | ||||
| 			), | ||||
| 		) | ||||
| 		app, err := application.MaterializeApplicationFromFrame(frame) | ||||
| @ -519,9 +773,7 @@ func (e *CeremonyExecutionEngine) RunWorker() { | ||||
| 			} | ||||
| 		case application.CEREMONY_APPLICATION_STATE_VALIDATING: | ||||
| 			e.logger.Info("round contribution validating") | ||||
| 			// Do a best effort to clear – Go's GC is noisy and unenforceable, but
 | ||||
| 			// this should at least mark it as dead space
 | ||||
| 			e.activeSecrets = []curves.Scalar{} | ||||
| 			e.alreadyPublishedShare = false | ||||
| 			for _, c := range e.peerChannels { | ||||
| 				c.Close() | ||||
| 			} | ||||
| @ -1134,7 +1386,7 @@ func (e *CeremonyExecutionEngine) VerifyExecution( | ||||
| 						return errors.Wrap(err, "verify execution") | ||||
| 					} | ||||
| 
 | ||||
| 					a, err = a.ApplyTransition(frame.FrameNumber, transition) | ||||
| 					a, _, _, err = a.ApplyTransition(frame.FrameNumber, transition, false) | ||||
| 					if err != nil { | ||||
| 						return errors.Wrap(err, "verify execution") | ||||
| 					} | ||||
							
								
								
									
										21
									
								
								node/go.mod
									
									
									
									
									
								
							
							
						
						
									
										21
									
								
								node/go.mod
									
									
									
									
									
								
							| @ -25,6 +25,11 @@ require ( | ||||
| 	source.quilibrium.com/quilibrium/monorepo/nekryptology v0.0.0-00010101000000-000000000000 | ||||
| ) | ||||
| 
 | ||||
| require ( | ||||
| 	go.opentelemetry.io/otel v1.14.0 // indirect | ||||
| 	go.opentelemetry.io/otel/trace v1.14.0 // indirect | ||||
| ) | ||||
| 
 | ||||
| require ( | ||||
| 	github.com/DataDog/zstd v1.4.5 // indirect | ||||
| 	github.com/aymanbagabas/go-osc52/v2 v2.0.1 // indirect | ||||
| @ -59,7 +64,7 @@ require ( | ||||
| 	github.com/quic-go/qtls-go1-19 v0.3.3 // indirect | ||||
| 	github.com/quic-go/qtls-go1-20 v0.2.3 // indirect | ||||
| 	github.com/rivo/uniseg v0.2.0 // indirect | ||||
| 	golang.org/x/term v0.14.0 | ||||
| 	golang.org/x/term v0.16.0 | ||||
| 	google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d // indirect | ||||
| 	google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect | ||||
| 	gopkg.in/yaml.v3 v3.0.1 // indirect | ||||
| @ -155,19 +160,17 @@ require ( | ||||
| 	github.com/stretchr/testify v1.8.4 | ||||
| 	github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 // indirect | ||||
| 	go.opencensus.io v0.24.0 // indirect | ||||
| 	go.opentelemetry.io/otel v1.14.0 // indirect | ||||
| 	go.opentelemetry.io/otel/trace v1.14.0 // indirect | ||||
| 	go.uber.org/dig v1.17.0 // indirect | ||||
| 	go.uber.org/fx v1.20.0 // indirect | ||||
| 	go.uber.org/multierr v1.11.0 // indirect | ||||
| 	go.uber.org/zap v1.25.0 | ||||
| 	golang.org/x/crypto v0.13.0 | ||||
| 	go.uber.org/zap v1.26.0 | ||||
| 	golang.org/x/crypto v0.18.0 | ||||
| 	golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 // indirect | ||||
| 	golang.org/x/mod v0.12.0 // indirect | ||||
| 	golang.org/x/net v0.15.0 // indirect | ||||
| 	golang.org/x/sync v0.3.0 | ||||
| 	golang.org/x/sys v0.14.0 // indirect | ||||
| 	golang.org/x/text v0.13.0 // indirect | ||||
| 	golang.org/x/net v0.20.0 // indirect | ||||
| 	golang.org/x/sync v0.6.0 | ||||
| 	golang.org/x/sys v0.16.0 // indirect | ||||
| 	golang.org/x/text v0.14.0 // indirect | ||||
| 	golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846 // indirect | ||||
| 	gonum.org/v1/gonum v0.11.0 // indirect | ||||
| 	google.golang.org/grpc v1.58.2 | ||||
|  | ||||
							
								
								
									
										28
									
								
								node/go.sum
									
									
									
									
									
								
							
							
						
						
									
										28
									
								
								node/go.sum
									
									
									
									
									
								
							| @ -494,8 +494,8 @@ go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN8 | ||||
| go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= | ||||
| go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= | ||||
| go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= | ||||
| go.uber.org/zap v1.25.0 h1:4Hvk6GtkucQ790dqmj7l1eEnRdKm3k3ZUrUMS2d5+5c= | ||||
| go.uber.org/zap v1.25.0/go.mod h1:JIAUzQIH94IC4fOJQm7gMmBJP5k7wQfdcnYdPoEXJYk= | ||||
| go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= | ||||
| go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= | ||||
| go4.org v0.0.0-20180809161055-417644f6feb5/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= | ||||
| golang.org/x/build v0.0.0-20190111050920-041ab4dc3f9d/go.mod h1:OWs+y06UdEOHN4y+MfF/py+xQ/tYqIWW03b70/CG9Rw= | ||||
| golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= | ||||
| @ -510,8 +510,8 @@ golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37/go.mod h1:LzIPMQfyMNhhGPh | ||||
| golang.org/x/crypto v0.0.0-20200602180216-279210d13fed/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= | ||||
| golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= | ||||
| golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= | ||||
| golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= | ||||
| golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= | ||||
| golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= | ||||
| golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= | ||||
| golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= | ||||
| golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 h1:m64FZMko/V45gv0bNmrNYoDEq8U5YUhetc9cBWKS1TQ= | ||||
| golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63/go.mod h1:0v4NqG35kSWCMzLaMeX+IQrlSnVE/bqGSyC2cz/9Le8= | ||||
| @ -546,8 +546,8 @@ golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwY | ||||
| golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= | ||||
| golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= | ||||
| golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= | ||||
| golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= | ||||
| golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= | ||||
| golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= | ||||
| golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= | ||||
| golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= | ||||
| golang.org/x/oauth2 v0.0.0-20181017192945-9dcd33a902f4/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= | ||||
| golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= | ||||
| @ -561,8 +561,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ | ||||
| golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | ||||
| golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | ||||
| golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= | ||||
| golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= | ||||
| golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= | ||||
| golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= | ||||
| golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= | ||||
| golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= | ||||
| golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= | ||||
| golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= | ||||
| @ -583,16 +583,16 @@ golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBc | ||||
| golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | ||||
| golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | ||||
| golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= | ||||
| golang.org/x/sys v0.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= | ||||
| golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= | ||||
| golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= | ||||
| golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= | ||||
| golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= | ||||
| golang.org/x/term v0.14.0 h1:LGK9IlZ8T9jvdy6cTdfKUCltatMFOehAQo9SRC46UQ8= | ||||
| golang.org/x/term v0.14.0/go.mod h1:TySc+nGkYR6qt8km8wUhuFRTVSMIX3XPR58y2lC8vww= | ||||
| golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= | ||||
| golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= | ||||
| golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= | ||||
| golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= | ||||
| golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= | ||||
| golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k= | ||||
| golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= | ||||
| golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= | ||||
| golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= | ||||
| golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= | ||||
| golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= | ||||
| golang.org/x/tools v0.0.0-20180828015842-6cd1fcedba52/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= | ||||
|  | ||||
| @ -18,8 +18,8 @@ import ( | ||||
| 	"github.com/pkg/errors" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/app" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/config" | ||||
| 	qcrypto "source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/kzg" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/rpc" | ||||
| ) | ||||
| 
 | ||||
| @ -131,7 +131,7 @@ func main() { | ||||
| 	} | ||||
| 
 | ||||
| 	fmt.Println("Loading ceremony state and starting node...") | ||||
| 	qcrypto.Init() | ||||
| 	kzg.Init() | ||||
| 
 | ||||
| 	node, err := app.NewNode(nodeConfig) | ||||
| 	if err != nil { | ||||
| @ -283,5 +283,5 @@ func printLogo() { | ||||
| 
 | ||||
| func printVersion() { | ||||
| 	fmt.Println(" ") | ||||
| 	fmt.Println("                         Quilibrium Node - v1.2.3 – Dawn") | ||||
| 	fmt.Println("                         Quilibrium Node - v1.2.4 – Dawn") | ||||
| } | ||||
|  | ||||
| @ -10,7 +10,7 @@ import ( | ||||
| 	"google.golang.org/protobuf/proto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/go-libp2p-blossomsub/pb" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/core/curves" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/crypto/channel" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
| ) | ||||
| @ -18,7 +18,7 @@ import ( | ||||
| // A simplified P2P channel – the pair of actors communicating is public
 | ||||
| // knowledge, even though the data itself is encrypted.
 | ||||
| type PublicP2PChannel struct { | ||||
| 	participant         *crypto.DoubleRatchetParticipant | ||||
| 	participant         *channel.DoubleRatchetParticipant | ||||
| 	sendMap             map[uint64][]byte | ||||
| 	receiveMap          map[uint64][]byte | ||||
| 	pubSub              PubSub | ||||
| @ -58,7 +58,7 @@ func NewPublicP2PChannel( | ||||
| 		senderIdentifier..., | ||||
| 	) | ||||
| 
 | ||||
| 	channel := &PublicP2PChannel{ | ||||
| 	ch := &PublicP2PChannel{ | ||||
| 		publicChannelClient: publicChannelClient, | ||||
| 		sendMap:             map[uint64][]byte{}, | ||||
| 		receiveMap:          map[uint64][]byte{}, | ||||
| @ -72,19 +72,19 @@ func NewPublicP2PChannel( | ||||
| 	} | ||||
| 
 | ||||
| 	var err error | ||||
| 	var participant *crypto.DoubleRatchetParticipant | ||||
| 	var participant *channel.DoubleRatchetParticipant | ||||
| 	if initiator { | ||||
| 		sendingEphemeralPrivateKey := curve.Scalar.Random( | ||||
| 			rand.Reader, | ||||
| 		) | ||||
| 		x3dh := crypto.SenderX3DH( | ||||
| 		x3dh := channel.SenderX3DH( | ||||
| 			sendingIdentityPrivateKey, | ||||
| 			sendingSignedPrePrivateKey, | ||||
| 			receivingIdentityKey, | ||||
| 			receivingSignedPreKey, | ||||
| 			96, | ||||
| 		) | ||||
| 		participant, err = crypto.NewDoubleRatchetParticipant( | ||||
| 		participant, err = channel.NewDoubleRatchetParticipant( | ||||
| 			x3dh[:32], | ||||
| 			x3dh[32:64], | ||||
| 			x3dh[64:], | ||||
| @ -98,14 +98,14 @@ func NewPublicP2PChannel( | ||||
| 			return nil, errors.Wrap(err, "new public p2p channel") | ||||
| 		} | ||||
| 	} else { | ||||
| 		x3dh := crypto.SenderX3DH( | ||||
| 		x3dh := channel.SenderX3DH( | ||||
| 			sendingIdentityPrivateKey, | ||||
| 			sendingSignedPrePrivateKey, | ||||
| 			receivingIdentityKey, | ||||
| 			receivingSignedPreKey, | ||||
| 			96, | ||||
| 		) | ||||
| 		participant, err = crypto.NewDoubleRatchetParticipant( | ||||
| 		participant, err = channel.NewDoubleRatchetParticipant( | ||||
| 			x3dh[:32], | ||||
| 			x3dh[32:64], | ||||
| 			x3dh[64:], | ||||
| @ -120,7 +120,7 @@ func NewPublicP2PChannel( | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	channel.participant = participant | ||||
| 	ch.participant = participant | ||||
| 	if publicChannelClient == nil { | ||||
| 		pubSub.Subscribe( | ||||
| 			sendFilter, | ||||
| @ -130,12 +130,12 @@ func NewPublicP2PChannel( | ||||
| 
 | ||||
| 		pubSub.Subscribe( | ||||
| 			receiveFilter, | ||||
| 			channel.handleReceive, | ||||
| 			ch.handleReceive, | ||||
| 			true, | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return channel, nil | ||||
| 	return ch, nil | ||||
| } | ||||
| 
 | ||||
| func (c *PublicP2PChannel) handleReceive(message *pb.Message) error { | ||||
|  | ||||
| @ -1,130 +1,13 @@ | ||||
| package protobufs | ||||
| 
 | ||||
| import ( | ||||
| 	"crypto" | ||||
| 	"crypto/rand" | ||||
| 	"encoding/binary" | ||||
| 	"math/big" | ||||
| 	"time" | ||||
| 
 | ||||
| 	"github.com/cloudflare/circl/sign/ed448" | ||||
| 	"github.com/iden3/go-iden3-crypto/ff" | ||||
| 	"github.com/iden3/go-iden3-crypto/poseidon" | ||||
| 	"github.com/pkg/errors" | ||||
| 	"golang.org/x/crypto/sha3" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/nekryptology/pkg/vdf" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| ) | ||||
| 
 | ||||
| func ProveMasterClockFrame( | ||||
| 	previousFrame *ClockFrame, | ||||
| 	difficulty uint32, | ||||
| ) (*ClockFrame, error) { | ||||
| 	input := []byte{} | ||||
| 	input = append(input, previousFrame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, previousFrame.FrameNumber+1) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	input = append(input, previousFrame.Output[:]...) | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 
 | ||||
| 	timestamp := time.Now().UnixMilli() | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], previousFrame.Output[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	frame := &ClockFrame{ | ||||
| 		Filter:          previousFrame.Filter, | ||||
| 		FrameNumber:     previousFrame.FrameNumber + 1, | ||||
| 		Timestamp:       timestamp, | ||||
| 		Difficulty:      difficulty, | ||||
| 		ParentSelector:  parent.Bytes(), | ||||
| 		Input:           previousFrame.Output, | ||||
| 		AggregateProofs: []*InclusionAggregateProof{}, | ||||
| 		Output:          o[:], | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (frame *ClockFrame) VerifyMasterClockFrame() error { | ||||
| 	input := []byte{} | ||||
| 	input = append(input, frame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, frame.FrameNumber) | ||||
| 	input = binary.BigEndian.AppendUint32(input, frame.Difficulty) | ||||
| 	input = append(input, frame.Input...) | ||||
| 
 | ||||
| 	if len(frame.Input) < 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if len(frame.AggregateProofs) > 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if frame.PublicKeySignature != nil { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if len(frame.Input) != 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(frame.Difficulty, b) | ||||
| 	proof := [516]byte{} | ||||
| 	copy(proof[:], frame.Output) | ||||
| 
 | ||||
| 	if !v.Verify(proof) { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid proof"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], frame.Input[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "verify clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	selector := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 	if parent.Cmp(selector) != 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("selector did not match input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (frame *ClockFrame) GetParentSelectorAndDistance( | ||||
| 	discriminator *big.Int, | ||||
| ) ( | ||||
| 	*big.Int, | ||||
| func (frame *ClockFrame) GetParentAndSelector() ( | ||||
| 	*big.Int, | ||||
| 	*big.Int, | ||||
| 	error, | ||||
| @ -134,32 +17,16 @@ func (frame *ClockFrame) GetParentSelectorAndDistance( | ||||
| 
 | ||||
| 	selector, err := poseidon.HashBytes(outputBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return nil, nil, nil, errors.Wrap(err, "get parent selector and distance") | ||||
| 		return nil, nil, errors.Wrap(err, "get parent selector and distance") | ||||
| 	} | ||||
| 
 | ||||
| 	if frame.FrameNumber == 0 { | ||||
| 		return big.NewInt(0), big.NewInt(0), selector, nil | ||||
| 		return big.NewInt(0), selector, nil | ||||
| 	} | ||||
| 
 | ||||
| 	parentSelector := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 
 | ||||
| 	var distance *big.Int | ||||
| 	if discriminator != nil { | ||||
| 		l := new(big.Int).Mod( | ||||
| 			new(big.Int).Sub(selector, discriminator), | ||||
| 			ff.Modulus(), | ||||
| 		) | ||||
| 		r := new(big.Int).Mod( | ||||
| 			new(big.Int).Sub(discriminator, selector), | ||||
| 			ff.Modulus(), | ||||
| 		) | ||||
| 		distance = r | ||||
| 		if l.Cmp(r) == 1 { | ||||
| 			distance = l | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	return parentSelector, distance, selector, nil | ||||
| 	return parentSelector, selector, nil | ||||
| } | ||||
| 
 | ||||
| func (frame *ClockFrame) GetSelector() (*big.Int, error) { | ||||
| @ -216,196 +83,3 @@ func (frame *ClockFrame) GetAddress() ([]byte, error) { | ||||
| 
 | ||||
| 	return addressBytes, nil | ||||
| } | ||||
| 
 | ||||
| func ProveDataClockFrame( | ||||
| 	previousFrame *ClockFrame, | ||||
| 	commitments [][]byte, | ||||
| 	aggregateProofs []*InclusionAggregateProof, | ||||
| 	provingKey crypto.Signer, | ||||
| 	difficulty uint32, | ||||
| ) (*ClockFrame, error) { | ||||
| 	var pubkey []byte | ||||
| 	pubkeyType := keys.KeyTypeEd448 | ||||
| 	ed448PublicKey, ok := provingKey.Public().(ed448.PublicKey) | ||||
| 	if ok { | ||||
| 		pubkey = []byte(ed448PublicKey) | ||||
| 	} else { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("no valid signature provided"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	h, err := poseidon.HashBytes(pubkey) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("could not hash proving key"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	address := h.Bytes() | ||||
| 	timestamp := time.Now().UnixMilli() | ||||
| 	input := []byte{} | ||||
| 	input = append(input, previousFrame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, previousFrame.FrameNumber+1) | ||||
| 	input = binary.BigEndian.AppendUint64(input, uint64(timestamp)) | ||||
| 	input = binary.BigEndian.AppendUint32(input, difficulty) | ||||
| 	input = append(input, address...) | ||||
| 	input = append(input, previousFrame.Output[:]...) | ||||
| 
 | ||||
| 	commitmentInput := []byte{} | ||||
| 	for _, commitment := range commitments { | ||||
| 		commitmentInput = append(commitmentInput, commitment...) | ||||
| 	} | ||||
| 
 | ||||
| 	input = append(input, commitmentInput...) | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(difficulty, b) | ||||
| 
 | ||||
| 	v.Execute() | ||||
| 	o := v.GetOutput() | ||||
| 
 | ||||
| 	// TODO: make this configurable for signing algorithms that allow
 | ||||
| 	// user-supplied hash functions
 | ||||
| 	signature, err := provingKey.Sign( | ||||
| 		rand.Reader, | ||||
| 		append(append([]byte{}, b[:]...), o[:]...), | ||||
| 		crypto.Hash(0), | ||||
| 	) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			err, | ||||
| 			"prove", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], previousFrame.Output[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return nil, errors.Wrap(err, "prove clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	frame := &ClockFrame{ | ||||
| 		Filter:         previousFrame.Filter, | ||||
| 		FrameNumber:    previousFrame.FrameNumber + 1, | ||||
| 		Timestamp:      timestamp, | ||||
| 		Difficulty:     difficulty, | ||||
| 		ParentSelector: parent.Bytes(), | ||||
| 		Input: append( | ||||
| 			append([]byte{}, previousFrame.Output...), | ||||
| 			commitmentInput..., | ||||
| 		), | ||||
| 		AggregateProofs: aggregateProofs, | ||||
| 		Output:          o[:], | ||||
| 	} | ||||
| 
 | ||||
| 	switch pubkeyType { | ||||
| 	case keys.KeyTypeEd448: | ||||
| 		frame.PublicKeySignature = &ClockFrame_PublicKeySignatureEd448{ | ||||
| 			PublicKeySignatureEd448: &Ed448Signature{ | ||||
| 				Signature: signature, | ||||
| 				PublicKey: &Ed448PublicKey{ | ||||
| 					KeyValue: pubkey, | ||||
| 				}, | ||||
| 			}, | ||||
| 		} | ||||
| 	default: | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.New("unsupported proving key"), | ||||
| 			"prove clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| func (frame *ClockFrame) VerifyDataClockFrame() error { | ||||
| 	var pubkey []byte | ||||
| 	var signature []byte | ||||
| 	pubkeyType := keys.KeyTypeEd448 | ||||
| 	ed448PublicKey := frame.GetPublicKeySignatureEd448() | ||||
| 	if ed448PublicKey != nil { | ||||
| 		pubkey = ed448PublicKey.PublicKey.KeyValue | ||||
| 		signature = ed448PublicKey.Signature | ||||
| 	} else { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("no valid signature provided"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	h, err := poseidon.HashBytes(pubkey) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("could not hash proving key"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	address := h.Bytes() | ||||
| 
 | ||||
| 	input := []byte{} | ||||
| 	input = append(input, frame.Filter...) | ||||
| 	input = binary.BigEndian.AppendUint64(input, frame.FrameNumber) | ||||
| 	input = binary.BigEndian.AppendUint64(input, uint64(frame.Timestamp)) | ||||
| 	input = binary.BigEndian.AppendUint32(input, frame.Difficulty) | ||||
| 	input = append(input, address...) | ||||
| 	input = append(input, frame.Input...) | ||||
| 
 | ||||
| 	if len(frame.Input) < 516 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	b := sha3.Sum256(input) | ||||
| 	v := vdf.New(frame.Difficulty, b) | ||||
| 	proof := [516]byte{} | ||||
| 	copy(proof[:], frame.Output) | ||||
| 
 | ||||
| 	// TODO: make this configurable for signing algorithms that allow
 | ||||
| 	// user-supplied hash functions
 | ||||
| 	switch pubkeyType { | ||||
| 	case keys.KeyTypeEd448: | ||||
| 		if len(pubkey) != 57 || len(signature) != 114 || !ed448.VerifyAny( | ||||
| 			pubkey, | ||||
| 			append(append([]byte{}, b[:]...), frame.Output...), | ||||
| 			signature, | ||||
| 			crypto.Hash(0), | ||||
| 		) { | ||||
| 			return errors.Wrap( | ||||
| 				errors.New("invalid signature for issuer"), | ||||
| 				"verify clock frame", | ||||
| 			) | ||||
| 		} | ||||
| 	} | ||||
| 	if !v.Verify(proof) { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("invalid proof"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	previousSelectorBytes := [516]byte{} | ||||
| 	copy(previousSelectorBytes[:], frame.Input[:516]) | ||||
| 
 | ||||
| 	parent, err := poseidon.HashBytes(previousSelectorBytes[:]) | ||||
| 	if err != nil { | ||||
| 		return errors.Wrap(err, "verify clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	selector := new(big.Int).SetBytes(frame.ParentSelector) | ||||
| 	if parent.Cmp(selector) != 0 { | ||||
| 		return errors.Wrap( | ||||
| 			errors.New("selector did not match input"), | ||||
| 			"verify clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return nil | ||||
| } | ||||
|  | ||||
| @ -17,7 +17,7 @@ import ( | ||||
| 	"google.golang.org/grpc/credentials/insecure" | ||||
| 	"google.golang.org/grpc/reflection" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/execution/intrinsics/ceremony/application" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/keys" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/p2p" | ||||
| 	"source.quilibrium.com/quilibrium/monorepo/node/protobufs" | ||||
|  | ||||
| @ -4,8 +4,6 @@ import ( | ||||
| 	"bytes" | ||||
| 	"encoding/binary" | ||||
| 	"fmt" | ||||
| 	"math/big" | ||||
| 	"sort" | ||||
| 
 | ||||
| 	"github.com/cockroachdb/pebble" | ||||
| 	"github.com/iden3/go-iden3-crypto/poseidon" | ||||
| @ -60,6 +58,12 @@ type ClockStore interface { | ||||
| 		frame *protobufs.ClockFrame, | ||||
| 		txn Transaction, | ||||
| 	) error | ||||
| 	GetCandidateDataClockFrame( | ||||
| 		filter []byte, | ||||
| 		frameNumber uint64, | ||||
| 		parentSelector []byte, | ||||
| 		distance []byte, | ||||
| 	) (*protobufs.ClockFrame, error) | ||||
| 	GetCandidateDataClockFrames( | ||||
| 		filter []byte, | ||||
| 		frameNumber uint64, | ||||
| @ -1055,6 +1059,45 @@ func (p *PebbleClockStore) PutDataClockFrame( | ||||
| 	return nil | ||||
| } | ||||
| 
 | ||||
| func (p *PebbleClockStore) GetCandidateDataClockFrame( | ||||
| 	filter []byte, | ||||
| 	frameNumber uint64, | ||||
| 	parentSelector []byte, | ||||
| 	distance []byte, | ||||
| ) (*protobufs.ClockFrame, error) { | ||||
| 	value, closer, err := p.db.Get(clockDataCandidateFrameKey( | ||||
| 		filter, | ||||
| 		frameNumber, | ||||
| 		parentSelector, | ||||
| 		distance, | ||||
| 	)) | ||||
| 	if err != nil { | ||||
| 		if errors.Is(err, pebble.ErrNotFound) { | ||||
| 			return nil, ErrNotFound | ||||
| 		} | ||||
| 
 | ||||
| 		return nil, errors.Wrap(err, "get candidate data clock frame") | ||||
| 	} | ||||
| 
 | ||||
| 	defer closer.Close() | ||||
| 	frame := &protobufs.ClockFrame{} | ||||
| 	if err := proto.Unmarshal(value, frame); err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.Wrap(err, ErrInvalidData.Error()), | ||||
| 			"get candidate data clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	if err = p.fillAggregateProofs(frame); err != nil { | ||||
| 		return nil, errors.Wrap( | ||||
| 			errors.Wrap(err, ErrInvalidData.Error()), | ||||
| 			"get candidate data clock frame", | ||||
| 		) | ||||
| 	} | ||||
| 
 | ||||
| 	return frame, nil | ||||
| } | ||||
| 
 | ||||
| // GetCandidateDataClockFrames implements ClockStore.
 | ||||
| // Distance is 32-byte aligned, so we just use a 0x00 * 32 -> 0xff * 32 range
 | ||||
| func (p *PebbleClockStore) GetCandidateDataClockFrames( | ||||
| @ -1421,162 +1464,6 @@ func (p *PebbleClockStore) GetCompressedDataClockFrames( | ||||
| 		return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 	} | ||||
| 
 | ||||
| 	if len(syncMessage.TruncatedClockFrames) < int( | ||||
| 		toFrameNumber-fromFrameNumber+1, | ||||
| 	) { | ||||
| 		newFrom := fromFrameNumber | ||||
| 		if len(syncMessage.TruncatedClockFrames) > 0 { | ||||
| 			newFrom = syncMessage.TruncatedClockFrames[len( | ||||
| 				syncMessage.TruncatedClockFrames, | ||||
| 			)-1].FrameNumber + 1 | ||||
| 		} | ||||
| 		from := clockDataCandidateFrameKey( | ||||
| 			filter, | ||||
| 			newFrom, | ||||
| 			[]byte{ | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			}, | ||||
| 			[]byte{ | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 				0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, | ||||
| 			}, | ||||
| 		) | ||||
| 		to := clockDataCandidateFrameKey( | ||||
| 			filter, | ||||
| 			toFrameNumber+1, | ||||
| 			[]byte{ | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 			}, | ||||
| 			[]byte{ | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 				0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 			}, | ||||
| 		) | ||||
| 
 | ||||
| 		iter, err := p.db.NewIter(from, to) | ||||
| 		if err != nil { | ||||
| 			return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 		} | ||||
| 
 | ||||
| 		candidates := []*protobufs.ClockFrame{} | ||||
| 		for iter.First(); iter.Valid(); iter.Next() { | ||||
| 			value := iter.Value() | ||||
| 			frame := &protobufs.ClockFrame{} | ||||
| 			if err := proto.Unmarshal(value, frame); err != nil { | ||||
| 				return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 			} | ||||
| 			candidates = append(candidates, frame) | ||||
| 		} | ||||
| 
 | ||||
| 		if err := iter.Close(); err != nil { | ||||
| 			return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 		} | ||||
| 
 | ||||
| 		sort.Slice(candidates, func(i, j int) bool { | ||||
| 			return candidates[i].FrameNumber < candidates[j].FrameNumber | ||||
| 		}) | ||||
| 
 | ||||
| 		if len(candidates) > 0 { | ||||
| 			cursorStart := candidates[0].FrameNumber | ||||
| 			paths := [][]*protobufs.ClockFrame{} | ||||
| 			for _, frame := range candidates { | ||||
| 				frame := frame | ||||
| 				if frame.FrameNumber == cursorStart { | ||||
| 					paths = append(paths, []*protobufs.ClockFrame{frame}) | ||||
| 				} | ||||
| 				if frame.FrameNumber > cursorStart { | ||||
| 					for i, path := range paths { | ||||
| 						s, err := path[len(path)-1].GetSelector() | ||||
| 						if err != nil { | ||||
| 							return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 						} | ||||
| 						parentSelector, _, _, err := frame.GetParentSelectorAndDistance(nil) | ||||
| 						if err != nil { | ||||
| 							return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 						} | ||||
| 						if s.Cmp(parentSelector) == 0 { | ||||
| 							paths[i] = append(paths[i], frame) | ||||
| 						} | ||||
| 					} | ||||
| 				} | ||||
| 			} | ||||
| 			sort.Slice(paths, func(i, j int) bool { | ||||
| 				return len(paths[i]) > len(paths[j]) | ||||
| 			}) | ||||
| 
 | ||||
| 			leadingIndex := 0 | ||||
| 			var leadingScore *big.Int | ||||
| 			length := len(paths[0]) | ||||
| 			for i := 0; i < len(paths); i++ { | ||||
| 				if len(paths[i]) < length { | ||||
| 					break | ||||
| 				} | ||||
| 				score := new(big.Int) | ||||
| 				for _, path := range paths[i] { | ||||
| 					master, err := p.GetMasterClockFrame( | ||||
| 						[]byte{ | ||||
| 							0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 							0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 							0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 							0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, | ||||
| 						}, | ||||
| 						path.FrameNumber, | ||||
| 					) | ||||
| 					if err != nil { | ||||
| 						return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 					} | ||||
| 
 | ||||
| 					discriminator, err := master.GetSelector() | ||||
| 					if err != nil { | ||||
| 						return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 					} | ||||
| 
 | ||||
| 					_, distance, _, err := path.GetParentSelectorAndDistance( | ||||
| 						discriminator, | ||||
| 					) | ||||
| 					if err != nil { | ||||
| 						return nil, errors.Wrap(err, "get compressed data clock frames") | ||||
| 					} | ||||
| 					score = score.Add(score, distance) | ||||
| 				} | ||||
| 				if leadingScore == nil || leadingScore.Cmp(score) > 0 { | ||||
| 					leadingIndex = i | ||||
| 					leadingScore = score | ||||
| 				} | ||||
| 			} | ||||
| 			for _, frame := range paths[leadingIndex] { | ||||
| 				frame := frame | ||||
| 				syncMessage.TruncatedClockFrames = append( | ||||
| 					syncMessage.TruncatedClockFrames, | ||||
| 					frame, | ||||
| 				) | ||||
| 				if frame.FrameNumber == 0 { | ||||
| 					continue | ||||
| 				} | ||||
| 
 | ||||
| 				for i := 0; i < len(frame.Input[516:])/74; i++ { | ||||
| 					aggregateCommit := frame.Input[516+(i*74) : 516+((i+1)*74)] | ||||
| 
 | ||||
| 					if _, ok := proofs[string(aggregateCommit)]; !ok { | ||||
| 						proofs[string(aggregateCommit)] = &protobufs.InclusionProofsMap{ | ||||
| 							FrameCommit: aggregateCommit, | ||||
| 						} | ||||
| 					} | ||||
| 				} | ||||
| 			} | ||||
| 		} | ||||
| 	} | ||||
| 
 | ||||
| 	for k, v := range proofs { | ||||
| 		k := k | ||||
| 		v := v | ||||
|  | ||||
		Loading…
	
		Reference in New Issue
	
	Block a user
	 Cassandra Heart
						Cassandra Heart