diff --git a/chain/auth_batch.go b/chain/auth_batch.go new file mode 100644 index 0000000000..016d1bf99c --- /dev/null +++ b/chain/auth_batch.go @@ -0,0 +1,89 @@ +// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package chain + +import ( + "github.com/ava-labs/hypersdk/workers" +) + +const authWorkerBacklog = 16_384 + +// Adding a signature to a verification batch +// may perform complex cryptographic operations. We should +// not block the caller when this happens and we should +// not require each batch package to re-implement this logic. +type AuthBatch struct { + vm VM + job *workers.Job + bvs map[uint8]*authBatchWorker +} + +func NewAuthBatch(vm VM, job *workers.Job, authTypes map[uint8]int) *AuthBatch { + bvs := map[uint8]*authBatchWorker{} + for t, count := range authTypes { + bv, ok := vm.GetAuthBatchVerifier(t, job.Workers(), count) + if !ok { + continue + } + bw := &authBatchWorker{ + vm, + job, + bv, + make(chan *authBatchObject, authWorkerBacklog), + make(chan struct{}), + } + go bw.start() + bvs[t] = bw + } + return &AuthBatch{vm, job, bvs} +} + +func (a *AuthBatch) Add(digest []byte, auth Auth) { + // If batch doesn't exist for auth, just add verify right to job and start + // processing. + bv, ok := a.bvs[auth.GetTypeID()] + if !ok { + a.job.Go(func() error { return auth.AsyncVerify(digest) }) + return + } + bv.items <- &authBatchObject{digest, auth} +} + +func (a *AuthBatch) Done(f func()) { + for _, bw := range a.bvs { + close(bw.items) + <-bw.done + + for _, item := range bw.bv.Done() { + a.job.Go(item) + a.vm.Logger().Debug("enqueued batch for processing during done") + } + } + a.job.Done(f) +} + +type authBatchObject struct { + digest []byte + auth Auth +} + +type authBatchWorker struct { + vm VM + job *workers.Job + bv AuthBatchVerifier + items chan *authBatchObject + done chan struct{} +} + +func (b *authBatchWorker) start() { + defer close(b.done) + + for object := range b.items { + if j := b.bv.Add(object.digest, object.auth); j != nil { + // May finish parts of batch early, let's start computing them as soon as possible + b.job.Go(j) + b.vm.Logger().Debug("enqueued batch for processing during add") + } + } +} diff --git a/chain/block.go b/chain/block.go index 7f6a43a91e..7d851b43a5 100644 --- a/chain/block.go +++ b/chain/block.go @@ -49,6 +49,10 @@ type StatefulBlock struct { WarpResults set.Bits64 `json:"warpResults"` size int + + // authCounts can be used by batch signature verification + // to preallocate memory + authCounts map[uint8]int } func (b *StatefulBlock) Size() int { @@ -147,23 +151,32 @@ func (b *StatelessBlock) populateTxs(ctx context.Context) error { defer span.End() // Setup signature verification job + _, sigVerifySpan := b.vm.Tracer().Start(ctx, "StatelessBlock.verifySignatures") job, err := b.vm.Workers().NewJob(len(b.Txs)) if err != nil { return err } b.sigJob = job + batchVerifier := NewAuthBatch(b.vm, b.sigJob, b.authCounts) - // Process transactions - _, sspan := b.vm.Tracer().Start(ctx, "StatelessBlock.verifySignatures") + // Confirm no transaction duplicates and setup + // AWM processing b.txsSet = set.NewSet[ids.ID](len(b.Txs)) b.warpMessages = map[ids.ID]*warpJob{} for _, tx := range b.Txs { - b.sigJob.Go(tx.AuthAsyncVerify()) + // Ensure there are no duplicate transactions if b.txsSet.Contains(tx.ID()) { return ErrDuplicateTx } b.txsSet.Add(tx.ID()) + // Verify signature async + txDigest, err := tx.Digest() + if err != nil { + return err + } + batchVerifier.Add(txDigest, tx.Auth) + // Check if we need the block context to verify the block (which contains // an Avalanche Warp Message) // @@ -187,7 +200,10 @@ func (b *StatelessBlock) populateTxs(ctx context.Context) error { b.containsWarp = true } } - b.sigJob.Done(func() { sspan.End() }) + + // BatchVerifier is given the responsibility to call [b.sigJob.Done()] because it may add things + // to the work queue async and that may not have completed by this point. + go batchVerifier.Done(func() { sigVerifySpan.End() }) return nil } @@ -453,10 +469,10 @@ func (b *StatelessBlock) innerVerify(ctx context.Context) (merkledb.TrieView, er ) return nil, ErrMissingBlockContext } - _, sspan := b.vm.Tracer().Start(ctx, "StatelessBlock.verifyWarpMessages") + _, warpVerifySpan := b.vm.Tracer().Start(ctx, "StatelessBlock.verifyWarpMessages") b.vdrState = b.vm.ValidatorState() go func() { - defer sspan.End() + defer warpVerifySpan.End() // We don't use [b.vm.Workers] here because we need the warp verification // results during normal execution. If we added a job to the workers queue, // it would get executed after all signatures. Additionally, BLS @@ -776,10 +792,12 @@ func (b *StatefulBlock) Marshal() ([]byte, error) { p.PackWindow(b.UnitWindow) p.PackInt(len(b.Txs)) + b.authCounts = map[uint8]int{} for _, tx := range b.Txs { if err := tx.Marshal(p); err != nil { return nil, err } + b.authCounts[tx.Auth.GetTypeID()]++ } p.PackID(b.StateRoot) @@ -815,14 +833,15 @@ func UnmarshalBlock(raw []byte, parser Parser) (*StatefulBlock, error) { txCount := p.UnpackInt(false) // can produce empty blocks actionRegistry, authRegistry := parser.Registry() b.Txs = []*Transaction{} // don't preallocate all to avoid DoS + b.authCounts = map[uint8]int{} for i := 0; i < txCount; i++ { tx, err := UnmarshalTx(p, actionRegistry, authRegistry) if err != nil { return nil, err } b.Txs = append(b.Txs, tx) + b.authCounts[tx.Auth.GetTypeID()]++ } - p.UnpackID(false, &b.StateRoot) b.UnitsConsumed = p.UnpackUint64(false) b.WarpResults = set.Bits64(p.UnpackUint64(false)) diff --git a/chain/dependencies.go b/chain/dependencies.go index fe4d62e5a4..532f95e70a 100644 --- a/chain/dependencies.go +++ b/chain/dependencies.go @@ -36,6 +36,10 @@ type VM interface { Tracer() trace.Tracer Logger() logging.Logger + // We don't include this in registry because it would never be used + // by any client of the hypersdk. + GetAuthBatchVerifier(authTypeID uint8, cores int, count int) (AuthBatchVerifier, bool) + IsBootstrapped() bool LastAcceptedBlock() *StatelessBlock SetLastAccepted(*StatelessBlock) error @@ -157,8 +161,14 @@ type Action interface { Marshal(p *codec.Packer) } +type AuthBatchVerifier interface { + Add([]byte, Auth) func() error + Done() []func() error +} + type Auth interface { GetTypeID() uint8 // identify uniquely the auth + MaxUnits(Rules) uint64 ValidRange(Rules) (start int64, end int64) // -1 means no start/end diff --git a/cli/dependencies.go b/cli/dependencies.go index 7b6a72ac50..a568ce225e 100644 --- a/cli/dependencies.go +++ b/cli/dependencies.go @@ -4,12 +4,12 @@ package cli import ( - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" ) type Controller interface { DatabasePath() string Symbol() string - Address(crypto.PublicKey) string - ParseAddress(string) (crypto.PublicKey, error) + Address(ed25519.PublicKey) string + ParseAddress(string) (ed25519.PublicKey, error) } diff --git a/cli/key.go b/cli/key.go index b12e1cb453..09f5714666 100644 --- a/cli/key.go +++ b/cli/key.go @@ -7,14 +7,14 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/rpc" "github.com/ava-labs/hypersdk/utils" ) func (h *Handler) GenerateKey() error { // TODO: encrypt key - priv, err := crypto.GeneratePrivateKey() + priv, err := ed25519.GeneratePrivateKey() if err != nil { return err } @@ -33,7 +33,7 @@ func (h *Handler) GenerateKey() error { } func (h *Handler) ImportKey(keyPath string) error { - priv, err := crypto.LoadKey(keyPath) + priv, err := ed25519.LoadKey(keyPath) if err != nil { return err } @@ -89,7 +89,7 @@ func (h *Handler) SetKey(lookupBalance func(int, string, string, uint32, ids.ID) return h.StoreDefaultKey(key.PublicKey()) } -func (h *Handler) Balance(checkAllChains bool, promptAsset bool, printBalance func(crypto.PublicKey, string, uint32, ids.ID, ids.ID) error) error { +func (h *Handler) Balance(checkAllChains bool, promptAsset bool, printBalance func(ed25519.PublicKey, string, uint32, ids.ID, ids.ID) error) error { priv, err := h.GetDefaultKey() if err != nil { return err diff --git a/cli/prompt.go b/cli/prompt.go index a743f23523..645afc4de4 100644 --- a/cli/prompt.go +++ b/cli/prompt.go @@ -10,12 +10,12 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/utils" "github.com/manifoldco/promptui" ) -func (h *Handler) PromptAddress(label string) (crypto.PublicKey, error) { +func (h *Handler) PromptAddress(label string) (ed25519.PublicKey, error) { promptText := promptui.Prompt{ Label: label, Validate: func(input string) error { @@ -28,7 +28,7 @@ func (h *Handler) PromptAddress(label string) (crypto.PublicKey, error) { } recipient, err := promptText.Run() if err != nil { - return crypto.EmptyPublicKey, err + return ed25519.EmptyPublicKey, err } recipient = strings.TrimSpace(recipient) return h.c.ParseAddress(recipient) diff --git a/cli/spam.go b/cli/spam.go index 3d407944a0..576a6193b5 100644 --- a/cli/spam.go +++ b/cli/spam.go @@ -21,7 +21,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/pubsub" "github.com/ava-labs/hypersdk/rpc" "github.com/ava-labs/hypersdk/utils" @@ -50,11 +50,11 @@ var ( func (h *Handler) Spam( maxTxBacklog int, randomRecipient bool, createClient func(string, uint32, ids.ID), // must save on caller side - getFactory func(crypto.PrivateKey) chain.AuthFactory, + getFactory func(ed25519.PrivateKey) chain.AuthFactory, lookupBalance func(int, string) (uint64, error), getParser func(context.Context, ids.ID) (chain.Parser, error), - getTransfer func(crypto.PublicKey, uint64) chain.Action, - submitDummy func(*rpc.JSONRPCClient, crypto.PrivateKey) func(context.Context, uint64) error, + getTransfer func(ed25519.PublicKey, uint64) chain.Action, + submitDummy func(*rpc.JSONRPCClient, ed25519.PrivateKey) func(context.Context, uint64) error, ) error { ctx := context.Background() @@ -121,12 +121,12 @@ func (h *Handler) Spam( h.ValueString(ids.Empty, distAmount), h.AssetString(ids.Empty), ) - accounts := make([]crypto.PrivateKey, numAccounts) + accounts := make([]ed25519.PrivateKey, numAccounts) dcli, err := rpc.NewWebSocketClient(uris[0], rpc.DefaultHandshakeTimeout, pubsub.MaxPendingMessages, pubsub.MaxReadMessageSize) // we write the max read if err != nil { return err } - funds := map[crypto.PublicKey]uint64{} + funds := map[ed25519.PublicKey]uint64{} parser, err := getParser(ctx, chainID) if err != nil { return err @@ -134,7 +134,7 @@ func (h *Handler) Spam( var fundsL sync.Mutex for i := 0; i < numAccounts; i++ { // Create account - pk, err := crypto.GeneratePrivateKey() + pk, err := ed25519.GeneratePrivateKey() if err != nil { return err } @@ -259,7 +259,7 @@ func (h *Handler) Spam( // Send transaction start := time.Now() - selected := map[crypto.PublicKey]int{} + selected := map[ed25519.PublicKey]int{} for k := 0; k < numTxsPerAccount; k++ { recipient, err := getNextRecipient(randomRecipient, i, accounts) if err != nil { @@ -463,11 +463,11 @@ func startIssuer(cctx context.Context, issuer *txIssuer) { }() } -func getNextRecipient(randomRecipient bool, self int, keys []crypto.PrivateKey) (crypto.PublicKey, error) { +func getNextRecipient(randomRecipient bool, self int, keys []ed25519.PrivateKey) (ed25519.PublicKey, error) { if randomRecipient { - priv, err := crypto.GeneratePrivateKey() + priv, err := ed25519.GeneratePrivateKey() if err != nil { - return crypto.EmptyPublicKey, err + return ed25519.EmptyPublicKey, err } return priv.PublicKey(), nil } diff --git a/cli/storage.go b/cli/storage.go index 510ea72420..4cb331df27 100644 --- a/cli/storage.go +++ b/cli/storage.go @@ -10,7 +10,7 @@ import ( "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/utils" ) @@ -65,9 +65,9 @@ func (h *Handler) GetDefaultChain() (ids.ID, []string, error) { return chainID, uris, nil } -func (h *Handler) StoreKey(privateKey crypto.PrivateKey) error { +func (h *Handler) StoreKey(privateKey ed25519.PrivateKey) error { publicKey := privateKey.PublicKey() - k := make([]byte, 1+crypto.PublicKeyLen) + k := make([]byte, 1+ed25519.PublicKeyLen) k[0] = keyPrefix copy(k[1:], publicKey[:]) has, err := h.db.Has(k) @@ -80,49 +80,49 @@ func (h *Handler) StoreKey(privateKey crypto.PrivateKey) error { return h.db.Put(k, privateKey[:]) } -func (h *Handler) GetKey(publicKey crypto.PublicKey) (crypto.PrivateKey, error) { - k := make([]byte, 1+crypto.PublicKeyLen) +func (h *Handler) GetKey(publicKey ed25519.PublicKey) (ed25519.PrivateKey, error) { + k := make([]byte, 1+ed25519.PublicKeyLen) k[0] = keyPrefix copy(k[1:], publicKey[:]) v, err := h.db.Get(k) if errors.Is(err, database.ErrNotFound) { - return crypto.EmptyPrivateKey, nil + return ed25519.EmptyPrivateKey, nil } if err != nil { - return crypto.EmptyPrivateKey, err + return ed25519.EmptyPrivateKey, err } - return crypto.PrivateKey(v), nil + return ed25519.PrivateKey(v), nil } -func (h *Handler) GetKeys() ([]crypto.PrivateKey, error) { +func (h *Handler) GetKeys() ([]ed25519.PrivateKey, error) { iter := h.db.NewIteratorWithPrefix([]byte{keyPrefix}) defer iter.Release() - privateKeys := []crypto.PrivateKey{} + privateKeys := []ed25519.PrivateKey{} for iter.Next() { // It is safe to use these bytes directly because the database copies the // iterator value for us. - privateKeys = append(privateKeys, crypto.PrivateKey(iter.Value())) + privateKeys = append(privateKeys, ed25519.PrivateKey(iter.Value())) } return privateKeys, iter.Error() } -func (h *Handler) StoreDefaultKey(pk crypto.PublicKey) error { +func (h *Handler) StoreDefaultKey(pk ed25519.PublicKey) error { return h.StoreDefault(defaultKeyKey, pk[:]) } -func (h *Handler) GetDefaultKey() (crypto.PrivateKey, error) { +func (h *Handler) GetDefaultKey() (ed25519.PrivateKey, error) { v, err := h.GetDefault(defaultKeyKey) if err != nil { - return crypto.EmptyPrivateKey, err + return ed25519.EmptyPrivateKey, err } if len(v) == 0 { - return crypto.EmptyPrivateKey, ErrNoKeys + return ed25519.EmptyPrivateKey, ErrNoKeys } - publicKey := crypto.PublicKey(v) + publicKey := ed25519.PublicKey(v) priv, err := h.GetKey(publicKey) if err != nil { - return crypto.EmptyPrivateKey, err + return ed25519.EmptyPrivateKey, err } utils.Outf("{{yellow}}address:{{/}} %s\n", h.c.Address(publicKey)) return priv, nil diff --git a/codec/optional_packer.go b/codec/optional_packer.go index 014ac686aa..76379c1d0d 100644 --- a/codec/optional_packer.go +++ b/codec/optional_packer.go @@ -7,7 +7,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" ) // OptionalPacker defines a struct that includes a Packer [ip], a bitset @@ -96,8 +96,8 @@ func (o *OptionalPacker) UnpackID(dest *ids.ID) { // PackPublicKey packs [pk] into OptionalPacker if [pk] is not an empty PublicKey. // Updates the bitset and offset accordingly. -func (o *OptionalPacker) PackPublicKey(pk crypto.PublicKey) { - if pk == crypto.EmptyPublicKey { +func (o *OptionalPacker) PackPublicKey(pk ed25519.PublicKey) { + if pk == ed25519.EmptyPublicKey { o.skipBit() return } @@ -107,11 +107,11 @@ func (o *OptionalPacker) PackPublicKey(pk crypto.PublicKey) { // UnpackPublicKey unpacks a PublicKey into [dest] if the bitset is set at // the current offset. Increments offset regardless. -func (o *OptionalPacker) UnpackPublicKey(dest *crypto.PublicKey) { +func (o *OptionalPacker) UnpackPublicKey(dest *ed25519.PublicKey) { if o.checkBit() { o.ip.UnpackPublicKey(true, dest) } else { - *dest = crypto.EmptyPublicKey + *dest = ed25519.EmptyPublicKey } } diff --git a/codec/optional_packer_test.go b/codec/optional_packer_test.go index 904ac5dbc8..deaadc4e6e 100644 --- a/codec/optional_packer_test.go +++ b/codec/optional_packer_test.go @@ -9,7 +9,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/stretchr/testify/require" ) @@ -30,7 +30,7 @@ func TestOptionalPackerWriter(t *testing.T) { require := require.New(t) opw := NewOptionalWriter(10_000) require.Empty(opw.ip.Bytes()) - var pubKey crypto.PublicKey + var pubKey ed25519.PublicKey copy(pubKey[:], TestPublicKey) // Fill OptionalPacker i := 0 @@ -39,7 +39,7 @@ func TestOptionalPackerWriter(t *testing.T) { i += 1 } require.Equal( - (consts.MaxUint64Offset+1)*crypto.PublicKeyLen, + (consts.MaxUint64Offset+1)*ed25519.PublicKeyLen, len(opw.ip.Bytes()), "Bytes not added correctly.", ) @@ -51,11 +51,11 @@ func TestOptionalPackerWriter(t *testing.T) { func TestOptionalPackerPublicKey(t *testing.T) { require := require.New(t) opw := NewOptionalWriter(10_000) - var pubKey crypto.PublicKey + var pubKey ed25519.PublicKey copy(pubKey[:], TestPublicKey) t.Run("Pack", func(t *testing.T) { // Pack empty - opw.PackPublicKey(crypto.EmptyPublicKey) + opw.PackPublicKey(ed25519.EmptyPublicKey) require.Empty(opw.ip.Bytes(), "PackPublickey packed an empty ID.") // Pack ID opw.PackPublicKey(pubKey) @@ -64,10 +64,10 @@ func TestOptionalPackerPublicKey(t *testing.T) { t.Run("Unpack", func(t *testing.T) { // Setup optional reader opr := opw.toReader() - var unpackedPubkey crypto.PublicKey + var unpackedPubkey ed25519.PublicKey // Unpack opr.UnpackPublicKey(&unpackedPubkey) - require.Equal(crypto.EmptyPublicKey[:], unpackedPubkey[:], "PublicKey unpacked correctly") + require.Equal(ed25519.EmptyPublicKey[:], unpackedPubkey[:], "PublicKey unpacked correctly") opr.UnpackPublicKey(&unpackedPubkey) require.Equal(pubKey, unpackedPubkey, "PublicKey unpacked correctly") opr.Done() diff --git a/codec/packer.go b/codec/packer.go index 02676129ea..3e9515c459 100644 --- a/codec/packer.go +++ b/codec/packer.go @@ -10,7 +10,7 @@ import ( "github.com/ava-labs/avalanchego/utils/wrappers" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/window" ) @@ -119,27 +119,27 @@ func (p *Packer) UnpackInt64(required bool) int64 { return int64(v) } -func (p *Packer) PackPublicKey(src crypto.PublicKey) { +func (p *Packer) PackPublicKey(src ed25519.PublicKey) { p.p.PackFixedBytes(src[:]) } -// UnpackPublicKey crypto.PublicKey into [dest]. -func (p *Packer) UnpackPublicKey(required bool, dest *crypto.PublicKey) { - copy((*dest)[:], p.p.UnpackFixedBytes(crypto.PublicKeyLen)) - if required && *dest == crypto.EmptyPublicKey { +// UnpackPublicKey ed25519.PublicKey into [dest]. +func (p *Packer) UnpackPublicKey(required bool, dest *ed25519.PublicKey) { + copy((*dest)[:], p.p.UnpackFixedBytes(ed25519.PublicKeyLen)) + if required && *dest == ed25519.EmptyPublicKey { p.addErr(fmt.Errorf("%w: PublicKey field is not populated", ErrFieldNotPopulated)) } } -func (p *Packer) PackSignature(src crypto.Signature) { +func (p *Packer) PackSignature(src ed25519.Signature) { p.p.PackFixedBytes(src[:]) } -// UnpackPublicKey crypto.Signature into [dest]. +// UnpackPublicKey ed25519.Signature into [dest]. // TODO: should add required param? -func (p *Packer) UnpackSignature(dest *crypto.Signature) { - copy((*dest)[:], p.p.UnpackFixedBytes(crypto.SignatureLen)) - if *dest == crypto.EmptySignature { +func (p *Packer) UnpackSignature(dest *ed25519.Signature) { + copy((*dest)[:], p.p.UnpackFixedBytes(ed25519.SignatureLen)) + if *dest == ed25519.EmptySignature { p.addErr(fmt.Errorf("%w: Signature field is not populated", ErrFieldNotPopulated)) } } diff --git a/codec/packer_test.go b/codec/packer_test.go index 53d8648128..bf74a18c94 100644 --- a/codec/packer_test.go +++ b/codec/packer_test.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/window" "github.com/stretchr/testify/require" ) @@ -48,8 +48,8 @@ func TestNewWriter(t *testing.T) { func TestPackerPublicKey(t *testing.T) { require := require.New(t) - wp := NewWriter(crypto.PublicKeyLen, crypto.PublicKeyLen) - var pubKey crypto.PublicKey + wp := NewWriter(ed25519.PublicKeyLen, ed25519.PublicKeyLen) + var pubKey ed25519.PublicKey copy(pubKey[:], TestPublicKey) t.Run("Pack", func(t *testing.T) { // Pack @@ -59,9 +59,9 @@ func TestPackerPublicKey(t *testing.T) { }) t.Run("Unpack", func(t *testing.T) { // Unpack - rp := NewReader(wp.Bytes(), crypto.PublicKeyLen) + rp := NewReader(wp.Bytes(), ed25519.PublicKeyLen) require.Equal(wp.Bytes(), rp.Bytes(), "Reader not initialized correctly.") - var unpackedPubKey crypto.PublicKey + var unpackedPubKey ed25519.PublicKey rp.UnpackPublicKey(true, &unpackedPubKey) require.Equal(pubKey, unpackedPubKey, "UnpackPublicKey unpacked incorrectly.") require.NoError(rp.Err(), "UnpackPublicKey set an error.") @@ -73,8 +73,8 @@ func TestPackerPublicKey(t *testing.T) { func TestPackerSignature(t *testing.T) { require := require.New(t) - wp := NewWriter(crypto.SignatureLen, crypto.SignatureLen) - var sig crypto.Signature + wp := NewWriter(ed25519.SignatureLen, ed25519.SignatureLen) + var sig ed25519.Signature copy(sig[:], TestSignature) t.Run("Pack", func(t *testing.T) { // Pack @@ -84,9 +84,9 @@ func TestPackerSignature(t *testing.T) { }) t.Run("Unpack", func(t *testing.T) { // Unpack - rp := NewReader(wp.Bytes(), crypto.SignatureLen) + rp := NewReader(wp.Bytes(), ed25519.SignatureLen) require.Equal(wp.Bytes(), rp.Bytes(), "Reader not initialized correctly.") - var unpackedSig crypto.Signature + var unpackedSig ed25519.Signature rp.UnpackSignature(&unpackedSig) require.Equal(sig, unpackedSig, "UnpackSignature unpacked incorrectly.") require.NoError(rp.Err(), "UnpackSignature set an error.") diff --git a/crypto/ed25519/cache.go b/crypto/ed25519/cache.go new file mode 100644 index 0000000000..cda6fee6e7 --- /dev/null +++ b/crypto/ed25519/cache.go @@ -0,0 +1,150 @@ +// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +// This file derives from the following file in the upstream oasisprotocol/curve25519-voi +// package: +// +// https://github.com/oasisprotocol/curve25519-voi/blob/db37f07504ce84aa791568a45ce9feb3174ae480/primitives/ed25519/extra/cache/cache.go +// +// As required by the license on that upstream package, the copyright notice, list of conditions, +// and disclaimer are preserved: +// +// Copyright (c) 2021 Oasis Labs Inc. All rights reserved. +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// 1. Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright +// notice, this list of conditions and the following disclaimer in the +// documentation and/or other materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS +// IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED +// TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A +// PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +// TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +// PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +// LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +// NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package ed25519 + +import ( + "github.com/oasisprotocol/curve25519-voi/curve" + "github.com/oasisprotocol/curve25519-voi/primitives/ed25519" +) + +// Cache is an expanded public key cache. +type Cache interface { + // Get returns a public key's corresponding expanded public key iff + // present in the cache, or returns nil. + Get(publicKey *curve.CompressedEdwardsY) *ed25519.ExpandedPublicKey + + // Put adds the expanded public key to the cache. + Put(publicKey *curve.CompressedEdwardsY, expanded *ed25519.ExpandedPublicKey) +} + +// Verifier verifies signatures, storing expanded public keys in a cache +// for reuse by subsequent verification with the same public key. +// +// Note: Unless there are more cache hits than misses, this will likely +// be a net performance loss. Integration should be followed by +// benchmarking. +type Verifier struct { + cache Cache +} + +// Verify repors whether sig is a valid Ed25519 signature by public key. +func (v *Verifier) Verify(publicKey ed25519.PublicKey, message, sig []byte) bool { + return v.VerifyWithOptions(publicKey, message, sig, &ed25519.Options{}) +} + +// VerifyWithOptions reports whether sig is a valid Ed25519 signature by +// publicKey, with extra Options. +// +// Unlike the upstream package, the publicKey is not expanded and cached +// if it doesn't exist. This is done to protect the cache from malicious +// transaction submissions. +func (v *Verifier) VerifyWithOptions(publicKey ed25519.PublicKey, message, sig []byte, opts *ed25519.Options) bool { + expanded, ok := v.getExpandedPublicKey(publicKey) + if !ok { + return ed25519.VerifyWithOptions(publicKey, message, sig, opts) + } + + return ed25519.VerifyExpandedWithOptions(expanded, message, sig, opts) +} + +// Add will add the signature to the batch verifier. +func (v *Verifier) Add(verifier *ed25519.BatchVerifier, publicKey ed25519.PublicKey, message, sig []byte) { + v.AddWithOptions(verifier, publicKey, message, sig, &ed25519.Options{}) +} + +// AddWithOptions will add the signature to the batch verifier, with +// extra Options. +func (v *Verifier) AddWithOptions(verifier *ed25519.BatchVerifier, publicKey ed25519.PublicKey, message, sig []byte, opts *ed25519.Options) { + expanded, ok := v.getExpandedPublicKey(publicKey) + if !ok { + verifier.AddWithOptions(publicKey, message, sig, opts) + return + } + verifier.AddExpandedWithOptions(expanded, message, sig, opts) +} + +// AddPublicKey will expand and add the public key to the cache. +// +// This should only be called on "fee protected" paths (like after block +// accept) to prevent trivial flushing of the LRU cache. +func (v *Verifier) AddPublicKey(publicKey ed25519.PublicKey) { + v.upsertPublicKey(publicKey) +} + +func (v *Verifier) getExpandedPublicKey(publicKey ed25519.PublicKey) (*ed25519.ExpandedPublicKey, bool) { + var ( + compressed curve.CompressedEdwardsY + err error + ) + if _, err = compressed.SetBytes(publicKey); err != nil { + return nil, false + } + + expanded := v.cache.Get(&compressed) + return expanded, expanded != nil +} + +func (v *Verifier) upsertPublicKey(publicKey ed25519.PublicKey) (*ed25519.ExpandedPublicKey, bool) { + var ( + compressed curve.CompressedEdwardsY + err error + ) + if _, err = compressed.SetBytes(publicKey); err != nil { + return nil, false + } + + expanded := v.cache.Get(&compressed) + if expanded == nil { + if expanded, err = ed25519.NewExpandedPublicKey(compressed[:]); err != nil { + return nil, false + } + v.cache.Put(&compressed, expanded) + } + + return expanded, true +} + +// NewVerifier creates a new Verifier instance backed by a Cache. +func NewVerifier(cache Cache) *Verifier { + return &Verifier{ + cache: cache, + } +} diff --git a/crypto/ed25519.go b/crypto/ed25519/ed25519.go similarity index 67% rename from crypto/ed25519.go rename to crypto/ed25519/ed25519.go index d57e77d7dd..d9b0977e73 100644 --- a/crypto/ed25519.go +++ b/crypto/ed25519/ed25519.go @@ -1,16 +1,16 @@ // Copyright (C) 2023, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -// Package crypto provides functionality for interacting with Ed25519 -// public and private keys. Package crypto uses the "crypto/ed25519" -// package from Go's standard library for the underlying cryptography. -package crypto +package ed25519 import ( - "crypto/ed25519" + "crypto/rand" "encoding/hex" "os" + "github.com/oasisprotocol/curve25519-voi/primitives/ed25519" + "github.com/oasisprotocol/curve25519-voi/primitives/ed25519/extra/cache" + "github.com/ava-labs/avalanchego/utils/formatting/address" ) @@ -28,14 +28,40 @@ const ( // to extract the publicKey below. PrivateKeySeedLen = ed25519.SeedSize SignatureLen = ed25519.SignatureSize + + // TODO: make this tunable + cacheSize = 128_000 // ~179MB (keys are ~1.4KB each) ) var ( EmptyPublicKey = [ed25519.PublicKeySize]byte{} EmptyPrivateKey = [ed25519.PrivateKeySize]byte{} EmptySignature = [ed25519.SignatureSize]byte{} + + verifyOptions ed25519.Options + cacheVerifier *Verifier ) +func init() { + // We use the ZIP-215 specification for ed25519 signature + // verification (https://zips.z.cash/zip-0215) because it provides + // an explicit validity criteria for signatures, supports batch + // verification, and is broadly compatible with signatures produced + // by almost all ed25519 implementations (which don't require + // canonically-encoded points). + // + // You can read more about the rationale for ZIP-215 here: + // https://hdevalence.ca/blog/2020-10-04-its-25519am + // + // You can read more about the challenge of ed25519 verification here: + // https://eprint.iacr.org/2020/1244.pdf + verifyOptions.Verify = ed25519.VerifyOptionsZIP_215 + + // cacheVerifier stores expanded ed25519 Public Keys (each is ~1.4KB). Using + // a cached expanded key reduces verification latency by ~25%. + cacheVerifier = NewVerifier(cache.NewLRUCache(cacheSize)) +} + // Address returns a Bech32 address from hrp and p. // This function uses avalanchego's FormatBech32 function. func Address(hrp string, p PublicKey) string { @@ -114,7 +140,7 @@ func Sign(msg []byte, pk PrivateKey) Signature { // Verify returns whether s is a valid signature of msg by p. func Verify(msg []byte, p PublicKey, s Signature) bool { - return ed25519.Verify(p[:], msg, s[:]) + return cacheVerifier.VerifyWithOptions(p[:], msg, s[:], &verifyOptions) } // HexToKey Converts a hexadecimal encoded key into a PrivateKey. Returns @@ -129,3 +155,35 @@ func HexToKey(key string) (PrivateKey, error) { } return PrivateKey(bytes), nil } + +func CachePublicKey(p PublicKey) { + cacheVerifier.AddPublicKey(p[:]) +} + +type Batch struct { + bv *ed25519.BatchVerifier +} + +func NewBatch(numItems int) *Batch { + if numItems <= 0 { + return &Batch{ed25519.NewBatchVerifier()} + } + return &Batch{ed25519.NewBatchVerifierWithCapacity(numItems)} +} + +func (b *Batch) Add(msg []byte, p PublicKey, s Signature) { + cacheVerifier.AddWithOptions(b.bv, p[:], msg, s[:], &verifyOptions) +} + +func (b *Batch) Verify() bool { + return b.bv.VerifyBatchOnly(rand.Reader) +} + +func (b *Batch) VerifyAsync() func() error { + return func() error { + if !b.Verify() { + return ErrInvalidSignature + } + return nil + } +} diff --git a/crypto/ed25519_test.go b/crypto/ed25519/ed25519_test.go similarity index 67% rename from crypto/ed25519_test.go rename to crypto/ed25519/ed25519_test.go index 4c3f6e8b6f..529811d129 100644 --- a/crypto/ed25519_test.go +++ b/crypto/ed25519/ed25519_test.go @@ -1,14 +1,19 @@ // Copyright (C) 2023, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package crypto +package ed25519 import ( "crypto/ed25519" + "crypto/rand" "os" "path/filepath" + "strconv" "testing" + oed25519 "github.com/oasisprotocol/curve25519-voi/primitives/ed25519" + "github.com/oasisprotocol/curve25519-voi/primitives/ed25519/extra/cache" + "github.com/stretchr/testify/require" ) @@ -251,3 +256,139 @@ func TestKeyToHex(t *testing.T) { require := require.New(t) require.Equal(TestPrivateKeyHex, TestPrivateKey.ToHex()) } + +func BenchmarkStdLibVerifySingle(b *testing.B) { + for i := 0; i < b.N; i++ { + b.StopTimer() + msg := make([]byte, 128) + _, err := rand.Read(msg) + if err != nil { + panic(err) + } + pub, priv, err := ed25519.GenerateKey(nil) + if err != nil { + panic(err) + } + sig := ed25519.Sign(priv, msg) + b.StartTimer() + if !ed25519.Verify(pub, msg, sig) { + panic("invalid signature") + } + } +} + +func BenchmarkOasisVerifySingle(b *testing.B) { + for i := 0; i < b.N; i++ { + b.StopTimer() + msg := make([]byte, 128) + _, err := rand.Read(msg) + if err != nil { + panic(err) + } + pub, priv, err := oed25519.GenerateKey(nil) + if err != nil { + panic(err) + } + sig := oed25519.Sign(priv, msg) + b.StartTimer() + if !oed25519.VerifyWithOptions(pub, msg, sig, &verifyOptions) { + panic("invalid signature") + } + } +} + +func BenchmarkOasisVerifyCache(b *testing.B) { + cacheVerifier := cache.NewVerifier(cache.NewLRUCache(10000)) + for i := 0; i < b.N; i++ { + b.StopTimer() + msg := make([]byte, 128) + _, err := rand.Read(msg) + if err != nil { + panic(err) + } + pub, priv, err := oed25519.GenerateKey(nil) + if err != nil { + panic(err) + } + sig := oed25519.Sign(priv, msg) + cacheVerifier.AddPublicKey(pub) + b.StartTimer() + if !cacheVerifier.VerifyWithOptions(pub, msg, sig, &verifyOptions) { + panic("invalid signature") + } + } +} + +func BenchmarkOasisBatchVerify(b *testing.B) { + for _, numItems := range []int{1, 4, 16, 64, 128, 512, 1024, 4096, 16384} { + b.Run(strconv.Itoa(numItems), func(b *testing.B) { + for i := 0; i < b.N; i++ { + b.StopTimer() + pubs := make([][]byte, numItems) + msgs := make([][]byte, numItems) + sigs := make([][]byte, numItems) + for j := 0; j < numItems; j++ { + pub, priv, err := oed25519.GenerateKey(nil) + if err != nil { + panic(err) + } + pubs[j] = pub + msg := make([]byte, 128) + _, err = rand.Read(msg) + if err != nil { + panic(err) + } + msgs[j] = msg + sig := oed25519.Sign(priv, msg) + sigs[j] = sig + } + b.StartTimer() + bv := oed25519.NewBatchVerifierWithCapacity(numItems) + for j := 0; j < numItems; j++ { + bv.AddWithOptions(pubs[j], msgs[j], sigs[j], &verifyOptions) + } + if !bv.VerifyBatchOnly(nil) { + panic("invalid signature") + } + } + }) + } +} + +func BenchmarkOasisBatchVerifyCache(b *testing.B) { + cacheVerifier := cache.NewVerifier(cache.NewLRUCache(30000)) + for _, numItems := range []int{1, 4, 16, 64, 128, 512, 1024, 4096, 16384} { + b.Run(strconv.Itoa(numItems), func(b *testing.B) { + for i := 0; i < b.N; i++ { + b.StopTimer() + pubs := make([][]byte, numItems) + msgs := make([][]byte, numItems) + sigs := make([][]byte, numItems) + for j := 0; j < numItems; j++ { + pub, priv, err := oed25519.GenerateKey(nil) + if err != nil { + panic(err) + } + cacheVerifier.AddPublicKey(pub) + pubs[j] = pub + msg := make([]byte, 128) + _, err = rand.Read(msg) + if err != nil { + panic(err) + } + msgs[j] = msg + sig := oed25519.Sign(priv, msg) + sigs[j] = sig + } + b.StartTimer() + bv := oed25519.NewBatchVerifierWithCapacity(numItems) + for j := 0; j < numItems; j++ { + cacheVerifier.AddWithOptions(bv, pubs[j], msgs[j], sigs[j], &verifyOptions) + } + if !bv.VerifyBatchOnly(nil) { + panic("invalid signature") + } + } + }) + } +} diff --git a/crypto/errors.go b/crypto/ed25519/errors.go similarity index 95% rename from crypto/errors.go rename to crypto/ed25519/errors.go index 5eb06d85f3..9ecd327082 100644 --- a/crypto/errors.go +++ b/crypto/ed25519/errors.go @@ -1,7 +1,7 @@ // Copyright (C) 2023, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package crypto +package ed25519 import "errors" diff --git a/examples/morpheusvm/actions/transfer.go b/examples/morpheusvm/actions/transfer.go index a8740d6d1b..be3bdbff64 100644 --- a/examples/morpheusvm/actions/transfer.go +++ b/examples/morpheusvm/actions/transfer.go @@ -11,7 +11,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/auth" "github.com/ava-labs/hypersdk/examples/morpheusvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -21,7 +21,7 @@ var _ chain.Action = (*Transfer)(nil) type Transfer struct { // To is the recipient of the [Value]. - To crypto.PublicKey `json:"to"` + To ed25519.PublicKey `json:"to"` // Amount are transferred to [To]. Value uint64 `json:"value"` @@ -64,11 +64,11 @@ func (t *Transfer) Execute( func (*Transfer) MaxUnits(chain.Rules) uint64 { // We use size as the price of this transaction but we could just as easily // use any other calculation. - return crypto.PublicKeyLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.Uint64Len } func (*Transfer) Size() int { - return crypto.PublicKeyLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.Uint64Len } func (t *Transfer) Marshal(p *codec.Packer) { diff --git a/examples/morpheusvm/auth/common.go b/examples/morpheusvm/auth/common.go index cf838e454d..67596fbc35 100644 --- a/examples/morpheusvm/auth/common.go +++ b/examples/morpheusvm/auth/common.go @@ -3,7 +3,15 @@ package auth +import "github.com/ava-labs/hypersdk/vm" + // Note: Registry will error during initialization if a duplicate ID is assigned. We explicitly assign IDs to avoid accidental remapping. const ( ed25519ID uint8 = 0 ) + +func Engines() map[uint8]vm.AuthEngine { + return map[uint8]vm.AuthEngine{ + ed25519ID: &ED25519AuthEngine{}, + } +} diff --git a/examples/morpheusvm/auth/ed25519.go b/examples/morpheusvm/auth/ed25519.go index 1b9aa99f3a..b1a74a5329 100644 --- a/examples/morpheusvm/auth/ed25519.go +++ b/examples/morpheusvm/auth/ed25519.go @@ -6,18 +6,19 @@ package auth import ( "context" + "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/storage" ) var _ chain.Auth = (*ED25519)(nil) type ED25519 struct { - Signer crypto.PublicKey `json:"signer"` - Signature crypto.Signature `json:"signature"` + Signer ed25519.PublicKey `json:"signer"` + Signature ed25519.Signature `json:"signature"` } func (*ED25519) GetTypeID() uint8 { @@ -27,7 +28,7 @@ func (*ED25519) GetTypeID() uint8 { func (*ED25519) MaxUnits( chain.Rules, ) uint64 { - return crypto.PublicKeyLen + crypto.SignatureLen*5 // make signatures more expensive + return ed25519.PublicKeyLen + ed25519.SignatureLen*5 // make signatures more expensive } func (*ED25519) ValidRange(chain.Rules) (int64, int64) { @@ -42,8 +43,8 @@ func (d *ED25519) StateKeys() [][]byte { } func (d *ED25519) AsyncVerify(msg []byte) error { - if !crypto.Verify(msg, d.Signer, d.Signature) { - return ErrInvalidSignature + if !ed25519.Verify(msg, d.Signer, d.Signature) { + return ed25519.ErrInvalidSignature } return nil } @@ -64,7 +65,7 @@ func (d *ED25519) Payer() []byte { } func (*ED25519) Size() int { - return crypto.PublicKeyLen + crypto.SignatureLen + return ed25519.PublicKeyLen + ed25519.SignatureLen } func (d *ED25519) Marshal(p *codec.Packer) { @@ -112,15 +113,66 @@ func (d *ED25519) Refund( var _ chain.AuthFactory = (*ED25519Factory)(nil) -func NewED25519Factory(priv crypto.PrivateKey) *ED25519Factory { +func NewED25519Factory(priv ed25519.PrivateKey) *ED25519Factory { return &ED25519Factory{priv} } type ED25519Factory struct { - priv crypto.PrivateKey + priv ed25519.PrivateKey } func (d *ED25519Factory) Sign(msg []byte, _ chain.Action) (chain.Auth, error) { - sig := crypto.Sign(msg, d.priv) + sig := ed25519.Sign(msg, d.priv) return &ED25519{d.priv.PublicKey(), sig}, nil } + +type ED25519AuthEngine struct{} + +func (*ED25519AuthEngine) GetBatchVerifier(cores int, count int) chain.AuthBatchVerifier { + batchSize := math.Max(count/cores, 16) + return &ED25519Batch{ + batchSize: batchSize, + total: count, + } +} + +func (*ED25519AuthEngine) Cache(auth chain.Auth) { + pk := GetSigner(auth) + ed25519.CachePublicKey(pk) +} + +type ED25519Batch struct { + batchSize int + total int + + counter int + totalCounter int + batch *ed25519.Batch +} + +func (b *ED25519Batch) Add(msg []byte, rauth chain.Auth) func() error { + auth := rauth.(*ED25519) + if b.batch == nil { + b.batch = ed25519.NewBatch(b.batchSize) + } + b.batch.Add(msg, auth.Signer, auth.Signature) + b.counter++ + b.totalCounter++ + if b.counter == b.batchSize { + last := b.batch + b.counter = 0 + if b.totalCounter < b.total { + // don't create a new batch if we are done + b.batch = ed25519.NewBatch(b.batchSize) + } + return last.VerifyAsync() + } + return nil +} + +func (b *ED25519Batch) Done() []func() error { + if b.batch == nil { + return nil + } + return []func() error{b.batch.VerifyAsync()} +} diff --git a/examples/morpheusvm/auth/errors.go b/examples/morpheusvm/auth/errors.go deleted file mode 100644 index 79462e2fc7..0000000000 --- a/examples/morpheusvm/auth/errors.go +++ /dev/null @@ -1,8 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package auth - -import "errors" - -var ErrInvalidSignature = errors.New("invalid signature") diff --git a/examples/morpheusvm/auth/helpers.go b/examples/morpheusvm/auth/helpers.go index 2b73e20cca..340f44f367 100644 --- a/examples/morpheusvm/auth/helpers.go +++ b/examples/morpheusvm/auth/helpers.go @@ -5,23 +5,23 @@ package auth import ( "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" ) -func GetActor(auth chain.Auth) crypto.PublicKey { +func GetActor(auth chain.Auth) ed25519.PublicKey { switch a := auth.(type) { case *ED25519: return a.Signer default: - return crypto.EmptyPublicKey + return ed25519.EmptyPublicKey } } -func GetSigner(auth chain.Auth) crypto.PublicKey { +func GetSigner(auth chain.Auth) ed25519.PublicKey { switch a := auth.(type) { case *ED25519: return a.Signer default: - return crypto.EmptyPublicKey + return ed25519.EmptyPublicKey } } diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/handler.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/handler.go index 555b4a2ec1..ef465b56aa 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/handler.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/handler.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/cli" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/auth" "github.com/ava-labs/hypersdk/examples/morpheusvm/consts" brpc "github.com/ava-labs/hypersdk/examples/morpheusvm/rpc" @@ -32,21 +32,21 @@ func (h *Handler) Root() *cli.Handler { } func (h *Handler) DefaultActor() ( - ids.ID, crypto.PrivateKey, *auth.ED25519Factory, + ids.ID, ed25519.PrivateKey, *auth.ED25519Factory, *rpc.JSONRPCClient, *brpc.JSONRPCClient, error, ) { priv, err := h.h.GetDefaultKey() if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } chainID, uris, err := h.h.GetDefaultChain() if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } cli := rpc.NewJSONRPCClient(uris[0]) networkID, _, _, err := cli.Network(context.TODO()) if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } // For [defaultActor], we always send requests to the first returned URI. return chainID, priv, auth.NewED25519Factory( @@ -62,7 +62,7 @@ func (h *Handler) DefaultActor() ( func (*Handler) GetBalance( ctx context.Context, cli *brpc.JSONRPCClient, - publicKey crypto.PublicKey, + publicKey ed25519.PublicKey, ) (uint64, error) { addr := utils.Address(publicKey) balance, err := cli.Balance(ctx, addr) @@ -99,10 +99,10 @@ func (*Controller) Symbol() string { return consts.Symbol } -func (*Controller) Address(pk crypto.PublicKey) string { +func (*Controller) Address(pk ed25519.PublicKey) string { return utils.Address(pk) } -func (*Controller) ParseAddress(address string) (crypto.PublicKey, error) { +func (*Controller) ParseAddress(address string) (ed25519.PublicKey, error) { return utils.ParseAddress(address) } diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/key.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/key.go index b8ff5896bd..3b84bb0fcb 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/key.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/key.go @@ -7,7 +7,7 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" brpc "github.com/ava-labs/hypersdk/examples/morpheusvm/rpc" hutils "github.com/ava-labs/hypersdk/utils" "github.com/spf13/cobra" @@ -64,7 +64,7 @@ var setKeyCmd = &cobra.Command{ }, } -func lookupKeyBalance(pk crypto.PublicKey, uri string, networkID uint32, chainID ids.ID, _ ids.ID) error { +func lookupKeyBalance(pk ed25519.PublicKey, uri string, networkID uint32, chainID ids.ID, _ ids.ID) error { _, err := handler.GetBalance(context.TODO(), brpc.NewJSONRPCClient(uri, networkID, chainID), pk) return err } diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go index 92d795534e..eaf7f4527f 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/actions" "github.com/ava-labs/hypersdk/examples/morpheusvm/auth" brpc "github.com/ava-labs/hypersdk/examples/morpheusvm/rpc" @@ -32,7 +32,7 @@ var runSpamCmd = &cobra.Command{ func(uri string, networkID uint32, chainID ids.ID) { bclient = brpc.NewJSONRPCClient(uri, networkID, chainID) }, - func(pk crypto.PrivateKey) chain.AuthFactory { + func(pk ed25519.PrivateKey) chain.AuthFactory { return auth.NewED25519Factory(pk) }, func(choice int, address string) (uint64, error) { @@ -52,13 +52,13 @@ var runSpamCmd = &cobra.Command{ func(ctx context.Context, chainID ids.ID) (chain.Parser, error) { return bclient.Parser(ctx) }, - func(pk crypto.PublicKey, amount uint64) chain.Action { + func(pk ed25519.PublicKey, amount uint64) chain.Action { return &actions.Transfer{ To: pk, Value: amount, } }, - func(cli *rpc.JSONRPCClient, pk crypto.PrivateKey) func(context.Context, uint64) error { + func(cli *rpc.JSONRPCClient, pk ed25519.PrivateKey) func(context.Context, uint64) error { return func(ictx context.Context, count uint64) error { _, _, err := sendAndWait(ictx, nil, &actions.Transfer{ To: pk.PublicKey(), diff --git a/examples/morpheusvm/controller/controller.go b/examples/morpheusvm/controller/controller.go index 4d2fcb9745..195b5378e6 100644 --- a/examples/morpheusvm/controller/controller.go +++ b/examples/morpheusvm/controller/controller.go @@ -20,6 +20,7 @@ import ( "go.uber.org/zap" "github.com/ava-labs/hypersdk/examples/morpheusvm/actions" + "github.com/ava-labs/hypersdk/examples/morpheusvm/auth" "github.com/ava-labs/hypersdk/examples/morpheusvm/config" "github.com/ava-labs/hypersdk/examples/morpheusvm/consts" "github.com/ava-labs/hypersdk/examples/morpheusvm/genesis" @@ -64,6 +65,7 @@ func (c *Controller) Initialize( vm.Handlers, chain.ActionRegistry, chain.AuthRegistry, + map[uint8]vm.AuthEngine, error, ) { c.inner = inner @@ -74,20 +76,20 @@ func (c *Controller) Initialize( var err error c.metrics, err = newMetrics(gatherer) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } // Load config and genesis c.config, err = config.New(c.snowCtx.NodeID, configBytes) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } c.snowCtx.Log.SetLevel(c.config.GetLogLevel()) snowCtx.Log.Info("initialized config", zap.Bool("loaded", c.config.Loaded()), zap.Any("contents", c.config)) c.genesis, err = genesis.New(genesisBytes, upgradeBytes) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, fmt.Errorf( + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, fmt.Errorf( "unable to read genesis: %w", err, ) @@ -97,7 +99,7 @@ func (c *Controller) Initialize( // Create DBs blockDB, stateDB, metaDB, err := hstorage.New(snowCtx.ChainDataDir, gatherer) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } c.metaDB = metaDB @@ -112,7 +114,7 @@ func (c *Controller) Initialize( common.NoLock, ) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } apis[rpc.JSONRPCEndpoint] = jsonRPCHandler @@ -130,7 +132,7 @@ func (c *Controller) Initialize( gcfg := gossiper.DefaultProposerConfig() gossip = gossiper.NewProposer(inner, gcfg) } - return c.config, c.genesis, build, gossip, blockDB, stateDB, apis, consts.ActionRegistry, consts.AuthRegistry, nil + return c.config, c.genesis, build, gossip, blockDB, stateDB, apis, consts.ActionRegistry, consts.AuthRegistry, auth.Engines(), nil } func (c *Controller) Rules(t int64) chain.Rules { diff --git a/examples/morpheusvm/controller/resolutions.go b/examples/morpheusvm/controller/resolutions.go index d307dc4390..5c80aa1adc 100644 --- a/examples/morpheusvm/controller/resolutions.go +++ b/examples/morpheusvm/controller/resolutions.go @@ -9,7 +9,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/trace" "github.com/ava-labs/avalanchego/utils/logging" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/genesis" "github.com/ava-labs/hypersdk/examples/morpheusvm/storage" ) @@ -35,7 +35,7 @@ func (c *Controller) GetTransaction( func (c *Controller) GetBalanceFromState( ctx context.Context, - pk crypto.PublicKey, + pk ed25519.PublicKey, ) (uint64, error) { return storage.GetBalanceFromState(ctx, c.inner.ReadState, pk) } diff --git a/examples/morpheusvm/go.mod b/examples/morpheusvm/go.mod index c376b87a60..6d71eb63d6 100644 --- a/examples/morpheusvm/go.mod +++ b/examples/morpheusvm/go.mod @@ -83,6 +83,7 @@ require ( github.com/mitchellh/pointerstructure v1.2.0 // indirect github.com/mr-tron/base58 v1.2.0 // indirect github.com/nbutton23/zxcvbn-go v0.0.0-20180912185939-ae427f1e4c1d // indirect + github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/openzipkin/zipkin-go v0.4.1 // indirect github.com/otiai10/copy v1.11.0 // indirect diff --git a/examples/morpheusvm/go.sum b/examples/morpheusvm/go.sum index 833d63acbb..9f2df9642c 100644 --- a/examples/morpheusvm/go.sum +++ b/examples/morpheusvm/go.sum @@ -442,6 +442,8 @@ github.com/neilotoole/errgroup v0.1.6 h1:PODGqPXdT5BC/zCYIMoTrwV+ujKcW+gBXM6Ye9V github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce h1:/pEpMk55wH0X+E5zedGEMOdLuWmV8P4+4W3+LZaM6kg= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce/go.mod h1:hVoHR2EVESiICEMbg137etN/Lx+lSrHPTD39Z/uE+2s= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= diff --git a/examples/morpheusvm/rpc/dependencies.go b/examples/morpheusvm/rpc/dependencies.go index b8243b15c1..ae6514bb5a 100644 --- a/examples/morpheusvm/rpc/dependencies.go +++ b/examples/morpheusvm/rpc/dependencies.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/trace" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/genesis" ) @@ -16,5 +16,5 @@ type Controller interface { Genesis() *genesis.Genesis Tracer() trace.Tracer GetTransaction(context.Context, ids.ID) (bool, int64, bool, uint64, error) - GetBalanceFromState(context.Context, crypto.PublicKey) (uint64, error) + GetBalanceFromState(context.Context, ed25519.PublicKey) (uint64, error) } diff --git a/examples/morpheusvm/storage/storage.go b/examples/morpheusvm/storage/storage.go index 864cbbfdf1..dc1d309ece 100644 --- a/examples/morpheusvm/storage/storage.go +++ b/examples/morpheusvm/storage/storage.go @@ -15,7 +15,7 @@ import ( smath "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/utils" ) @@ -50,7 +50,7 @@ var ( // TODO: extend to other types balancePrefixPool = sync.Pool{ New: func() any { - return make([]byte, 1+crypto.PublicKeyLen) + return make([]byte, 1+ed25519.PublicKeyLen) }, } ) @@ -107,7 +107,7 @@ func GetTransaction( } // [accountPrefix] + [address] -func PrefixBalanceKey(pk crypto.PublicKey) (k []byte) { +func PrefixBalanceKey(pk ed25519.PublicKey) (k []byte) { k = balancePrefixPool.Get().([]byte) k[0] = balancePrefix copy(k[1:], pk[:]) @@ -118,7 +118,7 @@ func PrefixBalanceKey(pk crypto.PublicKey) (k []byte) { func GetBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, ) (uint64, error) { dbKey, bal, err := getBalance(ctx, db, pk) balancePrefixPool.Put(dbKey) @@ -128,7 +128,7 @@ func GetBalance( func getBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, ) ([]byte, uint64, error) { k := PrefixBalanceKey(pk) bal, err := innerGetBalance(db.GetValue(ctx, k)) @@ -139,7 +139,7 @@ func getBalance( func GetBalanceFromState( ctx context.Context, f ReadState, - pk crypto.PublicKey, + pk ed25519.PublicKey, ) (uint64, error) { k := PrefixBalanceKey(pk) values, errs := f(ctx, [][]byte{k}) @@ -164,7 +164,7 @@ func innerGetBalance( func SetBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, balance uint64, ) error { k := PrefixBalanceKey(pk) @@ -183,7 +183,7 @@ func setBalance( func AddBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, amount uint64, ) error { dbKey, bal, err := getBalance(ctx, db, pk) @@ -206,7 +206,7 @@ func AddBalance( func SubBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, amount uint64, ) error { dbKey, bal, err := getBalance(ctx, db, pk) diff --git a/examples/morpheusvm/tests/e2e/e2e_test.go b/examples/morpheusvm/tests/e2e/e2e_test.go index 1ad10ba253..1aae15bfc0 100644 --- a/examples/morpheusvm/tests/e2e/e2e_test.go +++ b/examples/morpheusvm/tests/e2e/e2e_test.go @@ -16,7 +16,7 @@ import ( "github.com/ava-labs/avalanchego/config" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/logging" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/actions" "github.com/ava-labs/hypersdk/examples/morpheusvm/auth" "github.com/ava-labs/hypersdk/examples/morpheusvm/consts" @@ -184,7 +184,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Expect(err).Should(gomega.BeNil()) // Load default pk - priv, err = crypto.HexToKey( + priv, err = ed25519.HexToKey( "323b1d8f4eed5f0da9da93071b034f2dce9d2d22692c172f3cb252a64ddfafd01b057de320297c29ad0c1f589ea216869cf1938d88c9fbd70d6748323dbf2fa7", //nolint:lll ) gomega.Ω(err).Should(gomega.BeNil()) @@ -329,9 +329,9 @@ var _ = ginkgo.BeforeSuite(func() { }) var ( - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string instances []instance @@ -398,7 +398,7 @@ var _ = ginkgo.Describe("[Test]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(nativeBalance).Should(gomega.Equal(uint64(1000000000000))) - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) aother := utils.Address(other.PublicKey()) @@ -666,7 +666,7 @@ func generateBlocks( } for ctx.Err() == nil { // Generate transaction - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) submit, _, _, err := instances[cumulativeTxs%len(instances)].cli.GenerateTransaction( context.Background(), @@ -732,7 +732,7 @@ func acceptTransaction(cli *rpc.JSONRPCClient, lcli *lrpc.JSONRPCClient) { gomega.Ω(err).Should(gomega.BeNil()) for { // Generate transaction - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) submit, tx, _, err := cli.GenerateTransaction( context.Background(), diff --git a/examples/morpheusvm/tests/integration/integration_test.go b/examples/morpheusvm/tests/integration/integration_test.go index 5946fd0a07..4eda192537 100644 --- a/examples/morpheusvm/tests/integration/integration_test.go +++ b/examples/morpheusvm/tests/integration/integration_test.go @@ -35,7 +35,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/pubsub" "github.com/ava-labs/hypersdk/rpc" hutils "github.com/ava-labs/hypersdk/utils" @@ -101,19 +101,19 @@ func init() { } var ( - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string - priv2 crypto.PrivateKey + priv2 ed25519.PrivateKey factory2 *auth.ED25519Factory - rsender2 crypto.PublicKey + rsender2 ed25519.PublicKey sender2 string - priv3 crypto.PrivateKey + priv3 ed25519.PrivateKey factory3 *auth.ED25519Factory - rsender3 crypto.PublicKey + rsender3 ed25519.PublicKey sender3 string // when used with embedded VMs @@ -141,7 +141,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(vms).Should(gomega.BeNumerically(">", 1)) var err error - priv, err = crypto.GeneratePrivateKey() + priv, err = ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) factory = auth.NewED25519Factory(priv) rsender = priv.PublicKey() @@ -152,7 +152,7 @@ var _ = ginkgo.BeforeSuite(func() { zap.String("pk", hex.EncodeToString(priv[:])), ) - priv2, err = crypto.GeneratePrivateKey() + priv2, err = ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) factory2 = auth.NewED25519Factory(priv2) rsender2 = priv2.PublicKey() @@ -163,7 +163,7 @@ var _ = ginkgo.BeforeSuite(func() { zap.String("pk", hex.EncodeToString(priv2[:])), ) - priv3, err = crypto.GeneratePrivateKey() + priv3, err = ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) factory3 = auth.NewED25519Factory(priv3) rsender3 = priv3.PublicKey() @@ -603,7 +603,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) // Send tx - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) transfer := &actions.Transfer{ To: other.PublicKey(), @@ -656,7 +656,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) // Create tx - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) transfer := &actions.Transfer{ To: other.PublicKey(), diff --git a/examples/morpheusvm/tests/load/load_test.go b/examples/morpheusvm/tests/load/load_test.go index 6c4fd0a507..65584a3c1e 100644 --- a/examples/morpheusvm/tests/load/load_test.go +++ b/examples/morpheusvm/tests/load/load_test.go @@ -7,6 +7,7 @@ import ( "context" "encoding/hex" "encoding/json" + "errors" "flag" "fmt" "math/rand" @@ -35,7 +36,7 @@ import ( "github.com/ava-labs/hypersdk/chain" hconsts "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/pebble" hutils "github.com/ava-labs/hypersdk/utils" "github.com/ava-labs/hypersdk/vm" @@ -54,7 +55,6 @@ import ( const ( genesisBalance uint64 = hconsts.MaxUint64 transferTxUnits = 440 - maxTxsPerBlock int = 1_800_000 /* max block units */ / transferTxUnits ) var ( @@ -89,9 +89,9 @@ type instance struct { } type account struct { - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string } @@ -164,7 +164,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(vms).Should(gomega.BeNumerically(">", 1)) var err error - priv, err := crypto.GeneratePrivateKey() + priv, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) rsender := priv.PublicKey() sender := utils.Address(rsender) @@ -178,7 +178,8 @@ var _ = ginkgo.BeforeSuite(func() { // create embedded VMs instances = make([]*instance, vms) gen = genesis.Default() - gen.WindowTargetUnits = 1_000_000_000 // disable unit price increase + gen.WindowTargetUnits = 1_000_000_000 // disable unit price increase + gen.MaxBlockUnits = 4_500_000 gen.MinBlockGap = 0 // don't require time between blocks gen.ValidityWindow = 1_000 * hconsts.MillisecondsPerSecond // txs shouldn't expire gen.CustomAllocation = []*genesis.CustomAllocation{ @@ -361,7 +362,7 @@ var _ = ginkgo.Describe("load tests vm", func() { ginkgo.By("create accounts", func() { senders = make([]*account, accts) for i := 0; i < accts; i++ { - tpriv, err := crypto.GeneratePrivateKey() + tpriv, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) trsender := tpriv.PublicKey() tsender := utils.Address(trsender) @@ -375,10 +376,6 @@ var _ = ginkgo.Describe("load tests vm", func() { // leave some left over for root fundSplit := (genesisBalance - remainder) / uint64(accts) gomega.Ω(fundSplit).Should(gomega.Not(gomega.BeZero())) - requiredBlocks := accts / maxTxsPerBlock - if accts%maxTxsPerBlock > 0 { - requiredBlocks++ - } requiredTxs := map[ids.ID]struct{}{} for _, acct := range senders { id, err := issueSimpleTx(instances[0], acct.rsender, fundSplit, root.factory) @@ -386,13 +383,16 @@ var _ = ginkgo.Describe("load tests vm", func() { requiredTxs[id] = struct{}{} } - for i := 0; i < requiredBlocks; i++ { + for { blk := produceBlock(instances[0]) - log.Debug("block produced", zap.Int("txs", len(blk.Txs))) + if blk == nil { + break + } + log.Debug("block produced", zap.Uint64("height", blk.Hght), zap.Int("txs", len(blk.Txs))) for _, result := range blk.Results() { if !result.Success { // Used for debugging - fmt.Println(string(result.Output), i, requiredBlocks) + fmt.Println(string(result.Output)) } gomega.Ω(result.Success).Should(gomega.BeTrue()) } @@ -450,13 +450,12 @@ var _ = ginkgo.Describe("load tests vm", func() { ginkgo.By("producing blks", func() { start := time.Now() - requiredBlocks := txs / maxTxsPerBlock - if txs%maxTxsPerBlock > 0 { - requiredBlocks++ - } - for i := 0; i < requiredBlocks; i++ { + for { blk := produceBlock(instances[0]) - log.Debug("block produced", zap.Int("txs", len(blk.Txs))) + if blk == nil { + break + } + log.Debug("block produced", zap.Uint64("height", blk.Hght), zap.Int("txs", len(blk.Txs))) for _, tx := range blk.Txs { delete(allTxs, tx.ID()) } @@ -482,7 +481,7 @@ var _ = ginkgo.Describe("load tests vm", func() { func issueSimpleTx( i *instance, - to crypto.PublicKey, + to ed25519.PublicKey, amount uint64, factory chain.AuthFactory, ) (ids.ID, error) { @@ -510,6 +509,9 @@ func produceBlock(i *instance) *chain.StatelessBlock { ctx := context.TODO() blk, err := i.vm.BuildBlock(ctx) + if errors.Is(err, chain.ErrNoTxs) { + return nil + } gomega.Ω(err).To(gomega.BeNil()) gomega.Ω(blk).To(gomega.Not(gomega.BeNil())) diff --git a/examples/morpheusvm/utils/utils.go b/examples/morpheusvm/utils/utils.go index 022e59ae16..5257835624 100644 --- a/examples/morpheusvm/utils/utils.go +++ b/examples/morpheusvm/utils/utils.go @@ -4,15 +4,15 @@ package utils import ( - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/morpheusvm/consts" ) -func Address(pk crypto.PublicKey) string { - return crypto.Address(consts.HRP, pk) +func Address(pk ed25519.PublicKey) string { + return ed25519.Address(consts.HRP, pk) } -func ParseAddress(s string) (crypto.PublicKey, error) { - return crypto.ParseAddress(consts.HRP, s) +func ParseAddress(s string) (ed25519.PublicKey, error) { + return ed25519.ParseAddress(consts.HRP, s) } diff --git a/examples/tokenvm/actions/export_asset.go b/examples/tokenvm/actions/export_asset.go index dbda2e403f..6459bb5345 100644 --- a/examples/tokenvm/actions/export_asset.go +++ b/examples/tokenvm/actions/export_asset.go @@ -13,13 +13,13 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" ) -const exportAssetSize = crypto.PublicKeyLen + consts.IDLen + +const exportAssetSize = ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len + consts.BoolLen + consts.Uint64Len + /* op bits */ consts.Uint64Len + consts.Uint64Len + consts.IDLen + consts.Uint64Len + @@ -28,16 +28,16 @@ const exportAssetSize = crypto.PublicKeyLen + consts.IDLen + var _ chain.Action = (*ExportAsset)(nil) type ExportAsset struct { - To crypto.PublicKey `json:"to"` - Asset ids.ID `json:"asset"` - Value uint64 `json:"value"` - Return bool `json:"return"` - Reward uint64 `json:"reward"` - SwapIn uint64 `json:"swapIn"` - AssetOut ids.ID `json:"assetOut"` - SwapOut uint64 `json:"swapOut"` - SwapExpiry int64 `json:"swapExpiry"` - Destination ids.ID `json:"destination"` + To ed25519.PublicKey `json:"to"` + Asset ids.ID `json:"asset"` + Value uint64 `json:"value"` + Return bool `json:"return"` + Reward uint64 `json:"reward"` + SwapIn uint64 `json:"swapIn"` + AssetOut ids.ID `json:"assetOut"` + SwapOut uint64 `json:"swapOut"` + SwapExpiry int64 `json:"swapExpiry"` + Destination ids.ID `json:"destination"` } func (*ExportAsset) GetTypeID() uint8 { @@ -69,7 +69,7 @@ func (e *ExportAsset) executeReturn( ctx context.Context, r chain.Rules, db chain.Database, - actor crypto.PublicKey, + actor ed25519.PublicKey, txID ids.ID, ) (*chain.Result, error) { unitsUsed := e.MaxUnits(r) @@ -99,7 +99,7 @@ func (e *ExportAsset) executeReturn( return &chain.Result{Success: false, Units: unitsUsed, Output: utils.ErrBytes(err)}, nil } if newSupply > 0 { - if err := storage.SetAsset(ctx, db, e.Asset, metadata, newSupply, crypto.EmptyPublicKey, true); err != nil { + if err := storage.SetAsset(ctx, db, e.Asset, metadata, newSupply, ed25519.EmptyPublicKey, true); err != nil { return &chain.Result{Success: false, Units: unitsUsed, Output: utils.ErrBytes(err)}, nil } } else { @@ -147,7 +147,7 @@ func (e *ExportAsset) executeLoan( ctx context.Context, r chain.Rules, db chain.Database, - actor crypto.PublicKey, + actor ed25519.PublicKey, txID ids.ID, ) (*chain.Result, error) { unitsUsed := e.MaxUnits(r) diff --git a/examples/tokenvm/actions/fill_order.go b/examples/tokenvm/actions/fill_order.go index 8e69e6fba6..0c3128d601 100644 --- a/examples/tokenvm/actions/fill_order.go +++ b/examples/tokenvm/actions/fill_order.go @@ -13,7 +13,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -22,7 +22,7 @@ import ( var _ chain.Action = (*FillOrder)(nil) const ( - basePrice = 3*consts.IDLen + consts.Uint64Len + crypto.PublicKeyLen + basePrice = 3*consts.IDLen + consts.Uint64Len + ed25519.PublicKeyLen tradeSucceededPrice = 1_000 ) @@ -32,7 +32,7 @@ type FillOrder struct { // [Owner] is the owner of the order and the recipient of the trade // proceeds. - Owner crypto.PublicKey `json:"owner"` + Owner ed25519.PublicKey `json:"owner"` // [In] is the asset that will be sent to the owner from the fill. We need to provide this to // populate [StateKeys]. @@ -164,7 +164,7 @@ func (*FillOrder) MaxUnits(chain.Rules) uint64 { } func (*FillOrder) Size() int { - return consts.IDLen*3 + crypto.PublicKeyLen + consts.Uint64Len + return consts.IDLen*3 + ed25519.PublicKeyLen + consts.Uint64Len } func (f *FillOrder) Marshal(p *codec.Packer) { diff --git a/examples/tokenvm/actions/import_asset.go b/examples/tokenvm/actions/import_asset.go index adb65465b3..0a84cd4ab8 100644 --- a/examples/tokenvm/actions/import_asset.go +++ b/examples/tokenvm/actions/import_asset.go @@ -13,7 +13,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -77,7 +77,7 @@ func (i *ImportAsset) StateKeys(rauth chain.Auth, _ ids.ID) [][]byte { func (i *ImportAsset) executeMint( ctx context.Context, db chain.Database, - actor crypto.PublicKey, + actor ed25519.PublicKey, ) []byte { asset := ImportedAssetID(i.warpTransfer.Asset, i.warpMessage.SourceChainID) exists, metadata, supply, _, warp, err := storage.GetAsset(ctx, db, asset) @@ -99,7 +99,7 @@ func (i *ImportAsset) executeMint( if err != nil { return utils.ErrBytes(err) } - if err := storage.SetAsset(ctx, db, asset, metadata, newSupply, crypto.EmptyPublicKey, true); err != nil { + if err := storage.SetAsset(ctx, db, asset, metadata, newSupply, ed25519.EmptyPublicKey, true); err != nil { return utils.ErrBytes(err) } if err := storage.AddBalance(ctx, db, i.warpTransfer.To, asset, i.warpTransfer.Value); err != nil { @@ -116,7 +116,7 @@ func (i *ImportAsset) executeMint( func (i *ImportAsset) executeReturn( ctx context.Context, db chain.Database, - actor crypto.PublicKey, + actor ed25519.PublicKey, ) []byte { if err := storage.SubLoan( ctx, db, i.warpTransfer.Asset, diff --git a/examples/tokenvm/actions/mint_asset.go b/examples/tokenvm/actions/mint_asset.go index e7cb70e9fe..7ac22eb571 100644 --- a/examples/tokenvm/actions/mint_asset.go +++ b/examples/tokenvm/actions/mint_asset.go @@ -13,7 +13,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -23,7 +23,7 @@ var _ chain.Action = (*MintAsset)(nil) type MintAsset struct { // To is the recipient of the [Value]. - To crypto.PublicKey `json:"to"` + To ed25519.PublicKey `json:"to"` // Asset is the [TxID] that created the asset. Asset ids.ID `json:"asset"` @@ -93,11 +93,11 @@ func (m *MintAsset) Execute( func (*MintAsset) MaxUnits(chain.Rules) uint64 { // We use size as the price of this transaction but we could just as easily // use any other calculation. - return crypto.PublicKeyLen + consts.IDLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len } func (*MintAsset) Size() int { - return crypto.PublicKeyLen + consts.IDLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len } func (m *MintAsset) Marshal(p *codec.Packer) { diff --git a/examples/tokenvm/actions/modify_asset.go b/examples/tokenvm/actions/modify_asset.go index 06f529c0d9..c201e2b14b 100644 --- a/examples/tokenvm/actions/modify_asset.go +++ b/examples/tokenvm/actions/modify_asset.go @@ -11,7 +11,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -27,7 +27,7 @@ type ModifyAsset struct { // // If you want to retain ownership, set this to the signer. If you want to // revoke ownership, set this to another key or the empty public key. - Owner crypto.PublicKey `json:"owner"` + Owner ed25519.PublicKey `json:"owner"` // Metadata is the new metadata of the [Asset]. // @@ -86,11 +86,11 @@ func (m *ModifyAsset) Execute( func (m *ModifyAsset) MaxUnits(chain.Rules) uint64 { // We use size as the price of this transaction but we could just as easily // use any other calculation. - return consts.IDLen + crypto.PublicKeyLen + uint64(len(m.Metadata)) + return consts.IDLen + ed25519.PublicKeyLen + uint64(len(m.Metadata)) } func (m *ModifyAsset) Size() int { - return consts.IDLen + crypto.PublicKeyLen + codec.BytesLen(m.Metadata) + return consts.IDLen + ed25519.PublicKeyLen + codec.BytesLen(m.Metadata) } func (m *ModifyAsset) Marshal(p *codec.Packer) { diff --git a/examples/tokenvm/actions/transfer.go b/examples/tokenvm/actions/transfer.go index e587e35c1c..106995f0dd 100644 --- a/examples/tokenvm/actions/transfer.go +++ b/examples/tokenvm/actions/transfer.go @@ -11,7 +11,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/utils" @@ -21,7 +21,7 @@ var _ chain.Action = (*Transfer)(nil) type Transfer struct { // To is the recipient of the [Value]. - To crypto.PublicKey `json:"to"` + To ed25519.PublicKey `json:"to"` // Asset to transfer to [To]. Asset ids.ID @@ -67,11 +67,11 @@ func (t *Transfer) Execute( func (*Transfer) MaxUnits(chain.Rules) uint64 { // We use size as the price of this transaction but we could just as easily // use any other calculation. - return crypto.PublicKeyLen + consts.IDLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len } func (*Transfer) Size() int { - return crypto.PublicKeyLen + consts.IDLen + consts.Uint64Len + return ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len } func (t *Transfer) Marshal(p *codec.Packer) { diff --git a/examples/tokenvm/actions/warp_transfer.go b/examples/tokenvm/actions/warp_transfer.go index 6855be3584..38925201aa 100644 --- a/examples/tokenvm/actions/warp_transfer.go +++ b/examples/tokenvm/actions/warp_transfer.go @@ -8,20 +8,20 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/utils" ) -const WarpTransferSize = crypto.PublicKeyLen + consts.IDLen + +const WarpTransferSize = ed25519.PublicKeyLen + consts.IDLen + consts.Uint64Len + consts.BoolLen + consts.Uint64Len + /* op bits */ consts.Uint64Len + consts.Uint64Len + consts.IDLen + consts.Uint64Len + consts.Int64Len + consts.IDLen + consts.IDLen type WarpTransfer struct { - To crypto.PublicKey `json:"to"` - Asset ids.ID `json:"asset"` - Value uint64 `json:"value"` + To ed25519.PublicKey `json:"to"` + Asset ids.ID `json:"asset"` + Value uint64 `json:"value"` // Return is set to true when a warp message is sending funds back to the // chain where they were created. diff --git a/examples/tokenvm/auth/common.go b/examples/tokenvm/auth/common.go index cf838e454d..67596fbc35 100644 --- a/examples/tokenvm/auth/common.go +++ b/examples/tokenvm/auth/common.go @@ -3,7 +3,15 @@ package auth +import "github.com/ava-labs/hypersdk/vm" + // Note: Registry will error during initialization if a duplicate ID is assigned. We explicitly assign IDs to avoid accidental remapping. const ( ed25519ID uint8 = 0 ) + +func Engines() map[uint8]vm.AuthEngine { + return map[uint8]vm.AuthEngine{ + ed25519ID: &ED25519AuthEngine{}, + } +} diff --git a/examples/tokenvm/auth/ed25519.go b/examples/tokenvm/auth/ed25519.go index 4a4dc44a12..8996123850 100644 --- a/examples/tokenvm/auth/ed25519.go +++ b/examples/tokenvm/auth/ed25519.go @@ -7,18 +7,19 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" ) var _ chain.Auth = (*ED25519)(nil) type ED25519 struct { - Signer crypto.PublicKey `json:"signer"` - Signature crypto.Signature `json:"signature"` + Signer ed25519.PublicKey `json:"signer"` + Signature ed25519.Signature `json:"signature"` } func (*ED25519) GetTypeID() uint8 { @@ -28,7 +29,7 @@ func (*ED25519) GetTypeID() uint8 { func (*ED25519) MaxUnits( chain.Rules, ) uint64 { - return crypto.PublicKeyLen + crypto.SignatureLen*5 // make signatures more expensive + return ed25519.PublicKeyLen + ed25519.SignatureLen*5 // make signatures more expensive } func (*ED25519) ValidRange(chain.Rules) (int64, int64) { @@ -43,7 +44,7 @@ func (d *ED25519) StateKeys() [][]byte { } func (d *ED25519) AsyncVerify(msg []byte) error { - if !crypto.Verify(msg, d.Signer, d.Signature) { + if !ed25519.Verify(msg, d.Signer, d.Signature) { return ErrInvalidSignature } return nil @@ -65,7 +66,7 @@ func (d *ED25519) Payer() []byte { } func (*ED25519) Size() int { - return crypto.PublicKeyLen + crypto.SignatureLen + return ed25519.PublicKeyLen + ed25519.SignatureLen } func (d *ED25519) Marshal(p *codec.Packer) { @@ -113,15 +114,67 @@ func (d *ED25519) Refund( var _ chain.AuthFactory = (*ED25519Factory)(nil) -func NewED25519Factory(priv crypto.PrivateKey) *ED25519Factory { +func NewED25519Factory(priv ed25519.PrivateKey) *ED25519Factory { return &ED25519Factory{priv} } type ED25519Factory struct { - priv crypto.PrivateKey + priv ed25519.PrivateKey } func (d *ED25519Factory) Sign(msg []byte, _ chain.Action) (chain.Auth, error) { - sig := crypto.Sign(msg, d.priv) + sig := ed25519.Sign(msg, d.priv) return &ED25519{d.priv.PublicKey(), sig}, nil } + +// TODO: unify this with MorpheusVM +type ED25519AuthEngine struct{} + +func (*ED25519AuthEngine) GetBatchVerifier(cores int, count int) chain.AuthBatchVerifier { + batchSize := math.Max(count/cores, 16) + return &ED25519Batch{ + batchSize: batchSize, + total: count, + } +} + +func (*ED25519AuthEngine) Cache(auth chain.Auth) { + pk := GetSigner(auth) + ed25519.CachePublicKey(pk) +} + +type ED25519Batch struct { + batchSize int + total int + + counter int + totalCounter int + batch *ed25519.Batch +} + +func (b *ED25519Batch) Add(msg []byte, rauth chain.Auth) func() error { + auth := rauth.(*ED25519) + if b.batch == nil { + b.batch = ed25519.NewBatch(b.batchSize) + } + b.batch.Add(msg, auth.Signer, auth.Signature) + b.counter++ + b.totalCounter++ + if b.counter == b.batchSize { + last := b.batch + b.counter = 0 + if b.totalCounter < b.total { + // don't create a new batch if we are done + b.batch = ed25519.NewBatch(b.batchSize) + } + return last.VerifyAsync() + } + return nil +} + +func (b *ED25519Batch) Done() []func() error { + if b.batch == nil { + return nil + } + return []func() error{b.batch.VerifyAsync()} +} diff --git a/examples/tokenvm/auth/helpers.go b/examples/tokenvm/auth/helpers.go index 2b73e20cca..340f44f367 100644 --- a/examples/tokenvm/auth/helpers.go +++ b/examples/tokenvm/auth/helpers.go @@ -5,23 +5,23 @@ package auth import ( "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" ) -func GetActor(auth chain.Auth) crypto.PublicKey { +func GetActor(auth chain.Auth) ed25519.PublicKey { switch a := auth.(type) { case *ED25519: return a.Signer default: - return crypto.EmptyPublicKey + return ed25519.EmptyPublicKey } } -func GetSigner(auth chain.Auth) crypto.PublicKey { +func GetSigner(auth chain.Auth) ed25519.PublicKey { switch a := auth.(type) { case *ED25519: return a.Signer default: - return crypto.EmptyPublicKey + return ed25519.EmptyPublicKey } } diff --git a/examples/tokenvm/cmd/token-cli/cmd/action.go b/examples/tokenvm/cmd/token-cli/cmd/action.go index af5a6e7dc7..2f5387deb1 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/action.go +++ b/examples/tokenvm/cmd/token-cli/cmd/action.go @@ -14,7 +14,7 @@ import ( "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" trpc "github.com/ava-labs/hypersdk/examples/tokenvm/rpc" "github.com/ava-labs/hypersdk/examples/tokenvm/utils" @@ -419,7 +419,7 @@ func performImport( dcli *rpc.JSONRPCClient, dtcli *trpc.JSONRPCClient, exportTxID ids.ID, - priv crypto.PrivateKey, + priv ed25519.PrivateKey, factory chain.AuthFactory, ) error { // Select TxID (if not provided) diff --git a/examples/tokenvm/cmd/token-cli/cmd/handler.go b/examples/tokenvm/cmd/token-cli/cmd/handler.go index 8a967d35e9..71678eb9d8 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/handler.go +++ b/examples/tokenvm/cmd/token-cli/cmd/handler.go @@ -9,7 +9,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/cli" hconsts "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/consts" trpc "github.com/ava-labs/hypersdk/examples/tokenvm/rpc" @@ -35,7 +35,7 @@ func (h *Handler) Root() *cli.Handler { func (h *Handler) GetAssetInfo( ctx context.Context, cli *trpc.JSONRPCClient, - publicKey crypto.PublicKey, + publicKey ed25519.PublicKey, assetID ids.ID, checkBalance bool, ) (uint64, ids.ID, error) { @@ -91,21 +91,21 @@ func (h *Handler) GetAssetInfo( } func (h *Handler) DefaultActor() ( - ids.ID, crypto.PrivateKey, *auth.ED25519Factory, + ids.ID, ed25519.PrivateKey, *auth.ED25519Factory, *rpc.JSONRPCClient, *trpc.JSONRPCClient, error, ) { priv, err := h.h.GetDefaultKey() if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } chainID, uris, err := h.h.GetDefaultChain() if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } cli := rpc.NewJSONRPCClient(uris[0]) networkID, _, _, err := cli.Network(context.TODO()) if err != nil { - return ids.Empty, crypto.EmptyPrivateKey, nil, nil, nil, err + return ids.Empty, ed25519.EmptyPrivateKey, nil, nil, nil, err } // For [defaultActor], we always send requests to the first returned URI. return chainID, priv, auth.NewED25519Factory( @@ -134,10 +134,10 @@ func (*Controller) Symbol() string { return consts.Symbol } -func (*Controller) Address(pk crypto.PublicKey) string { +func (*Controller) Address(pk ed25519.PublicKey) string { return utils.Address(pk) } -func (*Controller) ParseAddress(address string) (crypto.PublicKey, error) { +func (*Controller) ParseAddress(address string) (ed25519.PublicKey, error) { return utils.ParseAddress(address) } diff --git a/examples/tokenvm/cmd/token-cli/cmd/key.go b/examples/tokenvm/cmd/token-cli/cmd/key.go index 26f248f3bf..f5ad535ac9 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/key.go +++ b/examples/tokenvm/cmd/token-cli/cmd/key.go @@ -7,7 +7,7 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" hutils "github.com/ava-labs/hypersdk/utils" "github.com/spf13/cobra" @@ -65,7 +65,7 @@ var setKeyCmd = &cobra.Command{ }, } -func lookupKeyBalance(pk crypto.PublicKey, uri string, networkID uint32, chainID ids.ID, assetID ids.ID) error { +func lookupKeyBalance(pk ed25519.PublicKey, uri string, networkID uint32, chainID ids.ID, assetID ids.ID) error { _, _, err := handler.GetAssetInfo( context.TODO(), trpc.NewJSONRPCClient(uri, networkID, chainID), pk, assetID, true) diff --git a/examples/tokenvm/cmd/token-cli/cmd/spam.go b/examples/tokenvm/cmd/token-cli/cmd/spam.go index a7210669fe..c828c2e41a 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/spam.go +++ b/examples/tokenvm/cmd/token-cli/cmd/spam.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" trpc "github.com/ava-labs/hypersdk/examples/tokenvm/rpc" @@ -32,7 +32,7 @@ var runSpamCmd = &cobra.Command{ func(uri string, networkID uint32, chainID ids.ID) { tclient = trpc.NewJSONRPCClient(uri, networkID, chainID) }, - func(pk crypto.PrivateKey) chain.AuthFactory { + func(pk ed25519.PrivateKey) chain.AuthFactory { return auth.NewED25519Factory(pk) }, func(choice int, address string) (uint64, error) { @@ -52,14 +52,14 @@ var runSpamCmd = &cobra.Command{ func(ctx context.Context, chainID ids.ID) (chain.Parser, error) { return tclient.Parser(ctx) }, - func(pk crypto.PublicKey, amount uint64) chain.Action { + func(pk ed25519.PublicKey, amount uint64) chain.Action { return &actions.Transfer{ To: pk, Asset: ids.Empty, Value: amount, } }, - func(cli *rpc.JSONRPCClient, pk crypto.PrivateKey) func(context.Context, uint64) error { + func(cli *rpc.JSONRPCClient, pk ed25519.PrivateKey) func(context.Context, uint64) error { return func(ictx context.Context, count uint64) error { _, _, err := sendAndWait(ictx, nil, &actions.Transfer{ To: pk.PublicKey(), diff --git a/examples/tokenvm/controller/controller.go b/examples/tokenvm/controller/controller.go index 634738b72e..4f298d0da7 100644 --- a/examples/tokenvm/controller/controller.go +++ b/examples/tokenvm/controller/controller.go @@ -68,6 +68,7 @@ func (c *Controller) Initialize( vm.Handlers, chain.ActionRegistry, chain.AuthRegistry, + map[uint8]vm.AuthEngine, error, ) { c.inner = inner @@ -78,20 +79,20 @@ func (c *Controller) Initialize( var err error c.metrics, err = newMetrics(gatherer) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } // Load config and genesis c.config, err = config.New(c.snowCtx.NodeID, configBytes) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } c.snowCtx.Log.SetLevel(c.config.GetLogLevel()) snowCtx.Log.Info("initialized config", zap.Bool("loaded", c.config.Loaded()), zap.Any("contents", c.config)) c.genesis, err = genesis.New(genesisBytes, upgradeBytes) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, fmt.Errorf( + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, fmt.Errorf( "unable to read genesis: %w", err, ) @@ -101,7 +102,7 @@ func (c *Controller) Initialize( // Create DBs blockDB, stateDB, metaDB, err := hstorage.New(snowCtx.ChainDataDir, gatherer) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } c.metaDB = metaDB @@ -116,7 +117,7 @@ func (c *Controller) Initialize( common.NoLock, ) if err != nil { - return nil, nil, nil, nil, nil, nil, nil, nil, nil, err + return nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, err } apis[rpc.JSONRPCEndpoint] = jsonRPCHandler @@ -143,7 +144,7 @@ func (c *Controller) Initialize( // Initialize order book used to track all open orders c.orderBook = orderbook.New(c, c.config.TrackedPairs) - return c.config, c.genesis, build, gossip, blockDB, stateDB, apis, consts.ActionRegistry, consts.AuthRegistry, nil + return c.config, c.genesis, build, gossip, blockDB, stateDB, apis, consts.ActionRegistry, consts.AuthRegistry, auth.Engines(), nil } func (c *Controller) Rules(t int64) chain.Rules { diff --git a/examples/tokenvm/controller/resolutions.go b/examples/tokenvm/controller/resolutions.go index f9915939cc..a9447cc062 100644 --- a/examples/tokenvm/controller/resolutions.go +++ b/examples/tokenvm/controller/resolutions.go @@ -9,7 +9,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/trace" "github.com/ava-labs/avalanchego/utils/logging" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/genesis" "github.com/ava-labs/hypersdk/examples/tokenvm/orderbook" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" @@ -37,13 +37,13 @@ func (c *Controller) GetTransaction( func (c *Controller) GetAssetFromState( ctx context.Context, asset ids.ID, -) (bool, []byte, uint64, crypto.PublicKey, bool, error) { +) (bool, []byte, uint64, ed25519.PublicKey, bool, error) { return storage.GetAssetFromState(ctx, c.inner.ReadState, asset) } func (c *Controller) GetBalanceFromState( ctx context.Context, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, ) (uint64, error) { return storage.GetBalanceFromState(ctx, c.inner.ReadState, pk, asset) diff --git a/examples/tokenvm/genesis/genesis.go b/examples/tokenvm/genesis/genesis.go index a01456fdb3..b140d032f0 100644 --- a/examples/tokenvm/genesis/genesis.go +++ b/examples/tokenvm/genesis/genesis.go @@ -14,7 +14,7 @@ import ( "github.com/ava-labs/hypersdk/chain" hconsts "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/consts" "github.com/ava-labs/hypersdk/examples/tokenvm/storage" "github.com/ava-labs/hypersdk/examples/tokenvm/utils" @@ -119,7 +119,7 @@ func (g *Genesis) Load(ctx context.Context, tracer trace.Tracer, db chain.Databa ids.Empty, []byte(consts.Symbol), supply, - crypto.EmptyPublicKey, + ed25519.EmptyPublicKey, false, ) } diff --git a/examples/tokenvm/go.mod b/examples/tokenvm/go.mod index 2a3578114b..d83a0df087 100644 --- a/examples/tokenvm/go.mod +++ b/examples/tokenvm/go.mod @@ -83,6 +83,7 @@ require ( github.com/mitchellh/pointerstructure v1.2.0 // indirect github.com/mr-tron/base58 v1.2.0 // indirect github.com/nbutton23/zxcvbn-go v0.0.0-20180912185939-ae427f1e4c1d // indirect + github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce // indirect github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/openzipkin/zipkin-go v0.4.1 // indirect github.com/otiai10/copy v1.11.0 // indirect diff --git a/examples/tokenvm/go.sum b/examples/tokenvm/go.sum index 833d63acbb..9f2df9642c 100644 --- a/examples/tokenvm/go.sum +++ b/examples/tokenvm/go.sum @@ -442,6 +442,8 @@ github.com/neilotoole/errgroup v0.1.6 h1:PODGqPXdT5BC/zCYIMoTrwV+ujKcW+gBXM6Ye9V github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce h1:/pEpMk55wH0X+E5zedGEMOdLuWmV8P4+4W3+LZaM6kg= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce/go.mod h1:hVoHR2EVESiICEMbg137etN/Lx+lSrHPTD39Z/uE+2s= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= diff --git a/examples/tokenvm/orderbook/orderbook.go b/examples/tokenvm/orderbook/orderbook.go index f8d1cb2a01..d570a80a46 100644 --- a/examples/tokenvm/orderbook/orderbook.go +++ b/examples/tokenvm/orderbook/orderbook.go @@ -7,7 +7,7 @@ import ( "sync" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" "github.com/ava-labs/hypersdk/examples/tokenvm/utils" "github.com/ava-labs/hypersdk/heap" @@ -26,7 +26,7 @@ type Order struct { OutTick uint64 `json:"outTick"` Remaining uint64 `json:"remaining"` - owner crypto.PublicKey + owner ed25519.PublicKey } type OrderBook struct { @@ -63,7 +63,7 @@ func New(c Controller, trackedPairs []string) *OrderBook { } } -func (o *OrderBook) Add(txID ids.ID, actor crypto.PublicKey, action *actions.CreateOrder) { +func (o *OrderBook) Add(txID ids.ID, actor ed25519.PublicKey, action *actions.CreateOrder) { pair := actions.PairID(action.In, action.Out) order := &Order{ txID, diff --git a/examples/tokenvm/rpc/dependencies.go b/examples/tokenvm/rpc/dependencies.go index 1c2f8996ae..7279497872 100644 --- a/examples/tokenvm/rpc/dependencies.go +++ b/examples/tokenvm/rpc/dependencies.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/trace" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/genesis" "github.com/ava-labs/hypersdk/examples/tokenvm/orderbook" ) @@ -17,8 +17,8 @@ type Controller interface { Genesis() *genesis.Genesis Tracer() trace.Tracer GetTransaction(context.Context, ids.ID) (bool, int64, bool, uint64, error) - GetAssetFromState(context.Context, ids.ID) (bool, []byte, uint64, crypto.PublicKey, bool, error) - GetBalanceFromState(context.Context, crypto.PublicKey, ids.ID) (uint64, error) + GetAssetFromState(context.Context, ids.ID) (bool, []byte, uint64, ed25519.PublicKey, bool, error) + GetBalanceFromState(context.Context, ed25519.PublicKey, ids.ID) (uint64, error) Orders(pair string, limit int) []*orderbook.Order GetLoanFromState(context.Context, ids.ID, ids.ID) (uint64, error) } diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index bc30c8e291..75fdd1125e 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -15,7 +15,7 @@ import ( smath "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/utils" ) @@ -58,7 +58,7 @@ var ( // TODO: extend to other types balancePrefixPool = sync.Pool{ New: func() any { - return make([]byte, 1+crypto.PublicKeyLen+consts.IDLen) + return make([]byte, 1+ed25519.PublicKeyLen+consts.IDLen) }, } ) @@ -115,11 +115,11 @@ func GetTransaction( } // [accountPrefix] + [address] + [asset] -func PrefixBalanceKey(pk crypto.PublicKey, asset ids.ID) (k []byte) { +func PrefixBalanceKey(pk ed25519.PublicKey, asset ids.ID) (k []byte) { k = balancePrefixPool.Get().([]byte) k[0] = balancePrefix copy(k[1:], pk[:]) - copy(k[1+crypto.PublicKeyLen:], asset[:]) + copy(k[1+ed25519.PublicKeyLen:], asset[:]) return } @@ -127,7 +127,7 @@ func PrefixBalanceKey(pk crypto.PublicKey, asset ids.ID) (k []byte) { func GetBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, ) (uint64, error) { dbKey, bal, err := getBalance(ctx, db, pk, asset) @@ -138,7 +138,7 @@ func GetBalance( func getBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, ) ([]byte, uint64, error) { k := PrefixBalanceKey(pk, asset) @@ -150,7 +150,7 @@ func getBalance( func GetBalanceFromState( ctx context.Context, f ReadState, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, ) (uint64, error) { k := PrefixBalanceKey(pk, asset) @@ -176,7 +176,7 @@ func innerGetBalance( func SetBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, balance uint64, ) error { @@ -196,7 +196,7 @@ func setBalance( func DeleteBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, ) error { return db.Remove(ctx, PrefixBalanceKey(pk, asset)) @@ -205,7 +205,7 @@ func DeleteBalance( func AddBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, amount uint64, ) error { @@ -230,7 +230,7 @@ func AddBalance( func SubBalance( ctx context.Context, db chain.Database, - pk crypto.PublicKey, + pk ed25519.PublicKey, asset ids.ID, amount uint64, ) error { @@ -270,7 +270,7 @@ func GetAssetFromState( ctx context.Context, f ReadState, asset ids.ID, -) (bool, []byte, uint64, crypto.PublicKey, bool, error) { +) (bool, []byte, uint64, ed25519.PublicKey, bool, error) { values, errs := f(ctx, [][]byte{PrefixAssetKey(asset)}) return innerGetAsset(values[0], errs[0]) } @@ -279,7 +279,7 @@ func GetAsset( ctx context.Context, db chain.Database, asset ids.ID, -) (bool, []byte, uint64, crypto.PublicKey, bool, error) { +) (bool, []byte, uint64, ed25519.PublicKey, bool, error) { k := PrefixAssetKey(asset) return innerGetAsset(db.GetValue(ctx, k)) } @@ -287,19 +287,19 @@ func GetAsset( func innerGetAsset( v []byte, err error, -) (bool, []byte, uint64, crypto.PublicKey, bool, error) { +) (bool, []byte, uint64, ed25519.PublicKey, bool, error) { if errors.Is(err, database.ErrNotFound) { - return false, nil, 0, crypto.EmptyPublicKey, false, nil + return false, nil, 0, ed25519.EmptyPublicKey, false, nil } if err != nil { - return false, nil, 0, crypto.EmptyPublicKey, false, err + return false, nil, 0, ed25519.EmptyPublicKey, false, err } metadataLen := binary.BigEndian.Uint16(v) metadata := v[consts.Uint16Len : consts.Uint16Len+metadataLen] supply := binary.BigEndian.Uint64(v[consts.Uint16Len+metadataLen:]) - var pk crypto.PublicKey + var pk ed25519.PublicKey copy(pk[:], v[consts.Uint16Len+metadataLen+consts.Uint64Len:]) - warp := v[consts.Uint16Len+metadataLen+consts.Uint64Len+crypto.PublicKeyLen] == 0x1 + warp := v[consts.Uint16Len+metadataLen+consts.Uint64Len+ed25519.PublicKeyLen] == 0x1 return true, metadata, supply, pk, warp, nil } @@ -309,12 +309,12 @@ func SetAsset( asset ids.ID, metadata []byte, supply uint64, - owner crypto.PublicKey, + owner ed25519.PublicKey, warp bool, ) error { k := PrefixAssetKey(asset) metadataLen := len(metadata) - v := make([]byte, consts.Uint16Len+metadataLen+consts.Uint64Len+crypto.PublicKeyLen+1) + v := make([]byte, consts.Uint16Len+metadataLen+consts.Uint64Len+ed25519.PublicKeyLen+1) binary.BigEndian.PutUint16(v, uint16(metadataLen)) copy(v[consts.Uint16Len:], metadata) binary.BigEndian.PutUint64(v[consts.Uint16Len+metadataLen:], supply) @@ -323,7 +323,7 @@ func SetAsset( if warp { b = 0x1 } - v[consts.Uint16Len+metadataLen+consts.Uint64Len+crypto.PublicKeyLen] = b + v[consts.Uint16Len+metadataLen+consts.Uint64Len+ed25519.PublicKeyLen] = b return db.Insert(ctx, k, v) } @@ -349,10 +349,10 @@ func SetOrder( out ids.ID, outTick uint64, supply uint64, - owner crypto.PublicKey, + owner ed25519.PublicKey, ) error { k := PrefixOrderKey(txID) - v := make([]byte, consts.IDLen*2+consts.Uint64Len*3+crypto.PublicKeyLen) + v := make([]byte, consts.IDLen*2+consts.Uint64Len*3+ed25519.PublicKeyLen) copy(v, in[:]) binary.BigEndian.PutUint64(v[consts.IDLen:], inTick) copy(v[consts.IDLen+consts.Uint64Len:], out[:]) @@ -373,16 +373,16 @@ func GetOrder( ids.ID, // out uint64, // outTick uint64, // remaining - crypto.PublicKey, // owner + ed25519.PublicKey, // owner error, ) { k := PrefixOrderKey(order) v, err := db.GetValue(ctx, k) if errors.Is(err, database.ErrNotFound) { - return false, ids.Empty, 0, ids.Empty, 0, 0, crypto.EmptyPublicKey, nil + return false, ids.Empty, 0, ids.Empty, 0, 0, ed25519.EmptyPublicKey, nil } if err != nil { - return false, ids.Empty, 0, ids.Empty, 0, 0, crypto.EmptyPublicKey, err + return false, ids.Empty, 0, ids.Empty, 0, 0, ed25519.EmptyPublicKey, err } var in ids.ID copy(in[:], v[:consts.IDLen]) @@ -391,7 +391,7 @@ func GetOrder( copy(out[:], v[consts.IDLen+consts.Uint64Len:consts.IDLen*2+consts.Uint64Len]) outTick := binary.BigEndian.Uint64(v[consts.IDLen*2+consts.Uint64Len:]) supply := binary.BigEndian.Uint64(v[consts.IDLen*2+consts.Uint64Len*2:]) - var owner crypto.PublicKey + var owner ed25519.PublicKey copy(owner[:], v[consts.IDLen*2+consts.Uint64Len*3:]) return true, in, inTick, out, outTick, supply, owner, nil } diff --git a/examples/tokenvm/tests/e2e/e2e_test.go b/examples/tokenvm/tests/e2e/e2e_test.go index 6d0bf44fd5..f4940f99b4 100644 --- a/examples/tokenvm/tests/e2e/e2e_test.go +++ b/examples/tokenvm/tests/e2e/e2e_test.go @@ -17,7 +17,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/vms/platformvm/warp" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/consts" @@ -392,7 +392,7 @@ var _ = ginkgo.BeforeSuite(func() { } // Load default pk - priv, err = crypto.HexToKey( + priv, err = ed25519.HexToKey( "323b1d8f4eed5f0da9da93071b034f2dce9d2d22692c172f3cb252a64ddfafd01b057de320297c29ad0c1f589ea216869cf1938d88c9fbd70d6748323dbf2fa7", //nolint:lll ) gomega.Ω(err).Should(gomega.BeNil()) @@ -403,9 +403,9 @@ var _ = ginkgo.BeforeSuite(func() { }) var ( - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string instancesA []instance @@ -503,7 +503,7 @@ var _ = ginkgo.Describe("[Test]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(nativeBalance).Should(gomega.Equal(uint64(1000000000000))) - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) aother := utils.Address(other.PublicKey()) @@ -571,7 +571,7 @@ var _ = ginkgo.Describe("[Test]", func() { }) ginkgo.It("performs a warp transfer of the native asset", func() { - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) aother := utils.Address(other.PublicKey()) source, err := ids.FromString(blockchainIDA) @@ -793,7 +793,7 @@ var _ = ginkgo.Describe("[Test]", func() { gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(metadata).Should(gomega.Equal(actions.ImportedAssetMetadata(ids.Empty, bIDA))) gomega.Ω(supply).Should(gomega.Equal(sendAmount)) - gomega.Ω(owner).Should(gomega.Equal(utils.Address(crypto.EmptyPublicKey))) + gomega.Ω(owner).Should(gomega.Equal(utils.Address(ed25519.EmptyPublicKey))) gomega.Ω(warp).Should(gomega.BeTrue()) }) @@ -887,7 +887,7 @@ var _ = ginkgo.Describe("[Test]", func() { gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(metadata).Should(gomega.Equal(actions.ImportedAssetMetadata(ids.Empty, bIDA))) gomega.Ω(supply).Should(gomega.Equal(uint64(2900))) - gomega.Ω(owner).Should(gomega.Equal(utils.Address(crypto.EmptyPublicKey))) + gomega.Ω(owner).Should(gomega.Equal(utils.Address(ed25519.EmptyPublicKey))) gomega.Ω(warp).Should(gomega.BeTrue()) }) @@ -1525,7 +1525,7 @@ func generateBlocks( } for ctx.Err() == nil { // Generate transaction - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) submit, _, _, err := instances[cumulativeTxs%len(instances)].cli.GenerateTransaction( context.Background(), @@ -1591,7 +1591,7 @@ func acceptTransaction(cli *rpc.JSONRPCClient, tcli *trpc.JSONRPCClient) { gomega.Ω(err).Should(gomega.BeNil()) for { // Generate transaction - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) submit, tx, _, err := cli.GenerateTransaction( context.Background(), diff --git a/examples/tokenvm/tests/integration/integration_test.go b/examples/tokenvm/tests/integration/integration_test.go index fe5b3f2371..78261c0ac5 100644 --- a/examples/tokenvm/tests/integration/integration_test.go +++ b/examples/tokenvm/tests/integration/integration_test.go @@ -36,7 +36,7 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/pubsub" "github.com/ava-labs/hypersdk/rpc" hutils "github.com/ava-labs/hypersdk/utils" @@ -102,14 +102,14 @@ func init() { } var ( - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string - priv2 crypto.PrivateKey + priv2 ed25519.PrivateKey factory2 *auth.ED25519Factory - rsender2 crypto.PublicKey + rsender2 ed25519.PublicKey sender2 string asset1 []byte @@ -143,7 +143,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(vms).Should(gomega.BeNumerically(">", 1)) var err error - priv, err = crypto.GeneratePrivateKey() + priv, err = ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) factory = auth.NewED25519Factory(priv) rsender = priv.PublicKey() @@ -154,7 +154,7 @@ var _ = ginkgo.BeforeSuite(func() { zap.String("pk", hex.EncodeToString(priv[:])), ) - priv2, err = crypto.GeneratePrivateKey() + priv2, err = ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) factory2 = auth.NewED25519Factory(priv2) rsender2 = priv2.PublicKey() @@ -273,7 +273,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(string(metadata)).Should(gomega.Equal(tconsts.Symbol)) gomega.Ω(supply).Should(gomega.Equal(csupply)) - gomega.Ω(owner).Should(gomega.Equal(utils.Address(crypto.EmptyPublicKey))) + gomega.Ω(owner).Should(gomega.Equal(utils.Address(ed25519.EmptyPublicKey))) gomega.Ω(warp).Should(gomega.BeFalse()) } @@ -619,7 +619,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) // Send tx - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) transfer := &actions.Transfer{ To: other.PublicKey(), @@ -673,7 +673,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) // Create tx - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) transfer := &actions.Transfer{ To: other.PublicKey(), @@ -721,7 +721,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { }) ginkgo.It("mint an asset that doesn't exist", func() { - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) assetID := ids.GenerateTestID() parser, err := instances[0].tcli.Parser(context.Background()) @@ -893,7 +893,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { }) ginkgo.It("mint asset from wrong owner", func() { - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) parser, err := instances[0].tcli.Parser(context.Background()) gomega.Ω(err).Should(gomega.BeNil()) @@ -1005,7 +1005,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { }) ginkgo.It("rejects empty mint", func() { - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) tx := chain.NewTx( &chain.Base{ @@ -1089,7 +1089,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { &actions.ModifyAsset{ Asset: asset1ID, Metadata: []byte("blah"), - Owner: crypto.EmptyPublicKey, + Owner: ed25519.EmptyPublicKey, }, factory, ) @@ -1115,7 +1115,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(metadata).Should(gomega.Equal([]byte("blah"))) gomega.Ω(supply).Should(gomega.Equal(uint64(10))) - gomega.Ω(owner).Should(gomega.Equal(utils.Address(crypto.EmptyPublicKey))) + gomega.Ω(owner).Should(gomega.Equal(utils.Address(ed25519.EmptyPublicKey))) gomega.Ω(warp).Should(gomega.BeFalse()) }) @@ -1150,7 +1150,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { }) ginkgo.It("rejects mint of native token", func() { - other, err := crypto.GeneratePrivateKey() + other, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) tx := chain.NewTx( &chain.Base{ diff --git a/examples/tokenvm/tests/load/load_test.go b/examples/tokenvm/tests/load/load_test.go index 3927963e54..5ed52ae5e2 100644 --- a/examples/tokenvm/tests/load/load_test.go +++ b/examples/tokenvm/tests/load/load_test.go @@ -7,6 +7,7 @@ import ( "context" "encoding/hex" "encoding/json" + "errors" "flag" "fmt" "math/rand" @@ -35,7 +36,7 @@ import ( "github.com/ava-labs/hypersdk/chain" hconsts "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/pebble" hutils "github.com/ava-labs/hypersdk/utils" "github.com/ava-labs/hypersdk/vm" @@ -54,7 +55,6 @@ import ( const ( genesisBalance uint64 = hconsts.MaxUint64 transferTxUnits = 472 - maxTxsPerBlock int = 1_800_000 /* max block units */ / transferTxUnits ) var ( @@ -89,9 +89,9 @@ type instance struct { } type account struct { - priv crypto.PrivateKey + priv ed25519.PrivateKey factory *auth.ED25519Factory - rsender crypto.PublicKey + rsender ed25519.PublicKey sender string } @@ -164,7 +164,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(vms).Should(gomega.BeNumerically(">", 1)) var err error - priv, err := crypto.GeneratePrivateKey() + priv, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) rsender := priv.PublicKey() sender := utils.Address(rsender) @@ -178,7 +178,8 @@ var _ = ginkgo.BeforeSuite(func() { // create embedded VMs instances = make([]*instance, vms) gen = genesis.Default() - gen.WindowTargetUnits = 1_000_000_000 // disable unit price increase + gen.WindowTargetUnits = 1_000_000_000 // disable unit price increase + gen.MaxBlockUnits = 4_000_000 gen.MinBlockGap = 0 // don't require time between blocks gen.ValidityWindow = 1_000 * hconsts.MillisecondsPerSecond // txs shouldn't expire gen.CustomAllocation = []*genesis.CustomAllocation{ @@ -361,7 +362,7 @@ var _ = ginkgo.Describe("load tests vm", func() { ginkgo.By("create accounts", func() { senders = make([]*account, accts) for i := 0; i < accts; i++ { - tpriv, err := crypto.GeneratePrivateKey() + tpriv, err := ed25519.GeneratePrivateKey() gomega.Ω(err).Should(gomega.BeNil()) trsender := tpriv.PublicKey() tsender := utils.Address(trsender) @@ -375,10 +376,6 @@ var _ = ginkgo.Describe("load tests vm", func() { // leave some left over for root fundSplit := (genesisBalance - remainder) / uint64(accts) gomega.Ω(fundSplit).Should(gomega.Not(gomega.BeZero())) - requiredBlocks := accts / maxTxsPerBlock - if accts%maxTxsPerBlock > 0 { - requiredBlocks++ - } requiredTxs := map[ids.ID]struct{}{} for _, acct := range senders { id, err := issueSimpleTx(instances[0], acct.rsender, fundSplit, root.factory) @@ -386,13 +383,16 @@ var _ = ginkgo.Describe("load tests vm", func() { requiredTxs[id] = struct{}{} } - for i := 0; i < requiredBlocks; i++ { + for { blk := produceBlock(instances[0]) - log.Debug("block produced", zap.Int("txs", len(blk.Txs))) + if blk == nil { + break + } + log.Debug("block produced", zap.Uint64("height", blk.Hght), zap.Int("txs", len(blk.Txs))) for _, result := range blk.Results() { if !result.Success { // Used for debugging - fmt.Println(string(result.Output), i, requiredBlocks) + fmt.Println(string(result.Output)) } gomega.Ω(result.Success).Should(gomega.BeTrue()) } @@ -450,13 +450,12 @@ var _ = ginkgo.Describe("load tests vm", func() { ginkgo.By("producing blks", func() { start := time.Now() - requiredBlocks := txs / maxTxsPerBlock - if txs%maxTxsPerBlock > 0 { - requiredBlocks++ - } - for i := 0; i < requiredBlocks; i++ { + for { blk := produceBlock(instances[0]) - log.Debug("block produced", zap.Int("txs", len(blk.Txs))) + if blk == nil { + break + } + log.Debug("block produced", zap.Uint64("height", blk.Hght), zap.Int("txs", len(blk.Txs))) for _, tx := range blk.Txs { delete(allTxs, tx.ID()) } @@ -482,7 +481,7 @@ var _ = ginkgo.Describe("load tests vm", func() { func issueSimpleTx( i *instance, - to crypto.PublicKey, + to ed25519.PublicKey, amount uint64, factory chain.AuthFactory, ) (ids.ID, error) { @@ -510,6 +509,9 @@ func produceBlock(i *instance) *chain.StatelessBlock { ctx := context.TODO() blk, err := i.vm.BuildBlock(ctx) + if errors.Is(err, chain.ErrNoTxs) { + return nil + } gomega.Ω(err).To(gomega.BeNil()) gomega.Ω(blk).To(gomega.Not(gomega.BeNil())) diff --git a/examples/tokenvm/utils/utils.go b/examples/tokenvm/utils/utils.go index 71b30e9b6b..98b53ea970 100644 --- a/examples/tokenvm/utils/utils.go +++ b/examples/tokenvm/utils/utils.go @@ -4,15 +4,15 @@ package utils import ( - "github.com/ava-labs/hypersdk/crypto" + "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/consts" ) -func Address(pk crypto.PublicKey) string { - return crypto.Address(consts.HRP, pk) +func Address(pk ed25519.PublicKey) string { + return ed25519.Address(consts.HRP, pk) } -func ParseAddress(s string) (crypto.PublicKey, error) { - return crypto.ParseAddress(consts.HRP, s) +func ParseAddress(s string) (ed25519.PublicKey, error) { + return ed25519.ParseAddress(consts.HRP, s) } diff --git a/go.mod b/go.mod index 450e25919b..055a481b8e 100644 --- a/go.mod +++ b/go.mod @@ -11,6 +11,7 @@ require ( github.com/gorilla/websocket v1.5.0 github.com/manifoldco/promptui v0.9.0 github.com/neilotoole/errgroup v0.1.6 + github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce github.com/onsi/ginkgo/v2 v2.8.1 github.com/pkg/browser v0.0.0-20210911075715-681adbf594b8 github.com/prometheus/client_golang v1.16.0 diff --git a/go.sum b/go.sum index eb61a831af..1f6dace7c7 100644 --- a/go.sum +++ b/go.sum @@ -436,6 +436,8 @@ github.com/neilotoole/errgroup v0.1.6/go.mod h1:Q2nLGf+594h0CLBs/Mbg6qOr7GtqDK7C github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce h1:/pEpMk55wH0X+E5zedGEMOdLuWmV8P4+4W3+LZaM6kg= +github.com/oasisprotocol/curve25519-voi v0.0.0-20230110094441-db37f07504ce/go.mod h1:hVoHR2EVESiICEMbg137etN/Lx+lSrHPTD39Z/uE+2s= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= diff --git a/vm/dependencies.go b/vm/dependencies.go index 8b6f973b3e..2190441926 100644 --- a/vm/dependencies.go +++ b/vm/dependencies.go @@ -47,6 +47,11 @@ type Genesis interface { Load(context.Context, atrace.Tracer, chain.Database) error } +type AuthEngine interface { + GetBatchVerifier(cores int, count int) chain.AuthBatchVerifier + Cache(auth chain.Auth) +} + type Controller interface { Initialize( inner *VM, // hypersdk VM @@ -67,6 +72,7 @@ type Controller interface { handler Handlers, actionRegistry chain.ActionRegistry, authRegistry chain.AuthRegistry, + authEngines map[uint8]AuthEngine, err error, ) diff --git a/vm/mock_controller.go b/vm/mock_controller.go index 433b90190a..cbc500c9a1 100644 --- a/vm/mock_controller.go +++ b/vm/mock_controller.go @@ -58,7 +58,7 @@ func (mr *MockControllerMockRecorder) Accepted(arg0, arg1 interface{}) *gomock.C } // Initialize mocks base method. -func (m *MockController) Initialize(arg0 *VM, arg1 *snow.Context, arg2 metrics.MultiGatherer, arg3, arg4, arg5 []byte) (Config, Genesis, builder.Builder, gossiper.Gossiper, database.Database, database.Database, Handlers, chain.ActionRegistry, chain.AuthRegistry, error) { +func (m *MockController) Initialize(arg0 *VM, arg1 *snow.Context, arg2 metrics.MultiGatherer, arg3, arg4, arg5 []byte) (Config, Genesis, builder.Builder, gossiper.Gossiper, database.Database, database.Database, Handlers, chain.ActionRegistry, chain.AuthRegistry, map[byte]AuthEngine, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Initialize", arg0, arg1, arg2, arg3, arg4, arg5) ret0, _ := ret[0].(Config) @@ -70,8 +70,9 @@ func (m *MockController) Initialize(arg0 *VM, arg1 *snow.Context, arg2 metrics.M ret6, _ := ret[6].(Handlers) ret7, _ := ret[7].(chain.ActionRegistry) ret8, _ := ret[8].(chain.AuthRegistry) - ret9, _ := ret[9].(error) - return ret0, ret1, ret2, ret3, ret4, ret5, ret6, ret7, ret8, ret9 + ret9, _ := ret[9].(map[byte]AuthEngine) + ret10, _ := ret[10].(error) + return ret0, ret1, ret2, ret3, ret4, ret5, ret6, ret7, ret8, ret9, ret10 } // Initialize indicates an expected call of Initialize. diff --git a/vm/resolutions.go b/vm/resolutions.go index b6831c54cc..c99ff2efd2 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -162,6 +162,9 @@ func (vm *VM) processAcceptedBlocks() { // Sign and store any warp messages (regardless if validator now, may become one) results := b.Results() for i, tx := range b.Txs { + // Only cache auth for accepted blocks to prevent cache manipulation from RPC submissions + vm.cacheAuth(tx.Auth) + result := results[i] if result.WarpMessage == nil { continue @@ -389,3 +392,19 @@ func (vm *VM) GetTargetGossipDuration() time.Duration { func (vm *VM) RecordEmptyBlockBuilt() { vm.metrics.emptyBlockBuilt.Inc() } + +func (vm *VM) GetAuthBatchVerifier(authTypeID uint8, cores int, count int) (chain.AuthBatchVerifier, bool) { + bv, ok := vm.authEngine[authTypeID] + if !ok { + return nil, false + } + return bv.GetBatchVerifier(cores, count), ok +} + +func (vm *VM) cacheAuth(auth chain.Auth) { + bv, ok := vm.authEngine[auth.GetTypeID()] + if !ok { + return + } + bv.Cache(auth) +} diff --git a/vm/vm.go b/vm/vm.go index 7f7836b5d1..be1d8c3b22 100644 --- a/vm/vm.go +++ b/vm/vm.go @@ -63,6 +63,7 @@ type VM struct { handlers Handlers actionRegistry chain.ActionRegistry authRegistry chain.AuthRegistry + authEngine map[uint8]AuthEngine tracer trace.Tracer mempool *mempool.Mempool[*chain.Transaction] @@ -168,7 +169,7 @@ func (vm *VM) Initialize( // Always initialize implementation first vm.config, vm.genesis, vm.builder, vm.gossiper, vm.vmDB, - vm.rawStateDB, vm.handlers, vm.actionRegistry, vm.authRegistry, err = vm.c.Initialize( + vm.rawStateDB, vm.handlers, vm.actionRegistry, vm.authRegistry, vm.authEngine, err = vm.c.Initialize( vm, snowCtx, gatherer, diff --git a/workers/workers.go b/workers/workers.go index a736e0e9f3..6c7890048b 100644 --- a/workers/workers.go +++ b/workers/workers.go @@ -142,6 +142,7 @@ func (w *Workers) Stop() { } type Job struct { + count int tasks chan func() error completed chan struct{} result chan error @@ -171,6 +172,12 @@ func (j *Job) Wait() error { return <-j.result } +// Workers returns the number of workers that will execute the job. This can be used +// by callers to generate batch sizes that lead to the most efficient computation. +func (j *Job) Workers() int { + return j.count +} + // NewJob creates a new job and adds it to the workers' queue. [taskBacklog] // specifies the maximum number of tasks that can be added to the created job // channel. @@ -185,6 +192,7 @@ func (w *Workers) NewJob(taskBacklog int) (*Job, error) { return nil, ErrShutdown } j := &Job{ + count: w.count, tasks: make(chan func() error, taskBacklog), completed: make(chan struct{}), result: make(chan error, 1),