From 29048b33ef75ad14a44692c992394b85b21e5a81 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 14:40:27 -0400 Subject: [PATCH 01/29] remove warp --- chain/block.go | 145 ---- chain/builder.go | 51 -- chain/consts.go | 13 - chain/dependencies.go | 23 +- chain/errors.go | 10 - chain/processor.go | 12 +- chain/result.go | 24 +- chain/transaction.go | 180 +---- chain/warp_signature.go | 13 - examples/morpheusvm/actions/transfer.go | 17 +- examples/morpheusvm/auth/bls.go | 3 +- examples/morpheusvm/auth/ed25519.go | 3 +- examples/morpheusvm/auth/secp256r1.go | 3 +- .../cmd/morpheus-cli/cmd/resolutions.go | 5 +- examples/morpheusvm/consts/consts.go | 5 +- examples/morpheusvm/genesis/genesis.go | 6 - examples/morpheusvm/genesis/rules.go | 16 - examples/morpheusvm/registry/registry.go | 5 +- examples/morpheusvm/storage/state_manager.go | 8 - examples/morpheusvm/storage/storage.go | 19 - .../tests/integration/integration_test.go | 2 - examples/tokenvm/actions/burn_asset.go | 13 +- examples/tokenvm/actions/close_order.go | 23 +- examples/tokenvm/actions/create_asset.go | 29 +- examples/tokenvm/actions/create_order.go | 25 +- examples/tokenvm/actions/export_asset.go | 284 ------- examples/tokenvm/actions/fill_order.go | 43 +- examples/tokenvm/actions/import_asset.go | 292 ------- examples/tokenvm/actions/mint_asset.go | 34 +- examples/tokenvm/actions/outputs.go | 4 - examples/tokenvm/actions/transfer.go | 19 +- examples/tokenvm/actions/warp_transfer.go | 156 ---- examples/tokenvm/auth/ed25519.go | 3 +- examples/tokenvm/cmd/token-cli/cmd/action.go | 305 +------- examples/tokenvm/cmd/token-cli/cmd/handler.go | 28 +- .../tokenvm/cmd/token-cli/cmd/resolutions.go | 49 +- examples/tokenvm/consts/consts.go | 5 +- examples/tokenvm/controller/state_manager.go | 8 - examples/tokenvm/genesis/genesis.go | 6 - examples/tokenvm/genesis/rules.go | 20 - examples/tokenvm/registry/registry.go | 8 +- examples/tokenvm/rpc/jsonrpc_client.go | 4 +- examples/tokenvm/rpc/jsonrpc_server.go | 4 +- examples/tokenvm/storage/storage.go | 28 +- examples/tokenvm/tests/e2e/e2e_test.go | 738 ------------------ .../tests/integration/integration_test.go | 277 +------ rpc/dependencies.go | 3 - rpc/jsonrpc_client.go | 149 +--- rpc/jsonrpc_server.go | 80 -- vm/network_warp.go | 71 -- vm/resolutions.go | 36 - vm/storage.go | 98 --- vm/vm.go | 9 - vm/warp_manager.go | 360 --------- .../simulator/vm/actions/program_create.go | 15 +- .../simulator/vm/actions/program_execute.go | 33 +- x/programs/cmd/simulator/vm/consts/consts.go | 5 +- .../cmd/simulator/vm/genesis/genesis.go | 6 - x/programs/cmd/simulator/vm/genesis/rules.go | 16 - .../cmd/simulator/vm/registry/registry.go | 5 +- .../cmd/simulator/vm/storage/state_manager.go | 8 - .../cmd/simulator/vm/storage/storage.go | 17 - 62 files changed, 156 insertions(+), 3723 deletions(-) delete mode 100644 chain/warp_signature.go delete mode 100644 examples/tokenvm/actions/export_asset.go delete mode 100644 examples/tokenvm/actions/import_asset.go delete mode 100644 examples/tokenvm/actions/warp_transfer.go delete mode 100644 vm/network_warp.go delete mode 100644 vm/warp_manager.go diff --git a/chain/block.go b/chain/block.go index 7be9c7ec6f..687ad80db5 100644 --- a/chain/block.go +++ b/chain/block.go @@ -15,7 +15,6 @@ import ( "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/x/merkledb" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" @@ -52,7 +51,6 @@ type StatefulBlock struct { // blocking the consensus engine from voting on the block, // starting the verification of another block, etc. StateRoot ids.ID `json:"stateRoot"` - WarpResults set.Bits64 `json:"warpResults"` size int @@ -73,16 +71,6 @@ func (b *StatefulBlock) ID() (ids.ID, error) { return utils.ToID(blk), nil } -// warpJob is used to signal to a listner that a *warp.Message has been -// verified. -type warpJob struct { - msg *warp.Message - signers int - verifiedChan chan bool - verified bool - warpNum int -} - func NewGenesisBlock(root ids.ID) *StatefulBlock { return &StatefulBlock{ // We set the genesis block timestamp to be after the ProposerVM fork activation. @@ -112,8 +100,6 @@ type StatelessBlock struct { bytes []byte txsSet set.Set[ids.ID] - warpMessages map[ids.ID]*warpJob - containsWarp bool // this allows us to avoid allocating a map when we build bctx *block.Context vdrState validators.State @@ -179,7 +165,6 @@ func (b *StatelessBlock) populateTxs(ctx context.Context) error { // 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 { // Ensure there are no duplicate transactions if b.txsSet.Contains(tx.ID()) { @@ -195,29 +180,6 @@ func (b *StatelessBlock) populateTxs(ctx context.Context) error { } batchVerifier.Add(txDigest, tx.Auth) } - - // Check if we need the block context to verify the block (which contains - // an Avalanche Warp Message) - // - // Instead of erroring out if a warp message is invalid, we mark the - // verification as skipped and include it in the verification result so - // that a fee can still be deducted. - if tx.WarpMessage != nil { - if len(b.warpMessages) == MaxWarpMessages { - return ErrTooManyWarpMessages - } - signers, err := tx.WarpMessage.Signature.NumSigners() - if err != nil { - return err - } - b.warpMessages[tx.ID()] = &warpJob{ - msg: tx.WarpMessage, - signers: signers, - verifiedChan: make(chan bool, 1), - warpNum: len(b.warpMessages), - } - b.containsWarp = true - } } return nil } @@ -287,9 +249,6 @@ func (b *StatelessBlock) initializeBuilt( b.txsSet = set.NewSet[ids.ID](len(b.Txs)) for _, tx := range b.Txs { b.txsSet.Add(tx.ID()) - if tx.WarpMessage != nil { - b.containsWarp = true - } } return nil } @@ -297,11 +256,6 @@ func (b *StatelessBlock) initializeBuilt( // implements "snowman.Block.choices.Decidable" func (b *StatelessBlock) ID() ids.ID { return b.id } -// implements "block.WithVerifyContext" -func (b *StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { - return b.containsWarp, nil -} - // implements "block.WithVerifyContext" func (b *StatelessBlock) VerifyWithContext(ctx context.Context, bctx *block.Context) error { start := time.Now() @@ -407,33 +361,6 @@ func (b *StatelessBlock) verify(ctx context.Context, stateReady bool) error { return nil } -// verifyWarpMessage will attempt to verify a given warp message provided by an -// Action. -func (b *StatelessBlock) verifyWarpMessage(ctx context.Context, r Rules, msg *warp.Message) bool { - // We do not check the validity of [SourceChainID] because a VM could send - // itself a message to trigger a chain upgrade. - allowed, num, denom := r.GetWarpConfig(msg.SourceChainID) - if !allowed { - b.vm.Logger(). - Warn("unable to verify warp message", zap.Stringer("warpID", msg.ID()), zap.Error(ErrDisabledChainID)) - return false - } - if err := msg.Signature.Verify( - ctx, - &msg.UnsignedMessage, - r.NetworkID(), - b.vdrState, - b.bctx.PChainHeight, - num, - denom, - ); err != nil { - b.vm.Logger(). - Warn("unable to verify warp message", zap.Stringer("warpID", msg.ID()), zap.Error(err)) - return false - } - return true -} - // innerVerify executes the block on top of the provided [VerifyContext]. // // Invariants: @@ -515,59 +442,6 @@ func (b *StatelessBlock) innerVerify(ctx context.Context, vctx VerifyContext) er } } - // Start validating warp messages, if they exist - var invalidWarpResult bool - if b.containsWarp { - if b.bctx == nil { - log.Error( - "missing verify block context", - zap.Uint64("height", b.Hght), - zap.Stringer("id", b.ID()), - ) - return ErrMissingBlockContext - } - _, warpVerifySpan := b.vm.Tracer().Start(ctx, "StatelessBlock.verifyWarpMessages") //nolint:spancheck - b.vdrState = b.vm.ValidatorState() - go func() { - 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 - // Multi-Signature verification is already parallelized so we should just - // do one at a time to avoid overwhelming the CPU. - for txID, msg := range b.warpMessages { - if ctx.Err() != nil { - return - } - blockVerified := b.WarpResults.Contains(uint(msg.warpNum)) - if b.vm.IsBootstrapped() && !invalidWarpResult { - start := time.Now() - verified := b.verifyWarpMessage(ctx, r, msg.msg) - msg.verifiedChan <- verified - msg.verified = verified - log.Info( - "processed warp message", - zap.Stringer("txID", txID), - zap.Bool("verified", verified), - zap.Int("signers", msg.signers), - zap.Duration("t", time.Since(start)), - ) - if blockVerified != verified { - invalidWarpResult = true - } - } else { - // When we are bootstrapping, we just use the result in the block. - // - // We also use the result in the block when we have found - // a verification mismatch (our verify result is different than the - // block) to avoid doing extra work. - msg.verifiedChan <- blockVerified - msg.verified = blockVerified - } - } - }() - } - // Compute next unit prices to use feeKey := FeeKey(b.vm.StateManager().FeeKey()) feeRaw, err := parentView.GetValue(ctx, feeKey) @@ -589,23 +463,6 @@ func (b *StatelessBlock) innerVerify(ctx context.Context, vctx VerifyContext) er b.results = results b.feeManager = feeManager - // Ensure warp results are correct - if invalidWarpResult { - return ErrWarpResultMismatch - } - numWarp := len(b.warpMessages) - if numWarp > MaxWarpMessages { - return ErrTooManyWarpMessages - } - var warpResultsLimit set.Bits64 - warpResultsLimit.Add(uint(numWarp)) - if b.WarpResults >= warpResultsLimit { - // If the value of [WarpResults] is greater than the value of uint64 with - // a 1-bit shifted [numWarp] times, then there are unused bits set to - // 1 (which should is not allowed). - return ErrWarpResultMismatch - } - // Update chain metadata heightKeyStr := string(heightKey) timestampKeyStr := string(timestampKey) @@ -999,7 +856,6 @@ func (b *StatefulBlock) Marshal() ([]byte, error) { } p.PackID(b.StateRoot) - p.PackUint64(uint64(b.WarpResults)) bytes := p.Bytes() if err := p.Err(); err != nil { return nil, err @@ -1034,7 +890,6 @@ func UnmarshalBlock(raw []byte, parser Parser) (*StatefulBlock, error) { } p.UnpackID(false, &b.StateRoot) - b.WarpResults = set.Bits64(p.UnpackUint64(false)) // Ensure no leftover bytes if !p.Empty() { diff --git a/chain/builder.go b/chain/builder.go index 71ec95575b..b589c558c7 100644 --- a/chain/builder.go +++ b/chain/builder.go @@ -125,7 +125,6 @@ func BuildBlock( cache = map[string]*fetchData{} blockLock sync.RWMutex - warpAdded = uint(0) start = time.Now() txsAttempted = 0 results = []*Result{} @@ -174,18 +173,6 @@ func BuildBlock( continue } - // Ensure we can process if transaction includes a warp message - if tx.WarpMessage != nil && blockContext == nil { - log.Debug( - "dropping pending warp message because no context provided", - zap.Stringer("txID", tx.ID()), - ) - restorableLock.Lock() - restorable = append(restorable, tx) - restorableLock.Unlock() - continue - } - stateKeys, err := tx.StateKeys(sm) if err != nil { // Drop bad transaction and continue @@ -287,35 +274,6 @@ func BuildBlock( return nil } - // Verify warp message, if it exists - // - // We don't drop invalid warp messages because we must collect fees for - // the work the sender made us do (otherwise this would be a DoS). - // - // We wait as long as possible to verify the signature to ensure we don't - // spend unnecessary time on an invalid tx. - var warpErr error - if tx.WarpMessage != nil { - // We do not check the validity of [SourceChainID] because a VM could send - // itself a message to trigger a chain upgrade. - allowed, num, denom := r.GetWarpConfig(tx.WarpMessage.SourceChainID) - if allowed { - warpErr = tx.WarpMessage.Signature.Verify( - ctx, &tx.WarpMessage.UnsignedMessage, r.NetworkID(), - vdrState, blockContext.PChainHeight, num, denom, - ) - } else { - warpErr = ErrDisabledChainID - } - if warpErr != nil { - log.Warn( - "warp verification failed", - zap.Stringer("txID", tx.ID()), - zap.Error(warpErr), - ) - } - } - // If execution works, keep moving forward with new state // // Note, these calculations must match block verification exactly @@ -347,7 +305,6 @@ func BuildBlock( r, tsv, nextTime, - tx.WarpMessage != nil && warpErr == nil, ) if err != nil { // Returning an error here should be avoided at all costs (can be a DoS). Rather, @@ -357,7 +314,6 @@ func BuildBlock( return err } - // Need to atomically check there aren't too many warp messages and add to block blockLock.Lock() defer blockLock.Unlock() @@ -385,13 +341,6 @@ func BuildBlock( tsv.Commit() b.Txs = append(b.Txs, tx) results = append(results, result) - if tx.WarpMessage != nil { - if warpErr == nil { - // Add a bit if the warp message was verified - b.WarpResults.Add(warpAdded) - } - warpAdded++ - } return nil }) } diff --git a/chain/consts.go b/chain/consts.go index 66d7315b10..816ef9f307 100644 --- a/chain/consts.go +++ b/chain/consts.go @@ -19,19 +19,6 @@ const ( // nodes may not build during their allocated window to avoid increasing the skew of the // chain time. FutureBound = 1 * time.Second - // MaxWarpMessageSize is the maximum size of a warp message. - MaxWarpMessageSize = 256 * units.KiB - // MaxWarpMessages is the maximum number of warp messages allows in a single - // block. - MaxWarpMessages = 64 - // MaxIncomingWarpChunks is the number of chunks stored for an incoming warp message. - MaxIncomingWarpChunks = 0 - // MaxOutgoingWarpChunks is the max number of chunks that can be stored for an outgoing warp message. - // - // This is defined as a constant because storage of warp messages is handled by the hypersdk, - // not the [Controller]. In this mechanism, we frequently query warp messages by TxID across - // ranges (so, we can't expose a way to modify this over time). - MaxOutgoingWarpChunks = 4 HeightKeyChunks = 1 TimestampKeyChunks = 1 FeeKeyChunks = 8 // 96 (per dimension) * 5 (num dimensions) diff --git a/chain/dependencies.go b/chain/dependencies.go index 6ceddda353..6ca1057feb 100644 --- a/chain/dependencies.go +++ b/chain/dependencies.go @@ -13,7 +13,6 @@ import ( "github.com/ava-labs/avalanchego/trace" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/x/merkledb" "github.com/ava-labs/hypersdk/codec" @@ -24,8 +23,8 @@ import ( ) type ( - ActionRegistry *codec.TypeParser[Action, *warp.Message, bool] - AuthRegistry *codec.TypeParser[Auth, *warp.Message, bool] + ActionRegistry *codec.TypeParser[Action, bool] + AuthRegistry *codec.TypeParser[Auth, bool] ) type Parser interface { @@ -131,9 +130,6 @@ type Rules interface { GetMaxBlockUnits() fees.Dimensions GetBaseComputeUnits() uint64 - GetBaseWarpComputeUnits() uint64 - GetWarpComputeUnitsPerSigner() uint64 - GetOutgoingWarpComputeUnits() uint64 // Invariants: // * Controllers must manage the max key length and max value length (max network @@ -155,8 +151,6 @@ type Rules interface { GetStorageKeyWriteUnits() uint64 GetStorageValueWriteUnits() uint64 // per chunk - GetWarpConfig(sourceChainID ids.ID) (bool, uint64, uint64) - FetchCustom(string) (any, bool) } @@ -166,11 +160,6 @@ type MetadataManager interface { FeeKey() []byte } -type WarpManager interface { - IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) []byte - OutgoingWarpKeyPrefix(txID ids.ID) []byte -} - type FeeHandler interface { // StateKeys is a full enumeration of all database keys that could be touched during fee payment // by [addr]. This is used to prefetch state and will be used to parallelize execution (making @@ -205,7 +194,6 @@ type FeeHandler interface { type StateManager interface { FeeHandler MetadataManager - WarpManager } type Object interface { @@ -267,12 +255,7 @@ type Action interface { timestamp int64, actor codec.Address, txID ids.ID, - warpVerified bool, - ) (success bool, computeUnits uint64, output []byte, warpMessage *warp.UnsignedMessage, err error) - - // OutputsWarpMessage indicates whether an [Action] will produce a warp message. The max size - // of any warp message is [MaxOutgoingWarpChunks]. - OutputsWarpMessage() bool + ) (success bool, computeUnits uint64, output []byte, err error) } type Auth interface { diff --git a/chain/errors.go b/chain/errors.go index fc520a41dd..a88d8ccb37 100644 --- a/chain/errors.go +++ b/chain/errors.go @@ -56,16 +56,6 @@ var ( ErrBlockTooBig = errors.New("block too big") ErrKeyNotSpecified = errors.New("key not specified") - // Warp - ErrDisabledChainID = errors.New("cannot import from chain ID") - ErrMissingBlockContext = errors.New("cannot verify warp messages without block context") - ErrUnexpectedWarpMessage = errors.New("unexpected warp message") - ErrExpectedWarpMessage = errors.New("expected warp message") - ErrWarpMessageNotInitialized = errors.New("warp message not initialized") - ErrEmptyWarpPayload = errors.New("empty warp payload") - ErrTooManyWarpMessages = errors.New("too many warp messages") - ErrWarpResultMismatch = errors.New("warp result mismatch") - // Misc ErrNotImplemented = errors.New("not implemented") ErrBlockNotProcessed = errors.New("block is not processed") diff --git a/chain/processor.go b/chain/processor.go index 3aef2c87a0..0b44215923 100644 --- a/chain/processor.go +++ b/chain/processor.go @@ -79,17 +79,7 @@ func (b *StatelessBlock) Execute( return err } - // Wait to execute transaction until we have the warp result processed. - var warpVerified bool - warpMsg, ok := b.warpMessages[tx.ID()] - if ok { - select { - case warpVerified = <-warpMsg.verifiedChan: - case <-ctx.Done(): - return ctx.Err() - } - } - result, err := tx.Execute(ctx, feeManager, reads, sm, r, tsv, t, ok && warpVerified) + result, err := tx.Execute(ctx, feeManager, reads, sm, r, tsv, t) if err != nil { return err } diff --git a/chain/result.go b/chain/result.go index 40b1604796..7154576acd 100644 --- a/chain/result.go +++ b/chain/result.go @@ -4,8 +4,6 @@ package chain import ( - "github.com/ava-labs/avalanchego/vms/platformvm/warp" - "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" "github.com/ava-labs/hypersdk/fees" @@ -17,17 +15,11 @@ type Result struct { Consumed fees.Dimensions Fee uint64 - - WarpMessage *warp.UnsignedMessage } func (r *Result) Size() int { size := consts.BoolLen + codec.BytesLen(r.Output) + fees.DimensionsLen + consts.Uint64Len - if r.WarpMessage != nil { - size += codec.BytesLen(r.WarpMessage.Bytes()) - } else { - size += codec.BytesLen(nil) - } + size += codec.BytesLen(nil) return size } @@ -36,11 +28,6 @@ func (r *Result) Marshal(p *codec.Packer) error { p.PackBytes(r.Output) p.PackFixedBytes(r.Consumed.Bytes()) p.PackUint64(r.Fee) - var warpBytes []byte - if r.WarpMessage != nil { - warpBytes = r.WarpMessage.Bytes() - } - p.PackBytes(warpBytes) return nil } @@ -73,15 +60,6 @@ func UnmarshalResult(p *codec.Packer) (*Result, error) { } result.Consumed = consumed result.Fee = p.UnpackUint64(false) - var warpMessage []byte - p.UnpackBytes(MaxWarpMessageSize, false, &warpMessage) - if len(warpMessage) > 0 { - msg, err := warp.ParseUnsignedMessage(warpMessage) - if err != nil { - return nil, err - } - result.WarpMessage = msg - } return result, p.Err() } diff --git a/chain/transaction.go b/chain/transaction.go index 2623a724fa..a281b75d4c 100644 --- a/chain/transaction.go +++ b/chain/transaction.go @@ -10,7 +10,6 @@ import ( "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" @@ -31,7 +30,6 @@ var ( type Transaction struct { Base *Base `json:"base"` - WarpMessage *warp.Message `json:"warpMessage"` // TODO: turn [Action] into an array (#335) Action Action `json:"action"` @@ -41,24 +39,12 @@ type Transaction struct { bytes []byte size int id ids.ID - numWarpSigners int - // warpID is just the hash of the *warp.Message.Payload. We assumed that - // all warp messages from a single source have some unique field that - // prevents duplicates (like txID). We will not allow 2 instances of the same - // warpID from the same sourceChainID to be accepted. - warpID ids.ID stateKeys state.Keys } -type WarpResult struct { - Message *warp.Message - VerifyErr error -} - -func NewTx(base *Base, wm *warp.Message, act Action) *Transaction { +func NewTx(base *Base, act Action) *Transaction { return &Transaction{ Base: base, - WarpMessage: wm, Action: act, } } @@ -68,16 +54,9 @@ func (t *Transaction) Digest() ([]byte, error) { return t.digest, nil } actionID := t.Action.GetTypeID() - var warpBytes []byte - if t.WarpMessage != nil { - warpBytes = t.WarpMessage.Bytes() - } - size := t.Base.Size() + - codec.BytesLen(warpBytes) + - consts.ByteLen + t.Action.Size() + size := t.Base.Size() + consts.ByteLen + t.Action.Size() p := codec.NewWriter(size, consts.NetworkSizeLimit) t.Base.Marshal(p) - p.PackBytes(warpBytes) p.PackByte(actionID) t.Action.Marshal(p) return p.Bytes(), p.Err() @@ -140,18 +119,6 @@ func (t *Transaction) StateKeys(sm StateManager) (state.Keys, error) { } } - // Add keys used to manage warp operations - if t.WarpMessage != nil { - p := sm.IncomingWarpKeyPrefix(t.WarpMessage.SourceChainID, t.warpID) - k := keys.EncodeChunks(p, MaxIncomingWarpChunks) - stateKeys.Add(string(k), state.All) - } - if t.Action.OutputsWarpMessage() { - p := sm.OutgoingWarpKeyPrefix(t.id) - k := keys.EncodeChunks(p, MaxOutgoingWarpChunks) - stateKeys.Add(string(k), state.Allocate|state.Write) - } - // Cache keys if called again t.stateKeys = stateKeys return stateKeys, nil @@ -167,14 +134,6 @@ func (t *Transaction) MaxUnits(sm StateManager, r Rules) (fees.Dimensions, error maxComputeUnitsOp := math.NewUint64Operator(r.GetBaseComputeUnits()) maxComputeUnitsOp.Add(t.Action.MaxComputeUnits(r)) maxComputeUnitsOp.Add(t.Auth.ComputeUnits(r)) - if t.WarpMessage != nil { - maxComputeUnitsOp.Add(r.GetBaseWarpComputeUnits()) - maxComputeUnitsOp.MulAdd(uint64(t.numWarpSigners), r.GetWarpComputeUnitsPerSigner()) - } - if t.Action.OutputsWarpMessage() { - // Chunks later accounted for by call to [StateKeys] - maxComputeUnitsOp.Add(r.GetOutgoingWarpComputeUnits()) - } maxComputeUnits, err := maxComputeUnitsOp.Value() if err != nil { return fees.Dimensions{}, err @@ -223,7 +182,7 @@ func (t *Transaction) MaxUnits(sm StateManager, r Rules) (fees.Dimensions, error // EstimateMaxUnits provides a pessimistic estimate of the cost to execute a transaction. This is // typically used during transaction construction. -func EstimateMaxUnits(r Rules, action Action, authFactory AuthFactory, warpMessage *warp.Message) (fees.Dimensions, error) { +func EstimateMaxUnits(r Rules, action Action, authFactory AuthFactory) (fees.Dimensions, error) { authBandwidth, authCompute := authFactory.MaxUnits() bandwidth := BaseSize + consts.ByteLen + uint64(action.Size()) + consts.ByteLen + authBandwidth actionStateKeysMaxChunks := action.StateKeysMaxChunks() @@ -236,20 +195,6 @@ func EstimateMaxUnits(r Rules, action Action, authFactory AuthFactory, warpMessa computeUnitsOp := math.NewUint64Operator(r.GetBaseComputeUnits()) computeUnitsOp.Add(authCompute) computeUnitsOp.Add(action.MaxComputeUnits(r)) - if warpMessage != nil { - bandwidth += uint64(codec.BytesLen(warpMessage.Bytes())) - stateKeysMaxChunks = append(stateKeysMaxChunks, MaxIncomingWarpChunks) - computeUnitsOp.Add(r.GetBaseWarpComputeUnits()) - numSigners, err := warpMessage.Signature.NumSigners() - if err != nil { - return fees.Dimensions{}, err - } - computeUnitsOp.MulAdd(uint64(numSigners), r.GetWarpComputeUnitsPerSigner()) - } - if action.OutputsWarpMessage() { - stateKeysMaxChunks = append(stateKeysMaxChunks, MaxOutgoingWarpChunks) - computeUnitsOp.Add(r.GetOutgoingWarpComputeUnits()) - } computeUnits, err := computeUnitsOp.Value() if err != nil { return fees.Dimensions{}, err @@ -338,7 +283,6 @@ func (t *Transaction) Execute( r Rules, ts *tstate.TStateView, timestamp int64, - warpVerified bool, ) (*Result, error) { // Always charge fee first (in case [Action] moves funds) maxUnits, err := t.MaxUnits(s, r) @@ -357,24 +301,6 @@ func (t *Transaction) Execute( return nil, err } - // Check warp message is not duplicate - if t.WarpMessage != nil { - p := s.IncomingWarpKeyPrefix(t.WarpMessage.SourceChainID, t.warpID) - k := keys.EncodeChunks(p, MaxIncomingWarpChunks) - _, err := ts.GetValue(ctx, k) - switch { - case err == nil: - // Override all errors because warp message is a duplicate - warpVerified = false - case errors.Is(err, database.ErrNotFound): - // This means there are no conflicts - case err != nil: - // An error here can indicate there is an issue with the database or that - // the key was not properly specified. - return &Result{false, utils.ErrBytes(err), maxUnits, maxFee, nil}, nil - } - } - // We create a temp state checkpoint to ensure we don't commit failed actions to state. actionStart := ts.OpIndex() handleRevert := func(rerr error) (*Result, error) { @@ -384,7 +310,7 @@ func (t *Transaction) Execute( ts.Rollback(ctx, actionStart) return &Result{false, utils.ErrBytes(rerr), maxUnits, maxFee, nil}, nil } - success, actionCUs, output, warpMessage, err := t.Action.Execute(ctx, r, ts, timestamp, t.Auth.Actor(), t.id, warpVerified) + success, actionCUs, output, err := t.Action.Execute(ctx, r, ts, timestamp, t.Auth.Actor(), t.id) if err != nil { return handleRevert(err) } @@ -393,56 +319,14 @@ func (t *Transaction) Execute( // fast) return handleRevert(ErrInvalidObject) } - outputsWarp := t.Action.OutputsWarpMessage() if !success { ts.Rollback(ctx, actionStart) - warpMessage = nil // warp messages can only be emitted on success - } else { - // Ensure constraints hold if successful - if (warpMessage == nil && outputsWarp) || (warpMessage != nil && !outputsWarp) { - return handleRevert(ErrInvalidObject) - } - - // Store incoming warp messages in state by their ID to prevent replays - if t.WarpMessage != nil { - p := s.IncomingWarpKeyPrefix(t.WarpMessage.SourceChainID, t.warpID) - k := keys.EncodeChunks(p, MaxIncomingWarpChunks) - if err := ts.Insert(ctx, k, nil); err != nil { - return handleRevert(err) - } - } - - // Store newly created warp messages in state by their txID to ensure we can - // always sign for a message - if warpMessage != nil { - // Enforce we are the source of our own messages - warpMessage.NetworkID = r.NetworkID() - warpMessage.SourceChainID = r.ChainID() - // Initialize message (compute bytes) now that everything is populated - if err := warpMessage.Initialize(); err != nil { - return handleRevert(err) - } - // We use txID here because did not know the warpID before execution (and - // we pre-reserve this key for the processor). - p := s.OutgoingWarpKeyPrefix(t.id) - k := keys.EncodeChunks(p, MaxOutgoingWarpChunks) - if err := ts.Insert(ctx, k, warpMessage.Bytes()); err != nil { - return handleRevert(err) - } - } } // Calculate units used computeUnitsOp := math.NewUint64Operator(r.GetBaseComputeUnits()) computeUnitsOp.Add(t.Auth.ComputeUnits(r)) computeUnitsOp.Add(actionCUs) - if t.WarpMessage != nil { - computeUnitsOp.Add(r.GetBaseWarpComputeUnits()) - computeUnitsOp.MulAdd(uint64(t.numWarpSigners), r.GetWarpComputeUnitsPerSigner()) - } - if success && outputsWarp { - computeUnitsOp.Add(r.GetOutgoingWarpComputeUnits()) - } computeUnits, err := computeUnitsOp.Value() if err != nil { return handleRevert(err) @@ -524,8 +408,6 @@ func (t *Transaction) Execute( Consumed: used, Fee: feeRequired, - - WarpMessage: warpMessage, }, nil } @@ -538,14 +420,6 @@ func (t *Transaction) Marshal(p *codec.Packer) error { actionID := t.Action.GetTypeID() authID := t.Auth.GetTypeID() t.Base.Marshal(p) - var warpBytes []byte - if t.WarpMessage != nil { - warpBytes = t.WarpMessage.Bytes() - if len(warpBytes) == 0 { - return ErrWarpMessageNotInitialized - } - } - p.PackBytes(warpBytes) p.PackByte(actionID) t.Action.Marshal(p) p.PackByte(authID) @@ -595,55 +469,30 @@ func UnmarshalTxs( func UnmarshalTx( p *codec.Packer, - actionRegistry *codec.TypeParser[Action, *warp.Message, bool], - authRegistry *codec.TypeParser[Auth, *warp.Message, bool], + actionRegistry *codec.TypeParser[Action, bool], + authRegistry *codec.TypeParser[Auth, bool], ) (*Transaction, error) { start := p.Offset() base, err := UnmarshalBase(p) if err != nil { return nil, fmt.Errorf("%w: could not unmarshal base", err) } - var warpBytes []byte - p.UnpackBytes(MaxWarpMessageSize, false, &warpBytes) - var warpMessage *warp.Message - var numWarpSigners int - if len(warpBytes) > 0 { - msg, err := warp.ParseMessage(warpBytes) - if err != nil { - return nil, fmt.Errorf("%w: could not unmarshal warp message", err) - } - if len(msg.Payload) == 0 { - return nil, ErrEmptyWarpPayload - } - warpMessage = msg - numSigners, err := msg.Signature.NumSigners() - if err != nil { - return nil, fmt.Errorf("%w: could not calculate number of warp signers", err) - } - numWarpSigners = numSigners - } actionType := p.UnpackByte() - unmarshalAction, actionWarp, ok := actionRegistry.LookupIndex(actionType) + unmarshalAction, ok := actionRegistry.LookupIndex(actionType) if !ok { return nil, fmt.Errorf("%w: %d is unknown action type", ErrInvalidObject, actionType) } - if actionWarp && warpMessage == nil { - return nil, fmt.Errorf("%w: action %d", ErrExpectedWarpMessage, actionType) - } - action, err := unmarshalAction(p, warpMessage) + action, err := unmarshalAction(p) if err != nil { return nil, fmt.Errorf("%w: could not unmarshal action", err) } digest := p.Offset() authType := p.UnpackByte() - unmarshalAuth, authWarp, ok := authRegistry.LookupIndex(authType) + unmarshalAuth, ok := authRegistry.LookupIndex(authType) if !ok { return nil, fmt.Errorf("%w: %d is unknown auth type", ErrInvalidObject, authType) } - if authWarp && warpMessage == nil { - return nil, fmt.Errorf("%w: auth %d", ErrExpectedWarpMessage, authType) - } - auth, err := unmarshalAuth(p, warpMessage) + auth, err := unmarshalAuth(p) if err != nil { return nil, fmt.Errorf("%w: could not unmarshal auth", err) } @@ -653,15 +502,10 @@ func UnmarshalTx( if sponsorType := auth.Sponsor()[0]; sponsorType != authType { return nil, fmt.Errorf("%w: sponsorType (%d) did not match authType (%d)", ErrInvalidSponsor, sponsorType, authType) } - warpExpected := actionWarp || authWarp - if !warpExpected && warpMessage != nil { - return nil, ErrUnexpectedWarpMessage - } var tx Transaction tx.Base = base tx.Action = action - tx.WarpMessage = warpMessage tx.Auth = auth if err := p.Err(); err != nil { return nil, p.Err() @@ -671,9 +515,5 @@ func UnmarshalTx( tx.bytes = codecBytes[start:p.Offset()] // ensure errors handled before grabbing memory tx.size = len(tx.bytes) tx.id = utils.ToID(tx.bytes) - if tx.WarpMessage != nil { - tx.numWarpSigners = numWarpSigners - tx.warpID = tx.WarpMessage.ID() - } return &tx, nil } diff --git a/chain/warp_signature.go b/chain/warp_signature.go deleted file mode 100644 index b499f70046..0000000000 --- a/chain/warp_signature.go +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package chain - -type WarpSignature struct { - PublicKey []byte `json:"publicKey"` - Signature []byte `json:"signature"` -} - -func NewWarpSignature(pk []byte, sig []byte) *WarpSignature { - return &WarpSignature{pk, sig} -} diff --git a/examples/morpheusvm/actions/transfer.go b/examples/morpheusvm/actions/transfer.go index 24e00514bc..8ff8c9c404 100644 --- a/examples/morpheusvm/actions/transfer.go +++ b/examples/morpheusvm/actions/transfer.go @@ -7,7 +7,6 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -42,10 +41,6 @@ func (*Transfer) StateKeysMaxChunks() []uint16 { return []uint16{storage.BalanceChunks, storage.BalanceChunks} } -func (*Transfer) OutputsWarpMessage() bool { - return false -} - func (t *Transfer) Execute( ctx context.Context, _ chain.Rules, @@ -54,17 +49,17 @@ func (t *Transfer) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if t.Value == 0 { - return false, 1, OutputValueZero, nil, nil + return false, 1, OutputValueZero, nil } if err := storage.SubBalance(ctx, mu, actor, t.Value); err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } if err := storage.AddBalance(ctx, mu, t.To, t.Value, true); err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } - return true, 1, nil, nil, nil + return true, 1, nil, nil } func (*Transfer) MaxComputeUnits(chain.Rules) uint64 { @@ -80,7 +75,7 @@ func (t *Transfer) Marshal(p *codec.Packer) { p.PackUint64(t.Value) } -func UnmarshalTransfer(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalTransfer(p *codec.Packer) (chain.Action, error) { var transfer Transfer p.UnpackAddress(&transfer.To) // we do not verify the typeID is valid transfer.Value = p.UnpackUint64(true) diff --git a/examples/morpheusvm/auth/bls.go b/examples/morpheusvm/auth/bls.go index e612109294..f56f59bf8b 100644 --- a/examples/morpheusvm/auth/bls.go +++ b/examples/morpheusvm/auth/bls.go @@ -6,7 +6,6 @@ package auth import ( "context" - "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" @@ -72,7 +71,7 @@ func (b *BLS) Marshal(p *codec.Packer) { p.PackFixedBytes(bls.SignatureToBytes(b.Signature)) } -func UnmarshalBLS(p *codec.Packer, _ *warp.Message) (chain.Auth, error) { +func UnmarshalBLS(p *codec.Packer) (chain.Auth, error) { var b BLS signer := make([]byte, bls.PublicKeyLen) diff --git a/examples/morpheusvm/auth/ed25519.go b/examples/morpheusvm/auth/ed25519.go index 3c58182dbd..e8dd4ac7fe 100644 --- a/examples/morpheusvm/auth/ed25519.go +++ b/examples/morpheusvm/auth/ed25519.go @@ -6,7 +6,6 @@ package auth import ( "context" - "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" @@ -72,7 +71,7 @@ func (d *ED25519) Marshal(p *codec.Packer) { p.PackFixedBytes(d.Signature[:]) } -func UnmarshalED25519(p *codec.Packer, _ *warp.Message) (chain.Auth, error) { +func UnmarshalED25519(p *codec.Packer) (chain.Auth, error) { var d ED25519 signer := d.Signer[:] // avoid allocating additional memory p.UnpackFixedBytes(ed25519.PublicKeyLen, &signer) diff --git a/examples/morpheusvm/auth/secp256r1.go b/examples/morpheusvm/auth/secp256r1.go index a0e6869482..41f9adde6c 100644 --- a/examples/morpheusvm/auth/secp256r1.go +++ b/examples/morpheusvm/auth/secp256r1.go @@ -6,7 +6,6 @@ package auth import ( "context" - "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" @@ -72,7 +71,7 @@ func (d *SECP256R1) Marshal(p *codec.Packer) { p.PackFixedBytes(d.Signature[:]) } -func UnmarshalSECP256R1(p *codec.Packer, _ *warp.Message) (chain.Auth, error) { +func UnmarshalSECP256R1(p *codec.Packer) (chain.Auth, error) { var d SECP256R1 signer := d.Signer[:] // avoid allocating additional memory p.UnpackFixedBytes(secp256r1.PublicKeyLen, &signer) diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/resolutions.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/resolutions.go index b1931cac64..675fc47e19 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/resolutions.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/resolutions.go @@ -9,7 +9,6 @@ import ( "reflect" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/cli" "github.com/ava-labs/hypersdk/codec" @@ -22,14 +21,14 @@ import ( // sendAndWait may not be used concurrently func sendAndWait( - ctx context.Context, warpMsg *warp.Message, action chain.Action, cli *rpc.JSONRPCClient, + ctx context.Context, action chain.Action, cli *rpc.JSONRPCClient, bcli *brpc.JSONRPCClient, ws *rpc.WebSocketClient, factory chain.AuthFactory, printStatus bool, ) (bool, ids.ID, error) { //nolint:unparam parser, err := bcli.Parser(ctx) if err != nil { return false, ids.Empty, err } - _, tx, _, err := cli.GenerateTransaction(ctx, parser, warpMsg, action, factory) + _, tx, _, err := cli.GenerateTransaction(ctx, parser, action, factory) if err != nil { return false, ids.Empty, err } diff --git a/examples/morpheusvm/consts/consts.go b/examples/morpheusvm/consts/consts.go index 5a6d79d2fa..4223730a4d 100644 --- a/examples/morpheusvm/consts/consts.go +++ b/examples/morpheusvm/consts/consts.go @@ -5,7 +5,6 @@ package consts import ( "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -33,6 +32,6 @@ func init() { // Instantiate registry here so it can be imported by any package. We set these // values in [controller/registry]. var ( - ActionRegistry *codec.TypeParser[chain.Action, *warp.Message, bool] - AuthRegistry *codec.TypeParser[chain.Auth, *warp.Message, bool] + ActionRegistry *codec.TypeParser[chain.Action, bool] + AuthRegistry *codec.TypeParser[chain.Auth, bool] ) diff --git a/examples/morpheusvm/genesis/genesis.go b/examples/morpheusvm/genesis/genesis.go index 74962b5c96..6ebf7b0361 100644 --- a/examples/morpheusvm/genesis/genesis.go +++ b/examples/morpheusvm/genesis/genesis.go @@ -47,9 +47,6 @@ type Genesis struct { // Tx Fee Parameters BaseComputeUnits uint64 `json:"baseUnits"` - BaseWarpComputeUnits uint64 `json:"baseWarpUnits"` - WarpComputeUnitsPerSigner uint64 `json:"warpUnitsPerSigner"` - OutgoingWarpComputeUnits uint64 `json:"outgoingWarpComputeUnits"` StorageKeyReadUnits uint64 `json:"storageKeyReadUnits"` StorageValueReadUnits uint64 `json:"storageValueReadUnits"` // per chunk StorageKeyAllocateUnits uint64 `json:"storageKeyAllocateUnits"` @@ -81,9 +78,6 @@ func Default() *Genesis { // Tx Fee Compute Parameters BaseComputeUnits: 1, - BaseWarpComputeUnits: 1_024, - WarpComputeUnitsPerSigner: 128, - OutgoingWarpComputeUnits: 1_024, // Tx Fee Storage Parameters // diff --git a/examples/morpheusvm/genesis/rules.go b/examples/morpheusvm/genesis/rules.go index 0a37046203..bb375af7d3 100644 --- a/examples/morpheusvm/genesis/rules.go +++ b/examples/morpheusvm/genesis/rules.go @@ -24,10 +24,6 @@ func (g *Genesis) Rules(_ int64, networkID uint32, chainID ids.ID) *Rules { return &Rules{g, networkID, chainID} } -func (*Rules) GetWarpConfig(ids.ID) (bool, uint64, uint64) { - return false, 0, 0 -} - func (r *Rules) NetworkID() uint32 { return r.networkID } @@ -56,18 +52,6 @@ func (r *Rules) GetBaseComputeUnits() uint64 { return r.g.BaseComputeUnits } -func (r *Rules) GetBaseWarpComputeUnits() uint64 { - return r.g.BaseWarpComputeUnits -} - -func (r *Rules) GetWarpComputeUnitsPerSigner() uint64 { - return r.g.WarpComputeUnitsPerSigner -} - -func (r *Rules) GetOutgoingWarpComputeUnits() uint64 { - return r.g.OutgoingWarpComputeUnits -} - func (*Rules) GetSponsorStateKeysMaxChunks() []uint16 { return []uint16{storage.BalanceChunks} } diff --git a/examples/morpheusvm/registry/registry.go b/examples/morpheusvm/registry/registry.go index 47d4279a38..1a1cf13e93 100644 --- a/examples/morpheusvm/registry/registry.go +++ b/examples/morpheusvm/registry/registry.go @@ -5,7 +5,6 @@ package registry import ( "github.com/ava-labs/avalanchego/utils/wrappers" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" @@ -16,8 +15,8 @@ import ( // Setup types func init() { - consts.ActionRegistry = codec.NewTypeParser[chain.Action, *warp.Message]() - consts.AuthRegistry = codec.NewTypeParser[chain.Auth, *warp.Message]() + consts.ActionRegistry = codec.NewTypeParser[chain.Action]() + consts.AuthRegistry = codec.NewTypeParser[chain.Auth]() errs := &wrappers.Errs{} errs.Add( diff --git a/examples/morpheusvm/storage/state_manager.go b/examples/morpheusvm/storage/state_manager.go index 6372517181..34a3c7810e 100644 --- a/examples/morpheusvm/storage/state_manager.go +++ b/examples/morpheusvm/storage/state_manager.go @@ -28,14 +28,6 @@ func (*StateManager) FeeKey() []byte { return FeeKey() } -func (*StateManager) IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) []byte { - return IncomingWarpKeyPrefix(sourceChainID, msgID) -} - -func (*StateManager) OutgoingWarpKeyPrefix(txID ids.ID) []byte { - return OutgoingWarpKeyPrefix(txID) -} - func (*StateManager) SponsorStateKeys(addr codec.Address) state.Keys { return state.Keys{ string(BalanceKey(addr)): state.Read | state.Write, diff --git a/examples/morpheusvm/storage/storage.go b/examples/morpheusvm/storage/storage.go index 01e0bb28aa..c4763eeb51 100644 --- a/examples/morpheusvm/storage/storage.go +++ b/examples/morpheusvm/storage/storage.go @@ -34,8 +34,6 @@ type ReadState func(context.Context, [][]byte) ([][]byte, []error) // 0x1/ (hypersdk-height) // 0x2/ (hypersdk-timestamp) // 0x3/ (hypersdk-fee) -// 0x4/ (hypersdk-incoming warp) -// 0x5/ (hypersdk-outgoing warp) const ( // metaDB @@ -46,8 +44,6 @@ const ( heightPrefix = 0x1 timestampPrefix = 0x2 feePrefix = 0x3 - incomingWarpPrefix = 0x4 - outgoingWarpPrefix = 0x5 ) const BalanceChunks uint16 = 1 @@ -257,18 +253,3 @@ func TimestampKey() (k []byte) { func FeeKey() (k []byte) { return feeKey } - -func IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen*2) - k[0] = incomingWarpPrefix - copy(k[1:], sourceChainID[:]) - copy(k[1+consts.IDLen:], msgID[:]) - return k -} - -func OutgoingWarpKeyPrefix(txID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen) - k[0] = outgoingWarpPrefix - copy(k[1:], txID[:]) - return k -} diff --git a/examples/morpheusvm/tests/integration/integration_test.go b/examples/morpheusvm/tests/integration/integration_test.go index a4f4dc9857..d6a4cd800a 100644 --- a/examples/morpheusvm/tests/integration/integration_test.go +++ b/examples/morpheusvm/tests/integration/integration_test.go @@ -26,7 +26,6 @@ import ( "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/fatih/color" ginkgo "github.com/onsi/ginkgo/v2" "github.com/onsi/gomega" @@ -211,7 +210,6 @@ var _ = ginkgo.BeforeSuite(func() { ChainDataDir: dname, Metrics: metrics.NewOptionalGatherer(), PublicKey: bls.PublicFromSecretKey(sk), - WarpSigner: warp.NewSigner(sk, networkID, chainID), ValidatorState: &validators.TestState{}, } diff --git a/examples/tokenvm/actions/burn_asset.go b/examples/tokenvm/actions/burn_asset.go index cbbda3ad83..8eb18e6bab 100644 --- a/examples/tokenvm/actions/burn_asset.go +++ b/examples/tokenvm/actions/burn_asset.go @@ -8,7 +8,6 @@ import ( "github.com/ava-labs/avalanchego/ids" smath "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" @@ -43,10 +42,6 @@ func (*BurnAsset) StateKeysMaxChunks() []uint16 { return []uint16{storage.AssetChunks, storage.BalanceChunks} } -func (*BurnAsset) OutputsWarpMessage() bool { - return false -} - func (b *BurnAsset) Execute( ctx context.Context, _ chain.Rules, @@ -55,14 +50,14 @@ func (b *BurnAsset) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if b.Value == 0 { return false, BurnComputeUnits, OutputValueZero, nil, nil } if err := storage.SubBalance(ctx, mu, actor, b.Asset, b.Value); err != nil { return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil } - exists, symbol, decimals, metadata, supply, owner, warp, err := storage.GetAsset(ctx, mu, b.Asset) + exists, symbol, decimals, metadata, supply, owner, err := storage.GetAsset(ctx, mu, b.Asset) if err != nil { return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil } @@ -73,7 +68,7 @@ func (b *BurnAsset) Execute( if err != nil { return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil } - if err := storage.SetAsset(ctx, mu, b.Asset, symbol, decimals, metadata, newSupply, owner, warp); err != nil { + if err := storage.SetAsset(ctx, mu, b.Asset, symbol, decimals, metadata, newSupply, owner); err != nil { return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil } return true, BurnComputeUnits, nil, nil, nil @@ -92,7 +87,7 @@ func (b *BurnAsset) Marshal(p *codec.Packer) { p.PackUint64(b.Value) } -func UnmarshalBurnAsset(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalBurnAsset(p *codec.Packer) (chain.Action, error) { var burn BurnAsset p.UnpackID(false, &burn.Asset) // can burn native asset burn.Value = p.UnpackUint64(true) diff --git a/examples/tokenvm/actions/close_order.go b/examples/tokenvm/actions/close_order.go index aa499b7eac..7990340f98 100644 --- a/examples/tokenvm/actions/close_order.go +++ b/examples/tokenvm/actions/close_order.go @@ -7,7 +7,6 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -42,10 +41,6 @@ func (*CloseOrder) StateKeysMaxChunks() []uint16 { return []uint16{storage.OrderChunks, storage.BalanceChunks} } -func (*CloseOrder) OutputsWarpMessage() bool { - return false -} - func (c *CloseOrder) Execute( ctx context.Context, _ chain.Rules, @@ -54,27 +49,27 @@ func (c *CloseOrder) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { exists, _, _, out, _, remaining, owner, err := storage.GetOrder(ctx, mu, c.Order) if err != nil { - return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil } if !exists { - return false, CloseOrderComputeUnits, OutputOrderMissing, nil, nil + return false, CloseOrderComputeUnits, OutputOrderMissing, nil } if owner != actor { - return false, CloseOrderComputeUnits, OutputUnauthorized, nil, nil + return false, CloseOrderComputeUnits, OutputUnauthorized, nil } if out != c.Out { - return false, CloseOrderComputeUnits, OutputWrongOut, nil, nil + return false, CloseOrderComputeUnits, OutputWrongOut, nil } if err := storage.DeleteOrder(ctx, mu, c.Order); err != nil { - return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil } if err := storage.AddBalance(ctx, mu, actor, c.Out, remaining, true); err != nil { - return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, CloseOrderComputeUnits, utils.ErrBytes(err), nil } - return true, CloseOrderComputeUnits, nil, nil, nil + return true, CloseOrderComputeUnits, nil, nil } func (*CloseOrder) MaxComputeUnits(chain.Rules) uint64 { @@ -90,7 +85,7 @@ func (c *CloseOrder) Marshal(p *codec.Packer) { p.PackID(c.Out) } -func UnmarshalCloseOrder(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalCloseOrder(p *codec.Packer) (chain.Action, error) { var cl CloseOrder p.UnpackID(true, &cl.Order) p.UnpackID(false, &cl.Out) // empty ID is the native asset diff --git a/examples/tokenvm/actions/create_asset.go b/examples/tokenvm/actions/create_asset.go index 7cc44ea9c7..e3a5dba162 100644 --- a/examples/tokenvm/actions/create_asset.go +++ b/examples/tokenvm/actions/create_asset.go @@ -7,7 +7,6 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -38,10 +37,6 @@ func (*CreateAsset) StateKeysMaxChunks() []uint16 { return []uint16{storage.AssetChunks} } -func (*CreateAsset) OutputsWarpMessage() bool { - return false -} - func (c *CreateAsset) Execute( ctx context.Context, _ chain.Rules, @@ -50,28 +45,28 @@ func (c *CreateAsset) Execute( actor codec.Address, txID ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if len(c.Symbol) == 0 { - return false, CreateAssetComputeUnits, OutputSymbolEmpty, nil, nil + return false, CreateAssetComputeUnits, OutputSymbolEmpty, nil } if len(c.Symbol) > MaxSymbolSize { - return false, CreateAssetComputeUnits, OutputSymbolTooLarge, nil, nil + return false, CreateAssetComputeUnits, OutputSymbolTooLarge, nil } if c.Decimals > MaxDecimals { - return false, CreateAssetComputeUnits, OutputDecimalsTooLarge, nil, nil + return false, CreateAssetComputeUnits, OutputDecimalsTooLarge, nil } if len(c.Metadata) == 0 { - return false, CreateAssetComputeUnits, OutputMetadataEmpty, nil, nil + return false, CreateAssetComputeUnits, OutputMetadataEmpty, nil } if len(c.Metadata) > MaxMetadataSize { - return false, CreateAssetComputeUnits, OutputMetadataTooLarge, nil, nil + return false, CreateAssetComputeUnits, OutputMetadataTooLarge, nil } // It should only be possible to overwrite an existing asset if there is // a hash collision. if err := storage.SetAsset(ctx, mu, txID, c.Symbol, c.Decimals, c.Metadata, 0, actor, false); err != nil { - return false, CreateAssetComputeUnits, utils.ErrBytes(err), nil, nil + return false, CreateAssetComputeUnits, utils.ErrBytes(err), nil } - return true, CreateAssetComputeUnits, nil, nil, nil + return true, CreateAssetComputeUnits, nil, nil } func (*CreateAsset) MaxComputeUnits(chain.Rules) uint64 { @@ -89,14 +84,6 @@ func (c *CreateAsset) Marshal(p *codec.Packer) { p.PackBytes(c.Metadata) } -func UnmarshalCreateAsset(p *codec.Packer, _ *warp.Message) (chain.Action, error) { - var create CreateAsset - p.UnpackBytes(MaxSymbolSize, true, &create.Symbol) - create.Decimals = p.UnpackByte() - p.UnpackBytes(MaxMetadataSize, true, &create.Metadata) - return &create, p.Err() -} - func (*CreateAsset) ValidRange(chain.Rules) (int64, int64) { // Returning -1, -1 means that the action is always valid. return -1, -1 diff --git a/examples/tokenvm/actions/create_order.go b/examples/tokenvm/actions/create_order.go index c8dd51e9d1..8435f05dd7 100644 --- a/examples/tokenvm/actions/create_order.go +++ b/examples/tokenvm/actions/create_order.go @@ -8,7 +8,6 @@ import ( "fmt" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -62,10 +61,6 @@ func (*CreateOrder) StateKeysMaxChunks() []uint16 { return []uint16{storage.BalanceChunks, storage.OrderChunks} } -func (*CreateOrder) OutputsWarpMessage() bool { - return false -} - func (c *CreateOrder) Execute( ctx context.Context, _ chain.Rules, @@ -74,29 +69,29 @@ func (c *CreateOrder) Execute( actor codec.Address, txID ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if c.In == c.Out { - return false, CreateOrderComputeUnits, OutputSameInOut, nil, nil + return false, CreateOrderComputeUnits, OutputSameInOut, nil } if c.InTick == 0 { - return false, CreateOrderComputeUnits, OutputInTickZero, nil, nil + return false, CreateOrderComputeUnits, OutputInTickZero, nil } if c.OutTick == 0 { - return false, CreateOrderComputeUnits, OutputOutTickZero, nil, nil + return false, CreateOrderComputeUnits, OutputOutTickZero, nil } if c.Supply == 0 { - return false, CreateOrderComputeUnits, OutputSupplyZero, nil, nil + return false, CreateOrderComputeUnits, OutputSupplyZero, nil } if c.Supply%c.OutTick != 0 { - return false, CreateOrderComputeUnits, OutputSupplyMisaligned, nil, nil + return false, CreateOrderComputeUnits, OutputSupplyMisaligned, nil } if err := storage.SubBalance(ctx, mu, actor, c.Out, c.Supply); err != nil { - return false, CreateOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, CreateOrderComputeUnits, utils.ErrBytes(err), nil } if err := storage.SetOrder(ctx, mu, txID, c.In, c.InTick, c.Out, c.OutTick, c.Supply, actor); err != nil { - return false, CreateOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, CreateOrderComputeUnits, utils.ErrBytes(err), nil } - return true, CreateOrderComputeUnits, nil, nil, nil + return true, CreateOrderComputeUnits, nil, nil } func (*CreateOrder) MaxComputeUnits(chain.Rules) uint64 { @@ -115,7 +110,7 @@ func (c *CreateOrder) Marshal(p *codec.Packer) { p.PackUint64(c.Supply) } -func UnmarshalCreateOrder(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalCreateOrder(p *codec.Packer) (chain.Action, error) { var create CreateOrder p.UnpackID(false, &create.In) // empty ID is the native asset create.InTick = p.UnpackUint64(true) diff --git a/examples/tokenvm/actions/export_asset.go b/examples/tokenvm/actions/export_asset.go deleted file mode 100644 index 6811698aef..0000000000 --- a/examples/tokenvm/actions/export_asset.go +++ /dev/null @@ -1,284 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package actions - -import ( - "context" - - "github.com/ava-labs/avalanchego/ids" - smath "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/consts" - "github.com/ava-labs/hypersdk/examples/tokenvm/storage" - "github.com/ava-labs/hypersdk/state" - "github.com/ava-labs/hypersdk/utils" -) - -var _ chain.Action = (*ExportAsset)(nil) - -type ExportAsset struct { - To codec.Address `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 { - return exportAssetID -} - -func (e *ExportAsset) StateKeys(actor codec.Address, _ ids.ID) state.Keys { - if e.Return { - return state.Keys{ - string(storage.AssetKey(e.Asset)): state.Read | state.Write, - string(storage.BalanceKey(actor, e.Asset)): state.Read | state.Write, - } - } - return state.Keys{ - string(storage.AssetKey(e.Asset)): state.Read | state.Write, - string(storage.LoanKey(e.Asset, e.Destination)): state.All, - string(storage.BalanceKey(actor, e.Asset)): state.Read | state.Write, - } -} - -func (e *ExportAsset) StateKeysMaxChunks() []uint16 { - if e.Return { - return []uint16{storage.AssetChunks, storage.BalanceChunks} - } - return []uint16{storage.AssetChunks, storage.LoanChunks, storage.BalanceChunks} -} - -func (*ExportAsset) OutputsWarpMessage() bool { - return true -} - -func (e *ExportAsset) executeReturn( - ctx context.Context, - mu state.Mutable, - actor codec.Address, - txID ids.ID, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { - exists, symbol, decimals, metadata, supply, _, isWarp, err := storage.GetAsset(ctx, mu, e.Asset) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if !exists { - return false, ExportAssetComputeUnits, OutputAssetMissing, nil, nil - } - if !isWarp { - return false, ExportAssetComputeUnits, OutputNotWarpAsset, nil, nil - } - allowedDestination, err := ids.ToID(metadata[consts.IDLen:]) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if allowedDestination != e.Destination { - return false, ExportAssetComputeUnits, OutputWrongDestination, nil, nil - } - newSupply, err := smath.Sub(supply, e.Value) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - newSupply, err = smath.Sub(newSupply, e.Reward) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if newSupply > 0 { - if err := storage.SetAsset(ctx, mu, e.Asset, symbol, decimals, metadata, newSupply, codec.EmptyAddress, true); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - } else { - if err := storage.DeleteAsset(ctx, mu, e.Asset); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - } - if err := storage.SubBalance(ctx, mu, actor, e.Asset, e.Value); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if e.Reward > 0 { - if err := storage.SubBalance(ctx, mu, actor, e.Asset, e.Reward); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - } - originalAsset, err := ids.ToID(metadata[:consts.IDLen]) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - wt := &WarpTransfer{ - To: e.To, - Symbol: symbol, - Decimals: decimals, - Asset: originalAsset, - Value: e.Value, - Return: e.Return, - Reward: e.Reward, - SwapIn: e.SwapIn, - AssetOut: e.AssetOut, - SwapOut: e.SwapOut, - SwapExpiry: e.SwapExpiry, - TxID: txID, - DestinationChainID: e.Destination, - } - payload, err := wt.Marshal() - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - wm := &warp.UnsignedMessage{ - // NetworkID + SourceChainID is populated by hypersdk - Payload: payload, - } - return true, ExportAssetComputeUnits, nil, wm, nil -} - -func (e *ExportAsset) executeLoan( - ctx context.Context, - mu state.Mutable, - actor codec.Address, - txID ids.ID, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { - exists, symbol, decimals, _, _, _, isWarp, err := storage.GetAsset(ctx, mu, e.Asset) - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if !exists { - return false, ExportAssetComputeUnits, OutputAssetMissing, nil, nil - } - if isWarp { - // Cannot export an asset if it was warped in and not returning - return false, ExportAssetComputeUnits, OutputWarpAsset, nil, nil - } - if err := storage.AddLoan(ctx, mu, e.Asset, e.Destination, e.Value); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if err := storage.SubBalance(ctx, mu, actor, e.Asset, e.Value); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if e.Reward > 0 { - if err := storage.AddLoan(ctx, mu, e.Asset, e.Destination, e.Reward); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if err := storage.SubBalance(ctx, mu, actor, e.Asset, e.Reward); err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - } - wt := &WarpTransfer{ - To: e.To, - Symbol: symbol, - Decimals: decimals, - Asset: e.Asset, - Value: e.Value, - Return: e.Return, - Reward: e.Reward, - SwapIn: e.SwapIn, - AssetOut: e.AssetOut, - SwapOut: e.SwapOut, - SwapExpiry: e.SwapExpiry, - TxID: txID, - DestinationChainID: e.Destination, - } - payload, err := wt.Marshal() - if err != nil { - return false, ExportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - wm := &warp.UnsignedMessage{ - // NetworkID + SourceChainID is populated by hypersdk - Payload: payload, - } - return true, ExportAssetComputeUnits, nil, wm, nil -} - -func (e *ExportAsset) Execute( - ctx context.Context, - _ chain.Rules, - mu state.Mutable, - _ int64, - actor codec.Address, - txID ids.ID, - _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { - if e.Value == 0 { - return false, ExportAssetComputeUnits, OutputValueZero, nil, nil - } - if e.Destination == ids.Empty { - // This would result in multiplying balance export by whoever imports the - // transaction. - return false, ExportAssetComputeUnits, OutputAnycast, nil, nil - } - // TODO: check if destination is ourselves - if e.Return { - return e.executeReturn(ctx, mu, actor, txID) - } - return e.executeLoan(ctx, mu, actor, txID) -} - -func (*ExportAsset) MaxComputeUnits(chain.Rules) uint64 { - return ExportAssetComputeUnits -} - -func (*ExportAsset) Size() int { - return codec.AddressLen + consts.IDLen + - consts.Uint64Len + consts.BoolLen + - consts.Uint64Len + /* op bits */ - consts.Uint64Len + consts.Uint64Len + consts.IDLen + consts.Uint64Len + - consts.Int64Len + consts.IDLen -} - -func (e *ExportAsset) Marshal(p *codec.Packer) { - p.PackAddress(e.To) - p.PackID(e.Asset) - p.PackUint64(e.Value) - p.PackBool(e.Return) - op := codec.NewOptionalWriter(consts.Uint64Len*3 + consts.Int64Len + consts.IDLen) - op.PackUint64(e.Reward) - op.PackUint64(e.SwapIn) - op.PackID(e.AssetOut) - op.PackUint64(e.SwapOut) - op.PackInt64(e.SwapExpiry) - p.PackOptional(op) - p.PackID(e.Destination) -} - -func UnmarshalExportAsset(p *codec.Packer, _ *warp.Message) (chain.Action, error) { - var export ExportAsset - p.UnpackAddress(&export.To) - p.UnpackID(false, &export.Asset) // may export native - export.Value = p.UnpackUint64(true) - export.Return = p.UnpackBool() - op := p.NewOptionalReader() - export.Reward = op.UnpackUint64() // reward not required - export.SwapIn = op.UnpackUint64() // optional - op.UnpackID(&export.AssetOut) - export.SwapOut = op.UnpackUint64() - export.SwapExpiry = op.UnpackInt64() - op.Done() - p.UnpackID(true, &export.Destination) - if err := p.Err(); err != nil { - return nil, err - } - // Handle swap checks - if !ValidSwapParams( - export.Value, - export.SwapIn, - export.AssetOut, - export.SwapOut, - export.SwapExpiry, - ) { - return nil, chain.ErrInvalidObject - } - return &export, nil -} - -func (*ExportAsset) ValidRange(chain.Rules) (int64, int64) { - // Returning -1, -1 means that the action is always valid. - return -1, -1 -} diff --git a/examples/tokenvm/actions/fill_order.go b/examples/tokenvm/actions/fill_order.go index bb216e77e1..199d1c134b 100644 --- a/examples/tokenvm/actions/fill_order.go +++ b/examples/tokenvm/actions/fill_order.go @@ -8,7 +8,6 @@ import ( "github.com/ava-labs/avalanchego/ids" smath "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" @@ -57,10 +56,6 @@ func (*FillOrder) StateKeysMaxChunks() []uint16 { return []uint16{storage.OrderChunks, storage.BalanceChunks, storage.BalanceChunks, storage.BalanceChunks} } -func (*FillOrder) OutputsWarpMessage() bool { - return false -} - func (f *FillOrder) Execute( ctx context.Context, _ chain.Rules, @@ -69,39 +64,39 @@ func (f *FillOrder) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { exists, in, inTick, out, outTick, remaining, owner, err := storage.GetOrder(ctx, mu, f.Order) if err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } if !exists { - return false, NoFillOrderComputeUnits, OutputOrderMissing, nil, nil + return false, NoFillOrderComputeUnits, OutputOrderMissing, nil } if owner != f.Owner { - return false, NoFillOrderComputeUnits, OutputWrongOwner, nil, nil + return false, NoFillOrderComputeUnits, OutputWrongOwner, nil } if in != f.In { - return false, NoFillOrderComputeUnits, OutputWrongIn, nil, nil + return false, NoFillOrderComputeUnits, OutputWrongIn, nil } if out != f.Out { - return false, NoFillOrderComputeUnits, OutputWrongOut, nil, nil + return false, NoFillOrderComputeUnits, OutputWrongOut, nil } if f.Value == 0 { // This should be guarded via [Unmarshal] but we check anyways. - return false, NoFillOrderComputeUnits, OutputValueZero, nil, nil + return false, NoFillOrderComputeUnits, OutputValueZero, nil } if f.Value%inTick != 0 { - return false, NoFillOrderComputeUnits, OutputValueMisaligned, nil, nil + return false, NoFillOrderComputeUnits, OutputValueMisaligned, nil } // Determine amount of [Out] counterparty will receive if the trade is // successful. outputAmount, err := smath.Mul64(outTick, f.Value/inTick) if err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } if outputAmount == 0 { // This should never happen because [f.Value] > 0 - return false, NoFillOrderComputeUnits, OutputInsufficientOutput, nil, nil + return false, NoFillOrderComputeUnits, OutputInsufficientOutput, nil } var ( inputAmount = f.Value @@ -127,32 +122,32 @@ func (f *FillOrder) Execute( } if inputAmount == 0 { // Don't allow free trades (can happen due to refund rounding) - return false, NoFillOrderComputeUnits, OutputInsufficientInput, nil, nil + return false, NoFillOrderComputeUnits, OutputInsufficientInput, nil } if err := storage.SubBalance(ctx, mu, actor, f.In, inputAmount); err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } if err := storage.AddBalance(ctx, mu, f.Owner, f.In, inputAmount, true); err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } if err := storage.AddBalance(ctx, mu, actor, f.Out, outputAmount, true); err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } if shouldDelete { if err := storage.DeleteOrder(ctx, mu, f.Order); err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } } else { if err := storage.SetOrder(ctx, mu, f.Order, in, inTick, out, outTick, orderRemaining, owner); err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } } or := &OrderResult{In: inputAmount, Out: outputAmount, Remaining: orderRemaining} output, err := or.Marshal() if err != nil { - return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil, nil + return false, NoFillOrderComputeUnits, utils.ErrBytes(err), nil } - return true, FillOrderComputeUnits, output, nil, nil + return true, FillOrderComputeUnits, output, nil } func (*FillOrder) MaxComputeUnits(chain.Rules) uint64 { @@ -171,7 +166,7 @@ func (f *FillOrder) Marshal(p *codec.Packer) { p.PackUint64(f.Value) } -func UnmarshalFillOrder(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalFillOrder(p *codec.Packer) (chain.Action, error) { var fill FillOrder p.UnpackID(true, &fill.Order) p.UnpackAddress(&fill.Owner) diff --git a/examples/tokenvm/actions/import_asset.go b/examples/tokenvm/actions/import_asset.go deleted file mode 100644 index 9af545778c..0000000000 --- a/examples/tokenvm/actions/import_asset.go +++ /dev/null @@ -1,292 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package actions - -import ( - "bytes" - "context" - - "github.com/ava-labs/avalanchego/ids" - smath "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/consts" - "github.com/ava-labs/hypersdk/examples/tokenvm/storage" - "github.com/ava-labs/hypersdk/state" - "github.com/ava-labs/hypersdk/utils" -) - -var _ chain.Action = (*ImportAsset)(nil) - -type ImportAsset struct { - // Fill indicates if the actor wishes to fill the order request in the warp - // message. This must be true if the warp message is in a block with - // a timestamp < [SwapExpiry]. - Fill bool `json:"fill"` - - // warpTransfer is parsed from the inner *warp.Message - warpTransfer *WarpTransfer - - // warpMessage is the full *warp.Message parsed from [chain.Transaction] - warpMessage *warp.Message -} - -func (*ImportAsset) GetTypeID() uint8 { - return importAssetID -} - -func (i *ImportAsset) StateKeys(actor codec.Address, _ ids.ID) state.Keys { - var ( - keys state.Keys - assetID ids.ID - ) - if i.warpTransfer.Return { - assetID = i.warpTransfer.Asset - keys = state.Keys{ - string(storage.AssetKey(i.warpTransfer.Asset)): state.Read | state.Write, - string(storage.LoanKey(i.warpTransfer.Asset, i.warpMessage.SourceChainID)): state.Read | state.Write, - string(storage.BalanceKey(i.warpTransfer.To, i.warpTransfer.Asset)): state.All, - } - } else { - assetID = ImportedAssetID(i.warpTransfer.Asset, i.warpMessage.SourceChainID) - keys = state.Keys{ - string(storage.AssetKey(assetID)): state.All, - string(storage.BalanceKey(i.warpTransfer.To, assetID)): state.All, - } - } - - // If the [warpTransfer] specified a reward, we add the state key to make - // sure it is paid. - if i.warpTransfer.Reward > 0 { - keys.Add(string(storage.BalanceKey(actor, assetID)), state.Read|state.Write) - } - - // If the [warpTransfer] requests a swap, we add the state keys to transfer - // the required balances. - if i.Fill && i.warpTransfer.SwapIn > 0 { - keys.Add(string(storage.BalanceKey(actor, i.warpTransfer.AssetOut)), state.All) - keys.Add(string(storage.BalanceKey(actor, assetID)), state.All) - keys.Add(string(storage.BalanceKey(i.warpTransfer.To, i.warpTransfer.AssetOut)), state.All) - } - return keys -} - -func (i *ImportAsset) StateKeysMaxChunks() []uint16 { - // Can't use [warpTransfer] because it may not be populated yet - chunks := []uint16{} - chunks = append(chunks, storage.LoanChunks) - chunks = append(chunks, storage.AssetChunks) - chunks = append(chunks, storage.BalanceChunks) - - // If the [warpTransfer] specified a reward, we add the state key to make - // sure it is paid. - chunks = append(chunks, storage.BalanceChunks) - - // If the [warpTransfer] requests a swap, we add the state keys to transfer - // the required balances. - if i.Fill { - chunks = append(chunks, storage.BalanceChunks) - chunks = append(chunks, storage.BalanceChunks) - chunks = append(chunks, storage.BalanceChunks) - } - return chunks -} - -func (*ImportAsset) OutputsWarpMessage() bool { - return false -} - -func (i *ImportAsset) executeMint( - ctx context.Context, - mu state.Mutable, - actor codec.Address, -) []byte { - asset := ImportedAssetID(i.warpTransfer.Asset, i.warpMessage.SourceChainID) - exists, symbol, decimals, metadata, supply, _, warp, err := storage.GetAsset(ctx, mu, asset) - if err != nil { - return utils.ErrBytes(err) - } - if exists && !warp { - // Should not be possible - return OutputConflictingAsset - } - if !exists { - symbol = i.warpTransfer.Symbol - decimals = i.warpTransfer.Decimals - metadata = ImportedAssetMetadata(i.warpTransfer.Asset, i.warpMessage.SourceChainID) - } - newSupply, err := smath.Add64(supply, i.warpTransfer.Value) - if err != nil { - return utils.ErrBytes(err) - } - newSupply, err = smath.Add64(newSupply, i.warpTransfer.Reward) - if err != nil { - return utils.ErrBytes(err) - } - if err := storage.SetAsset(ctx, mu, asset, symbol, decimals, metadata, newSupply, codec.EmptyAddress, true); err != nil { - return utils.ErrBytes(err) - } - if err := storage.AddBalance(ctx, mu, i.warpTransfer.To, asset, i.warpTransfer.Value, true); err != nil { - return utils.ErrBytes(err) - } - if i.warpTransfer.Reward > 0 { - if err := storage.AddBalance(ctx, mu, actor, asset, i.warpTransfer.Reward, true); err != nil { - return utils.ErrBytes(err) - } - } - return nil -} - -func (i *ImportAsset) executeReturn( - ctx context.Context, - mu state.Mutable, - actor codec.Address, -) []byte { - exists, symbol, decimals, _, _, _, warp, err := storage.GetAsset(ctx, mu, i.warpTransfer.Asset) - if err != nil { - return utils.ErrBytes(err) - } - if !exists { - return OutputAssetMissing - } - if !bytes.Equal(i.warpTransfer.Symbol, symbol) { - return OutputSymbolIncorrect - } - if i.warpTransfer.Decimals != decimals { - return OutputDecimalsIncorrect - } - if warp { - return OutputWarpAsset - } - if err := storage.SubLoan( - ctx, mu, i.warpTransfer.Asset, - i.warpMessage.SourceChainID, i.warpTransfer.Value, - ); err != nil { - return utils.ErrBytes(err) - } - if err := storage.AddBalance( - ctx, mu, i.warpTransfer.To, - i.warpTransfer.Asset, i.warpTransfer.Value, - true, - ); err != nil { - return utils.ErrBytes(err) - } - if i.warpTransfer.Reward > 0 { - if err := storage.SubLoan( - ctx, mu, i.warpTransfer.Asset, - i.warpMessage.SourceChainID, i.warpTransfer.Reward, - ); err != nil { - return utils.ErrBytes(err) - } - if err := storage.AddBalance( - ctx, mu, actor, - i.warpTransfer.Asset, i.warpTransfer.Reward, - true, - ); err != nil { - return utils.ErrBytes(err) - } - } - return nil -} - -func (i *ImportAsset) Execute( - ctx context.Context, - r chain.Rules, - mu state.Mutable, - t int64, - actor codec.Address, - _ ids.ID, - warpVerified bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { - if !warpVerified { - return false, ImportAssetComputeUnits, OutputWarpVerificationFailed, nil, nil - } - if i.warpTransfer.DestinationChainID != r.ChainID() { - return false, ImportAssetComputeUnits, OutputInvalidDestination, nil, nil - } - if i.warpTransfer.Value == 0 { - return false, ImportAssetComputeUnits, OutputValueZero, nil, nil - } - var output []byte - if i.warpTransfer.Return { - output = i.executeReturn(ctx, mu, actor) - } else { - output = i.executeMint(ctx, mu, actor) - } - if len(output) > 0 { - return false, ImportAssetComputeUnits, output, nil, nil - } - if i.warpTransfer.SwapIn == 0 { - // We are ensured that [i.Fill] is false here because of logic in unmarshal - return true, ImportAssetComputeUnits, nil, nil, nil - } - if !i.Fill { - if i.warpTransfer.SwapExpiry > t { - return false, ImportAssetComputeUnits, OutputMustFill, nil, nil - } - return true, ImportAssetComputeUnits, nil, nil, nil - } - // TODO: charge more if swap is performed - var assetIn ids.ID - if i.warpTransfer.Return { - assetIn = i.warpTransfer.Asset - } else { - assetIn = ImportedAssetID(i.warpTransfer.Asset, i.warpMessage.SourceChainID) - } - if err := storage.SubBalance(ctx, mu, i.warpTransfer.To, assetIn, i.warpTransfer.SwapIn); err != nil { - return false, ImportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if err := storage.AddBalance(ctx, mu, actor, assetIn, i.warpTransfer.SwapIn, true); err != nil { - return false, ImportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if err := storage.SubBalance(ctx, mu, actor, i.warpTransfer.AssetOut, i.warpTransfer.SwapOut); err != nil { - return false, ImportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - if err := storage.AddBalance(ctx, mu, i.warpTransfer.To, i.warpTransfer.AssetOut, i.warpTransfer.SwapOut, true); err != nil { - return false, ImportAssetComputeUnits, utils.ErrBytes(err), nil, nil - } - return true, ImportAssetComputeUnits, nil, nil, nil -} - -func (*ImportAsset) MaxComputeUnits(chain.Rules) uint64 { - return ImportAssetComputeUnits -} - -func (*ImportAsset) Size() int { - return consts.BoolLen -} - -// All we encode that is action specific for now is the type byte from the -// registry. -func (i *ImportAsset) Marshal(p *codec.Packer) { - p.PackBool(i.Fill) -} - -func UnmarshalImportAsset(p *codec.Packer, wm *warp.Message) (chain.Action, error) { - var ( - imp ImportAsset - err error - ) - imp.Fill = p.UnpackBool() - if err := p.Err(); err != nil { - return nil, err - } - imp.warpMessage = wm - imp.warpTransfer, err = UnmarshalWarpTransfer(imp.warpMessage.Payload) - if err != nil { - return nil, err - } - // Ensure we can fill the swap if it exists - if imp.Fill && imp.warpTransfer.SwapIn == 0 { - return nil, ErrNoSwapToFill - } - return &imp, nil -} - -func (*ImportAsset) ValidRange(chain.Rules) (int64, int64) { - // Returning -1, -1 means that the action is always valid. - return -1, -1 -} diff --git a/examples/tokenvm/actions/mint_asset.go b/examples/tokenvm/actions/mint_asset.go index a8eea1919e..c349b39609 100644 --- a/examples/tokenvm/actions/mint_asset.go +++ b/examples/tokenvm/actions/mint_asset.go @@ -8,7 +8,6 @@ import ( "github.com/ava-labs/avalanchego/ids" smath "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" @@ -46,10 +45,6 @@ func (*MintAsset) StateKeysMaxChunks() []uint16 { return []uint16{storage.AssetChunks, storage.BalanceChunks} } -func (*MintAsset) OutputsWarpMessage() bool { - return false -} - func (m *MintAsset) Execute( ctx context.Context, _ chain.Rules, @@ -58,37 +53,34 @@ func (m *MintAsset) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if m.Asset == ids.Empty { - return false, MintAssetComputeUnits, OutputAssetIsNative, nil, nil + return false, MintAssetComputeUnits, OutputAssetIsNative, nil } if m.Value == 0 { - return false, MintAssetComputeUnits, OutputValueZero, nil, nil + return false, MintAssetComputeUnits, OutputValueZero, nil } - exists, symbol, decimals, metadata, supply, owner, isWarp, err := storage.GetAsset(ctx, mu, m.Asset) + exists, symbol, decimals, metadata, supply, owner, err := storage.GetAsset(ctx, mu, m.Asset) if err != nil { - return false, MintAssetComputeUnits, utils.ErrBytes(err), nil, nil + return false, MintAssetComputeUnits, utils.ErrBytes(err), nil } if !exists { - return false, MintAssetComputeUnits, OutputAssetMissing, nil, nil - } - if isWarp { - return false, MintAssetComputeUnits, OutputWarpAsset, nil, nil + return false, MintAssetComputeUnits, OutputAssetMissing, nil } if owner != actor { - return false, MintAssetComputeUnits, OutputWrongOwner, nil, nil + return false, MintAssetComputeUnits, OutputWrongOwner, nil } newSupply, err := smath.Add64(supply, m.Value) if err != nil { - return false, MintAssetComputeUnits, utils.ErrBytes(err), nil, nil + return false, MintAssetComputeUnits, utils.ErrBytes(err), nil } - if err := storage.SetAsset(ctx, mu, m.Asset, symbol, decimals, metadata, newSupply, actor, isWarp); err != nil { - return false, MintAssetComputeUnits, utils.ErrBytes(err), nil, nil + if err := storage.SetAsset(ctx, mu, m.Asset, symbol, decimals, metadata, newSupply, actor); err != nil { + return false, MintAssetComputeUnits, utils.ErrBytes(err), nil } if err := storage.AddBalance(ctx, mu, m.To, m.Asset, m.Value, true); err != nil { - return false, MintAssetComputeUnits, utils.ErrBytes(err), nil, nil + return false, MintAssetComputeUnits, utils.ErrBytes(err), nil } - return true, MintAssetComputeUnits, nil, nil, nil + return true, MintAssetComputeUnits, nil, nil } func (*MintAsset) MaxComputeUnits(chain.Rules) uint64 { @@ -105,7 +97,7 @@ func (m *MintAsset) Marshal(p *codec.Packer) { p.PackUint64(m.Value) } -func UnmarshalMintAsset(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalMintAsset(p *codec.Packer) (chain.Action, error) { var mint MintAsset p.UnpackAddress(&mint.To) p.UnpackID(true, &mint.Asset) // empty ID is the native asset diff --git a/examples/tokenvm/actions/outputs.go b/examples/tokenvm/actions/outputs.go index 5a976e8b89..424443e823 100644 --- a/examples/tokenvm/actions/outputs.go +++ b/examples/tokenvm/actions/outputs.go @@ -29,12 +29,8 @@ var ( OutputMetadataEmpty = []byte("metadata is empty") OutputMetadataTooLarge = []byte("metadata is too large") OutputSameInOut = []byte("same asset used for in and out") - OutputConflictingAsset = []byte("warp has same asset as another") OutputAnycast = []byte("anycast output") - OutputNotWarpAsset = []byte("not warp asset") - OutputWarpAsset = []byte("warp asset") OutputWrongDestination = []byte("wrong destination") OutputMustFill = []byte("must fill request") - OutputWarpVerificationFailed = []byte("warp verification failed") OutputInvalidDestination = []byte("invalid destination") ) diff --git a/examples/tokenvm/actions/transfer.go b/examples/tokenvm/actions/transfer.go index e1b52d3995..29f05bb7ff 100644 --- a/examples/tokenvm/actions/transfer.go +++ b/examples/tokenvm/actions/transfer.go @@ -7,7 +7,6 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -47,10 +46,6 @@ func (*Transfer) StateKeysMaxChunks() []uint16 { return []uint16{storage.BalanceChunks, storage.BalanceChunks} } -func (*Transfer) OutputsWarpMessage() bool { - return false -} - func (t *Transfer) Execute( ctx context.Context, _ chain.Rules, @@ -59,21 +54,21 @@ func (t *Transfer) Execute( actor codec.Address, _ ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if t.Value == 0 { - return false, TransferComputeUnits, OutputValueZero, nil, nil + return false, TransferComputeUnits, OutputValueZero, nil } if len(t.Memo) > MaxMemoSize { - return false, CreateAssetComputeUnits, OutputMemoTooLarge, nil, nil + return false, CreateAssetComputeUnits, OutputMemoTooLarge, nil } if err := storage.SubBalance(ctx, mu, actor, t.Asset, t.Value); err != nil { - return false, TransferComputeUnits, utils.ErrBytes(err), nil, nil + return false, TransferComputeUnits, utils.ErrBytes(err), nil } // TODO: allow sender to configure whether they will pay to create if err := storage.AddBalance(ctx, mu, t.To, t.Asset, t.Value, true); err != nil { - return false, TransferComputeUnits, utils.ErrBytes(err), nil, nil + return false, TransferComputeUnits, utils.ErrBytes(err), nil } - return true, TransferComputeUnits, nil, nil, nil + return true, TransferComputeUnits, nil, nil } func (*Transfer) MaxComputeUnits(chain.Rules) uint64 { @@ -91,7 +86,7 @@ func (t *Transfer) Marshal(p *codec.Packer) { p.PackBytes(t.Memo) } -func UnmarshalTransfer(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalTransfer(p *codec.Packer) (chain.Action, error) { var transfer Transfer p.UnpackAddress(&transfer.To) p.UnpackID(false, &transfer.Asset) // empty ID is the native asset diff --git a/examples/tokenvm/actions/warp_transfer.go b/examples/tokenvm/actions/warp_transfer.go deleted file mode 100644 index 03c448df21..0000000000 --- a/examples/tokenvm/actions/warp_transfer.go +++ /dev/null @@ -1,156 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package actions - -import ( - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/codec" - "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/utils" -) - -type WarpTransfer struct { - To codec.Address `json:"to"` - Symbol []byte `json:"symbol"` - Decimals uint8 `json:"decimals"` - 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. - Return bool `json:"return"` - - // Reward is the amount of [Asset] to send the [Actor] that submits this - // transaction. - Reward uint64 `json:"reward"` - - // SwapIn is the amount of [Asset] we are willing to swap for [AssetOut]. - SwapIn uint64 `json:"swapIn"` - // AssetOut is the asset we are seeking to get for [SwapIn]. - AssetOut ids.ID `json:"assetOut"` - // SwapOut is the amount of [AssetOut] we are seeking. - SwapOut uint64 `json:"swapOut"` - // SwapExpiry is the unix timestamp at which the swap becomes invalid (and - // the message can be processed without a swap. - SwapExpiry int64 `json:"swapExpiry"` - - // TxID is the transaction that created this message. This is used to ensure - // there is WarpID uniqueness. - TxID ids.ID `json:"txID"` - - // DestinationChainID is the destination of this transfer. We assume this - // must be populated (not anycast). - DestinationChainID ids.ID `json:"destinationChainID"` -} - -func (w *WarpTransfer) size() int { - return codec.AddressLen + codec.BytesLen(w.Symbol) + consts.Uint8Len + consts.IDLen + - consts.Uint64Len + consts.BoolLen + - consts.Uint64Len + /* op bits */ - consts.Uint64Len + consts.Uint64Len + consts.IDLen + consts.Uint64Len + consts.Int64Len + - consts.IDLen + consts.IDLen -} - -func (w *WarpTransfer) Marshal() ([]byte, error) { - p := codec.NewWriter(w.size(), w.size()) - p.PackAddress(w.To) - p.PackBytes(w.Symbol) - p.PackByte(w.Decimals) - p.PackID(w.Asset) - p.PackUint64(w.Value) - p.PackBool(w.Return) - op := codec.NewOptionalWriter(consts.Uint64Len*3 + consts.IDLen + consts.Int64Len) - op.PackUint64(w.Reward) - op.PackUint64(w.SwapIn) - op.PackID(w.AssetOut) - op.PackUint64(w.SwapOut) - op.PackInt64(w.SwapExpiry) - p.PackOptional(op) - p.PackID(w.TxID) - p.PackID(w.DestinationChainID) - return p.Bytes(), p.Err() -} - -func ImportedAssetID(assetID ids.ID, sourceChainID ids.ID) ids.ID { - return utils.ToID(ImportedAssetMetadata(assetID, sourceChainID)) -} - -func ImportedAssetMetadata(assetID ids.ID, sourceChainID ids.ID) []byte { - k := make([]byte, consts.IDLen*2) - copy(k, assetID[:]) - copy(k[consts.IDLen:], sourceChainID[:]) - return k -} - -func UnmarshalWarpTransfer(b []byte) (*WarpTransfer, error) { - maxWarpTransferSize := codec.AddressLen + codec.BytesLenSize(MaxSymbolSize) + consts.Uint8Len + consts.IDLen + - consts.Uint64Len + consts.BoolLen + - consts.Uint64Len + /* op bits */ - consts.Uint64Len + consts.Uint64Len + consts.IDLen + consts.Uint64Len + consts.Int64Len + - consts.IDLen + consts.IDLen - - var transfer WarpTransfer - p := codec.NewReader(b, maxWarpTransferSize) - p.UnpackAddress(&transfer.To) - p.UnpackBytes(MaxSymbolSize, true, &transfer.Symbol) - transfer.Decimals = p.UnpackByte() - p.UnpackID(false, &transfer.Asset) - transfer.Value = p.UnpackUint64(true) - transfer.Return = p.UnpackBool() - op := p.NewOptionalReader() - transfer.Reward = op.UnpackUint64() // reward not required - transfer.SwapIn = op.UnpackUint64() // optional - op.UnpackID(&transfer.AssetOut) - transfer.SwapOut = op.UnpackUint64() - transfer.SwapExpiry = op.UnpackInt64() - op.Done() - p.UnpackID(true, &transfer.TxID) - p.UnpackID(true, &transfer.DestinationChainID) - if err := p.Err(); err != nil { - return nil, err - } - if !p.Empty() { - return nil, chain.ErrInvalidObject - } - // Handle swap checks - if !ValidSwapParams( - transfer.Value, - transfer.SwapIn, - transfer.AssetOut, - transfer.SwapOut, - transfer.SwapExpiry, - ) { - return nil, chain.ErrInvalidObject - } - return &transfer, nil -} - -func ValidSwapParams( - value uint64, - swapIn uint64, - assetOut ids.ID, - swapOut uint64, - swapExpiry int64, -) bool { - if swapExpiry < 0 { - return false - } - if swapIn > value { - return false - } - if swapIn > 0 { - return swapOut != 0 - } - if assetOut != ids.Empty { - return false - } - if swapOut != 0 { - return false - } - if swapExpiry != 0 { - return false - } - return true -} diff --git a/examples/tokenvm/auth/ed25519.go b/examples/tokenvm/auth/ed25519.go index fc5a088b39..ee78b9a0bc 100644 --- a/examples/tokenvm/auth/ed25519.go +++ b/examples/tokenvm/auth/ed25519.go @@ -6,7 +6,6 @@ package auth import ( "context" - "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" @@ -71,7 +70,7 @@ func (d *ED25519) Marshal(p *codec.Packer) { p.PackFixedBytes(d.Signature[:]) } -func UnmarshalED25519(p *codec.Packer, _ *warp.Message) (chain.Auth, error) { +func UnmarshalED25519(p *codec.Packer) (chain.Auth, error) { var d ED25519 signer := d.Signer[:] // avoid allocating additional memory p.UnpackFixedBytes(ed25519.PublicKeyLen, &signer) diff --git a/examples/tokenvm/cmd/token-cli/cmd/action.go b/examples/tokenvm/cmd/token-cli/cmd/action.go index b5e5cfa614..eeb991e602 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/action.go +++ b/examples/tokenvm/cmd/token-cli/cmd/action.go @@ -11,7 +11,6 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/set" - "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/consts" @@ -193,7 +192,7 @@ var mintAssetCmd = &cobra.Command{ if err != nil { return err } - exists, symbol, decimals, metadata, supply, owner, warp, err := tcli.Asset(ctx, assetID, false) + exists, symbol, decimals, metadata, supply, owner, err := tcli.Asset(ctx, assetID, false) if err != nil { return err } @@ -202,11 +201,6 @@ var mintAssetCmd = &cobra.Command{ hutils.Outf("{{red}}exiting...{{/}}\n") return nil } - if warp { - hutils.Outf("{{red}}cannot mint a warped asset{{/}}\n", assetID) - hutils.Outf("{{red}}exiting...{{/}}\n") - return nil - } if owner != codec.MustAddressBech32(tconsts.HRP, priv.Address) { hutils.Outf("{{red}}%s is the owner of %s, you are not{{/}}\n", owner, assetID) hutils.Outf("{{red}}exiting...{{/}}\n") @@ -298,7 +292,7 @@ var createOrderCmd = &cobra.Command{ if err != nil { return err } - exists, symbol, decimals, metadata, supply, _, warp, err := tcli.Asset(ctx, inAssetID, false) + exists, symbol, decimals, metadata, supply, _, err := tcli.Asset(ctx, inAssetID, false) if err != nil { return err } @@ -309,12 +303,11 @@ var createOrderCmd = &cobra.Command{ return nil } hutils.Outf( - "{{yellow}}symbol:{{/}} %s {{yellow}}decimals:{{/}} %d {{yellow}}metadata:{{/}} %s {{yellow}}supply:{{/}} %d {{yellow}}warp:{{/}} %t\n", + "{{yellow}}symbol:{{/}} %s {{yellow}}decimals:{{/}} %d {{yellow}}metadata:{{/}} %s {{yellow}}supply:{{/}} %d\n", string(symbol), decimals, string(metadata), supply, - warp, ) } @@ -490,295 +483,3 @@ var fillOrderCmd = &cobra.Command{ return err }, } - -func performImport( - ctx context.Context, - scli *rpc.JSONRPCClient, - dcli *rpc.JSONRPCClient, - dscli *rpc.WebSocketClient, - dtcli *trpc.JSONRPCClient, - exportTxID ids.ID, - factory chain.AuthFactory, -) error { - // Select TxID (if not provided) - var err error - if exportTxID == ids.Empty { - exportTxID, err = handler.Root().PromptID("export txID") - if err != nil { - return err - } - } - - // Generate warp signature (as long as >= 80% stake) - var ( - msg *warp.Message - subnetWeight, sigWeight uint64 - ) - for ctx.Err() == nil { - msg, subnetWeight, sigWeight, err = scli.GenerateAggregateWarpSignature(ctx, exportTxID) - if sigWeight >= (subnetWeight*4)/5 && err == nil { - break - } - if err == nil { - hutils.Outf( - "{{yellow}}waiting for signature weight:{{/}} %d {{yellow}}observed:{{/}} %d\n", - subnetWeight, - sigWeight, - ) - } else { - hutils.Outf("{{red}}encountered error:{{/}} %v\n", err) - } - cont, err := handler.Root().PromptBool("try again") - if err != nil { - return err - } - if !cont { - hutils.Outf("{{red}}exiting...{{/}}\n") - return nil - } - } - if ctx.Err() != nil { - return ctx.Err() - } - wt, err := actions.UnmarshalWarpTransfer(msg.UnsignedMessage.Payload) - if err != nil { - return err - } - outputAssetID := wt.Asset - if !wt.Return { - outputAssetID = actions.ImportedAssetID(wt.Asset, msg.SourceChainID) - } - hutils.Outf( - "%s {{yellow}}to:{{/}} %s {{yellow}}source assetID:{{/}} %s {{yellow}}source symbol:{{/}} %s {{yellow}}output assetID:{{/}} %s {{yellow}}value:{{/}} %s {{yellow}}reward:{{/}} %s {{yellow}}return:{{/}} %t\n", - hutils.ToID( - msg.UnsignedMessage.Payload, - ), - codec.MustAddressBech32(tconsts.HRP, wt.To), - wt.Asset, - wt.Symbol, - outputAssetID, - hutils.FormatBalance(wt.Value, wt.Decimals), - hutils.FormatBalance(wt.Reward, wt.Decimals), - wt.Return, - ) - if wt.SwapIn > 0 { - _, outSymbol, outDecimals, _, _, _, _, err := dtcli.Asset(ctx, wt.AssetOut, false) - if err != nil { - return err - } - hutils.Outf( - "{{yellow}}asset in:{{/}} %s {{yellow}}swap in:{{/}} %s {{yellow}}asset out:{{/}} %s {{yellow}}symbol out:{{/}} %s {{yellow}}swap out:{{/}} %s {{yellow}}swap expiry:{{/}} %d\n", - outputAssetID, - hutils.FormatBalance(wt.SwapIn, wt.Decimals), - wt.AssetOut, - outSymbol, - hutils.FormatBalance(wt.SwapOut, outDecimals), - wt.SwapExpiry, - ) - } - hutils.Outf( - "{{yellow}}signature weight:{{/}} %d {{yellow}}total weight:{{/}} %d\n", - sigWeight, - subnetWeight, - ) - - // Select fill - var fill bool - if wt.SwapIn > 0 { - fill, err = handler.Root().PromptBool("fill") - if err != nil { - return err - } - } - if !fill && wt.SwapExpiry > time.Now().UnixMilli() { - return ErrMustFill - } - - // Generate transaction - _, _, err = sendAndWait(ctx, msg, &actions.ImportAsset{ - Fill: fill, - }, dcli, dscli, dtcli, factory, true) - return err -} - -var importAssetCmd = &cobra.Command{ - Use: "import-asset", - RunE: func(*cobra.Command, []string) error { - ctx := context.Background() - currentChainID, _, factory, dcli, dscli, dtcli, err := handler.DefaultActor() - if err != nil { - return err - } - - // Select source - _, uris, err := handler.Root().PromptChain("sourceChainID", set.Of(currentChainID)) - if err != nil { - return err - } - scli := rpc.NewJSONRPCClient(uris[0]) - - // Perform import - return performImport(ctx, scli, dcli, dscli, dtcli, ids.Empty, factory) - }, -} - -var exportAssetCmd = &cobra.Command{ - Use: "export-asset", - RunE: func(*cobra.Command, []string) error { - ctx := context.Background() - currentChainID, priv, factory, cli, scli, tcli, err := handler.DefaultActor() - if err != nil { - return err - } - - // Select token to send - assetID, err := handler.Root().PromptAsset("assetID", true) - if err != nil { - return err - } - _, decimals, balance, sourceChainID, err := handler.GetAssetInfo(ctx, tcli, priv.Address, assetID, true) - if balance == 0 || err != nil { - return err - } - - // Select recipient - recipient, err := handler.Root().PromptAddress("recipient") - if err != nil { - return err - } - - // Select amount - amount, err := handler.Root().PromptAmount("amount", decimals, balance, nil) - if err != nil { - return err - } - - // Determine return - var ret bool - if sourceChainID != ids.Empty { - ret = true - } - - // Select reward - reward, err := handler.Root().PromptAmount("reward", decimals, balance-amount, nil) - if err != nil { - return err - } - - // Determine destination - destination := sourceChainID - if !ret { - destination, _, err = handler.Root().PromptChain("destination", set.Of(currentChainID)) - if err != nil { - return err - } - } - - // Determine if swap in - swap, err := handler.Root().PromptBool("swap on import") - if err != nil { - return err - } - var ( - swapIn uint64 - assetOut ids.ID - swapOut uint64 - swapExpiry int64 - ) - if swap { - swapIn, err = handler.Root().PromptAmount("swap in", decimals, amount, nil) - if err != nil { - return err - } - assetOut, err = handler.Root().PromptAsset("asset out (on destination)", true) - if err != nil { - return err - } - uris, err := handler.Root().GetChain(destination) - if err != nil { - return err - } - networkID, _, _, err := cli.Network(ctx) - if err != nil { - return err - } - dcli := trpc.NewJSONRPCClient(uris[0], networkID, destination) - _, decimals, _, _, err := handler.GetAssetInfo(ctx, dcli, priv.Address, assetOut, false) - if err != nil { - return err - } - swapOut, err = handler.Root().PromptAmount( - "swap out (on destination, no decimals)", - decimals, - consts.MaxUint64, - nil, - ) - if err != nil { - return err - } - swapExpiry, err = handler.Root().PromptTime("swap expiry") - if err != nil { - return err - } - } - - // Confirm action - cont, err := handler.Root().PromptContinue() - if !cont || err != nil { - return err - } - - // Generate transaction - success, txID, err := sendAndWait(ctx, nil, &actions.ExportAsset{ - To: recipient, - Asset: assetID, - Value: amount, - Return: ret, - Reward: reward, - SwapIn: swapIn, - AssetOut: assetOut, - SwapOut: swapOut, - SwapExpiry: swapExpiry, - Destination: destination, - }, cli, scli, tcli, factory, true) - if err != nil { - return err - } - if !success { - return errors.New("not successful") - } - - // Perform import - imp, err := handler.Root().PromptBool("perform import on destination") - if err != nil { - return err - } - if imp { - uris, err := handler.Root().GetChain(destination) - if err != nil { - return err - } - networkID, _, _, err := cli.Network(ctx) - if err != nil { - return err - } - dscli, err := rpc.NewWebSocketClient(uris[0], rpc.DefaultHandshakeTimeout, pubsub.MaxPendingMessages, pubsub.MaxReadMessageSize) - if err != nil { - return err - } - if err := performImport(ctx, cli, rpc.NewJSONRPCClient(uris[0]), dscli, trpc.NewJSONRPCClient(uris[0], networkID, destination), txID, factory); err != nil { - return err - } - } - - // Ask if user would like to switch to destination chain - sw, err := handler.Root().PromptBool("switch default chain to destination") - if err != nil { - return err - } - if !sw { - return nil - } - return handler.Root().StoreDefaultChain(destination) - }, -} diff --git a/examples/tokenvm/cmd/token-cli/cmd/handler.go b/examples/tokenvm/cmd/token-cli/cmd/handler.go index c3fee8c16c..2c303adc82 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/handler.go +++ b/examples/tokenvm/cmd/token-cli/cmd/handler.go @@ -42,7 +42,7 @@ func (*Handler) GetAssetInfo( checkBalance bool, ) ([]byte, uint8, uint64, ids.ID, error) { var sourceChainID ids.ID - exists, symbol, decimals, metadata, supply, _, warp, err := cli.Asset(ctx, assetID, false) + exists, symbol, decimals, metadata, supply, _, err := cli.Asset(ctx, assetID, false) if err != nil { return nil, 0, 0, ids.Empty, err } @@ -52,25 +52,13 @@ func (*Handler) GetAssetInfo( hutils.Outf("{{red}}exiting...{{/}}\n") return nil, 0, 0, ids.Empty, nil } - if warp { - sourceChainID = ids.ID(metadata[hconsts.IDLen:]) - sourceAssetID := ids.ID(metadata[:hconsts.IDLen]) - hutils.Outf( - "{{yellow}}sourceChainID:{{/}} %s {{yellow}}sourceAssetID:{{/}} %s {{yellow}}supply:{{/}} %d\n", - sourceChainID, - sourceAssetID, - supply, - ) - } else { - hutils.Outf( - "{{yellow}}symbol:{{/}} %s {{yellow}}decimals:{{/}} %d {{yellow}}metadata:{{/}} %s {{yellow}}supply:{{/}} %d {{yellow}}warp:{{/}} %t\n", - symbol, - decimals, - metadata, - supply, - warp, - ) - } + hutils.Outf( + "{{yellow}}symbol:{{/}} %s {{yellow}}decimals:{{/}} %d {{yellow}}metadata:{{/}} %s {{yellow}}supply:{{/}} %d\n", + symbol, + decimals, + metadata, + supply, + ) } if !checkBalance { return symbol, decimals, 0, sourceChainID, nil diff --git a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go index eb62abfbb2..54dce659f6 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go +++ b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go @@ -9,7 +9,6 @@ import ( "reflect" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/cli" "github.com/ava-labs/hypersdk/codec" @@ -22,14 +21,14 @@ import ( // sendAndWait may not be used concurrently func sendAndWait( - ctx context.Context, warpMsg *warp.Message, action chain.Action, cli *rpc.JSONRPCClient, + ctx context.Context, action chain.Action, cli *rpc.JSONRPCClient, scli *rpc.WebSocketClient, tcli *trpc.JSONRPCClient, factory chain.AuthFactory, printStatus bool, ) (bool, ids.ID, error) { parser, err := tcli.Parser(ctx) if err != nil { return false, ids.Empty, err } - _, tx, _, err := cli.GenerateTransaction(ctx, parser, warpMsg, action, factory) + _, tx, _, err := cli.GenerateTransaction(ctx, parser, action, factory) if err != nil { return false, ids.Empty, err } @@ -126,50 +125,6 @@ func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result ) case *actions.CloseOrder: summaryStr = fmt.Sprintf("orderID: %s", action.Order) - - case *actions.ImportAsset: - wm := tx.WarpMessage - signers, _ := wm.Signature.NumSigners() - wt, _ := actions.UnmarshalWarpTransfer(wm.Payload) - summaryStr = fmt.Sprintf("source: %s signers: %d | ", wm.SourceChainID, signers) - if wt.Return { - summaryStr += fmt.Sprintf("%s %s -> %s (return: %t)", utils.FormatBalance(wt.Value, wt.Decimals), wt.Symbol, codec.MustAddressBech32(tconsts.HRP, wt.To), wt.Return) - } else { - summaryStr += fmt.Sprintf("%s %s (new: %s, original: %s) -> %s (return: %t)", utils.FormatBalance(wt.Value, wt.Decimals), wt.Symbol, actions.ImportedAssetID(wt.Asset, wm.SourceChainID), wt.Asset, codec.MustAddressBech32(tconsts.HRP, wt.To), wt.Return) - } - if wt.Reward > 0 { - summaryStr += fmt.Sprintf(" | reward: %s", utils.FormatBalance(wt.Reward, wt.Decimals)) - } - if wt.SwapIn > 0 { - _, outSymbol, outDecimals, _, _, _, _, err := c.Asset(context.TODO(), wt.AssetOut, true) - if err != nil { - utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) - return - } - summaryStr += fmt.Sprintf(" | swap in: %s %s swap out: %s %s expiry: %d fill: %t", utils.FormatBalance(wt.SwapIn, wt.Decimals), wt.Symbol, utils.FormatBalance(wt.SwapOut, outDecimals), outSymbol, wt.SwapExpiry, action.Fill) - } - case *actions.ExportAsset: - wt, _ := actions.UnmarshalWarpTransfer(result.WarpMessage.Payload) - summaryStr = fmt.Sprintf("destination: %s | ", action.Destination) - var outputAssetID ids.ID - if !action.Return { - outputAssetID = actions.ImportedAssetID(action.Asset, result.WarpMessage.SourceChainID) - summaryStr += fmt.Sprintf("%s %s (%s) -> %s (return: %t)", utils.FormatBalance(action.Value, wt.Decimals), wt.Symbol, action.Asset, codec.MustAddressBech32(tconsts.HRP, action.To), action.Return) - } else { - outputAssetID = wt.Asset - summaryStr += fmt.Sprintf("%s %s (current: %s, original: %s) -> %s (return: %t)", utils.FormatBalance(action.Value, wt.Decimals), wt.Symbol, action.Asset, wt.Asset, codec.MustAddressBech32(tconsts.HRP, action.To), action.Return) - } - if wt.Reward > 0 { - summaryStr += fmt.Sprintf(" | reward: %s", utils.FormatBalance(wt.Reward, wt.Decimals)) - } - if wt.SwapIn > 0 { - _, outSymbol, outDecimals, _, _, _, _, err := c.Asset(context.TODO(), wt.AssetOut, true) - if err != nil { - utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) - return - } - summaryStr += fmt.Sprintf(" | swap in: %s %s (%s) swap out: %s %s expiry: %d", utils.FormatBalance(wt.SwapIn, wt.Decimals), wt.Symbol, outputAssetID, utils.FormatBalance(wt.SwapOut, outDecimals), outSymbol, wt.SwapExpiry) - } } } utils.Outf( diff --git a/examples/tokenvm/consts/consts.go b/examples/tokenvm/consts/consts.go index e188cad88d..b221ebddfc 100644 --- a/examples/tokenvm/consts/consts.go +++ b/examples/tokenvm/consts/consts.go @@ -5,7 +5,6 @@ package consts import ( "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -33,6 +32,6 @@ func init() { // Instantiate registry here so it can be imported by any package. We set these // values in [controller/registry]. var ( - ActionRegistry *codec.TypeParser[chain.Action, *warp.Message, bool] - AuthRegistry *codec.TypeParser[chain.Auth, *warp.Message, bool] + ActionRegistry *codec.TypeParser[chain.Action, bool] + AuthRegistry *codec.TypeParser[chain.Auth, bool] ) diff --git a/examples/tokenvm/controller/state_manager.go b/examples/tokenvm/controller/state_manager.go index c3cfb34993..3a1fe82dce 100644 --- a/examples/tokenvm/controller/state_manager.go +++ b/examples/tokenvm/controller/state_manager.go @@ -29,14 +29,6 @@ func (*StateManager) FeeKey() []byte { return storage.FeeKey() } -func (*StateManager) IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) []byte { - return storage.IncomingWarpKeyPrefix(sourceChainID, msgID) -} - -func (*StateManager) OutgoingWarpKeyPrefix(txID ids.ID) []byte { - return storage.OutgoingWarpKeyPrefix(txID) -} - func (*StateManager) SponsorStateKeys(addr codec.Address) state.Keys { return state.Keys{ string(storage.BalanceKey(addr, ids.Empty)): state.Read | state.Write, diff --git a/examples/tokenvm/genesis/genesis.go b/examples/tokenvm/genesis/genesis.go index 23f45e91c8..983ff98763 100644 --- a/examples/tokenvm/genesis/genesis.go +++ b/examples/tokenvm/genesis/genesis.go @@ -48,9 +48,6 @@ type Genesis struct { // Tx Fee Parameters BaseComputeUnits uint64 `json:"baseUnits"` - BaseWarpComputeUnits uint64 `json:"baseWarpUnits"` - WarpComputeUnitsPerSigner uint64 `json:"warpUnitsPerSigner"` - OutgoingWarpComputeUnits uint64 `json:"outgoingWarpComputeUnits"` StorageKeyReadUnits uint64 `json:"storageKeyReadUnits"` StorageValueReadUnits uint64 `json:"storageValueReadUnits"` // per chunk StorageKeyAllocateUnits uint64 `json:"storageKeyAllocateUnits"` @@ -82,9 +79,6 @@ func Default() *Genesis { // Tx Fee Compute Parameters BaseComputeUnits: 1, - BaseWarpComputeUnits: 1_024, - WarpComputeUnitsPerSigner: 128, - OutgoingWarpComputeUnits: 1_024, // Tx Fee Storage Parameters // diff --git a/examples/tokenvm/genesis/rules.go b/examples/tokenvm/genesis/rules.go index b2fac9c3f5..c787c5ac6f 100644 --- a/examples/tokenvm/genesis/rules.go +++ b/examples/tokenvm/genesis/rules.go @@ -24,14 +24,6 @@ func (g *Genesis) Rules(_ int64, networkID uint32, chainID ids.ID) *Rules { return &Rules{g, networkID, chainID} } -func (*Rules) GetWarpConfig(ids.ID) (bool, uint64, uint64) { - // We allow inbound transfers from all sources as long as 80% of stake has - // signed a message. - // - // This is safe because the tokenvm scopes all assets by their source chain. - return true, 4, 5 -} - func (r *Rules) NetworkID() uint32 { return r.networkID } @@ -60,18 +52,6 @@ func (r *Rules) GetBaseComputeUnits() uint64 { return r.g.BaseComputeUnits } -func (r *Rules) GetBaseWarpComputeUnits() uint64 { - return r.g.BaseWarpComputeUnits -} - -func (r *Rules) GetWarpComputeUnitsPerSigner() uint64 { - return r.g.WarpComputeUnitsPerSigner -} - -func (r *Rules) GetOutgoingWarpComputeUnits() uint64 { - return r.g.OutgoingWarpComputeUnits -} - func (*Rules) GetSponsorStateKeysMaxChunks() []uint16 { return []uint16{storage.BalanceChunks} } diff --git a/examples/tokenvm/registry/registry.go b/examples/tokenvm/registry/registry.go index be08862f60..b330518a5d 100644 --- a/examples/tokenvm/registry/registry.go +++ b/examples/tokenvm/registry/registry.go @@ -5,7 +5,6 @@ package registry import ( "github.com/ava-labs/avalanchego/utils/wrappers" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" @@ -16,8 +15,8 @@ import ( // Setup types func init() { - consts.ActionRegistry = codec.NewTypeParser[chain.Action, *warp.Message]() - consts.AuthRegistry = codec.NewTypeParser[chain.Auth, *warp.Message]() + consts.ActionRegistry = codec.NewTypeParser[chain.Action]() + consts.AuthRegistry = codec.NewTypeParser[chain.Auth]() errs := &wrappers.Errs{} errs.Add( @@ -32,9 +31,6 @@ func init() { consts.ActionRegistry.Register((&actions.FillOrder{}).GetTypeID(), actions.UnmarshalFillOrder, false), consts.ActionRegistry.Register((&actions.CloseOrder{}).GetTypeID(), actions.UnmarshalCloseOrder, false), - consts.ActionRegistry.Register((&actions.ImportAsset{}).GetTypeID(), actions.UnmarshalImportAsset, true), - consts.ActionRegistry.Register((&actions.ExportAsset{}).GetTypeID(), actions.UnmarshalExportAsset, false), - // When registering new auth, ALWAYS make sure to append at the end. consts.AuthRegistry.Register((&auth.ED25519{}).GetTypeID(), auth.UnmarshalED25519, false), ) diff --git a/examples/tokenvm/rpc/jsonrpc_client.go b/examples/tokenvm/rpc/jsonrpc_client.go index a4916c3a03..de55235e1b 100644 --- a/examples/tokenvm/rpc/jsonrpc_client.go +++ b/examples/tokenvm/rpc/jsonrpc_client.go @@ -91,7 +91,7 @@ func (cli *JSONRPCClient) Asset( r, ok := cli.assets[asset] cli.assetsL.Unlock() if ok && useCache { - return true, r.Symbol, r.Decimals, r.Metadata, r.Supply, r.Owner, r.Warp, nil + return true, r.Symbol, r.Decimals, r.Metadata, r.Supply, r.Owner, nil } resp := new(AssetReply) err := cli.requester.SendRequest( @@ -113,7 +113,7 @@ func (cli *JSONRPCClient) Asset( cli.assetsL.Lock() cli.assets[asset] = resp cli.assetsL.Unlock() - return true, resp.Symbol, resp.Decimals, resp.Metadata, resp.Supply, resp.Owner, resp.Warp, nil + return true, resp.Symbol, resp.Decimals, resp.Metadata, resp.Supply, resp.Owner, nil } func (cli *JSONRPCClient) Balance(ctx context.Context, addr string, asset ids.ID) (uint64, error) { diff --git a/examples/tokenvm/rpc/jsonrpc_server.go b/examples/tokenvm/rpc/jsonrpc_server.go index fd94f5a94c..da64956056 100644 --- a/examples/tokenvm/rpc/jsonrpc_server.go +++ b/examples/tokenvm/rpc/jsonrpc_server.go @@ -71,14 +71,13 @@ type AssetReply struct { Metadata []byte `json:"metadata"` Supply uint64 `json:"supply"` Owner string `json:"owner"` - Warp bool `json:"warp"` } func (j *JSONRPCServer) Asset(req *http.Request, args *AssetArgs, reply *AssetReply) error { ctx, span := j.c.Tracer().Start(req.Context(), "Server.Asset") defer span.End() - exists, symbol, decimals, metadata, supply, owner, warp, err := j.c.GetAssetFromState(ctx, args.Asset) + exists, symbol, decimals, metadata, supply, owner, err := j.c.GetAssetFromState(ctx, args.Asset) if err != nil { return err } @@ -90,7 +89,6 @@ func (j *JSONRPCServer) Asset(req *http.Request, args *AssetArgs, reply *AssetRe reply.Metadata = metadata reply.Supply = supply reply.Owner = codec.MustAddressBech32(consts.HRP, owner) - reply.Warp = warp return err } diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index 2dcf9cb364..35e1b9c64e 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -30,7 +30,7 @@ type ReadState func(context.Context, [][]byte) ([][]byte, []error) // 0x0/ (balance) // -> [owner|asset] => balance // 0x1/ (assets) -// -> [asset] => metadataLen|metadata|supply|owner|warp +// -> [asset] => metadataLen|metadata|supply|owner // 0x2/ (orders) // -> [txID] => in|out|rate|remaining|owner // 0x3/ (loans) @@ -38,8 +38,6 @@ type ReadState func(context.Context, [][]byte) ([][]byte, []error) // 0x4/ (hypersdk-height) // 0x5/ (hypersdk-timestamp) // 0x6/ (hypersdk-fee) -// 0x7/ (hypersdk-incoming warp) -// 0x8/ (hypersdk-outgoing warp) const ( // metaDB @@ -53,8 +51,6 @@ const ( heightPrefix = 0x4 timestampPrefix = 0x5 feePrefix = 0x6 - incomingWarpPrefix = 0x7 - outgoingWarpPrefix = 0x8 ) const ( @@ -330,8 +326,7 @@ func innerGetAsset( supply := binary.BigEndian.Uint64(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen:]) var addr codec.Address copy(addr[:], v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len:]) - warp := v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len+codec.AddressLen] == 0x1 - return true, symbol, decimals, metadata, supply, addr, warp, nil + return true, symbol, decimals, metadata, supply, addr, nil } func SetAsset( @@ -343,7 +338,6 @@ func SetAsset( metadata []byte, supply uint64, owner codec.Address, - warp bool, ) error { k := AssetKey(asset) symbolLen := len(symbol) @@ -357,9 +351,6 @@ func SetAsset( binary.BigEndian.PutUint64(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen:], supply) copy(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len:], owner[:]) b := byte(0x0) - if warp { - b = 0x1 - } v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len+codec.AddressLen] = b return mu.Insert(ctx, k, v) } @@ -588,18 +579,3 @@ func TimestampKey() (k []byte) { func FeeKey() (k []byte) { return feeKey } - -func IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen*2) - k[0] = incomingWarpPrefix - copy(k[1:], sourceChainID[:]) - copy(k[1+consts.IDLen:], msgID[:]) - return k -} - -func OutgoingWarpKeyPrefix(txID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen) - k[0] = outgoingWarpPrefix - copy(k[1:], txID[:]) - return k -} diff --git a/examples/tokenvm/tests/e2e/e2e_test.go b/examples/tokenvm/tests/e2e/e2e_test.go index ac1c43e6bc..5b73854c1a 100644 --- a/examples/tokenvm/tests/e2e/e2e_test.go +++ b/examples/tokenvm/tests/e2e/e2e_test.go @@ -16,7 +16,6 @@ import ( "github.com/ava-labs/avalanchego/config" "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/codec" "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" @@ -583,743 +582,6 @@ var _ = ginkgo.Describe("[Test]", func() { }) }) - ginkgo.It("performs a warp transfer of the native asset", func() { - other, err := ed25519.GeneratePrivateKey() - gomega.Ω(err).Should(gomega.BeNil()) - aother := codec.MustAddressBech32(consts.HRP, auth.NewED25519Address(other.PublicKey())) - source, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - destination, err := ids.FromString(blockchainIDB) - gomega.Ω(err).Should(gomega.BeNil()) - otherFactory := auth.NewED25519Factory(other) - - var txID ids.ID - ginkgo.By("submitting an export action on source", func() { - otherBalance, err := instancesA[0].tcli.Balance(context.Background(), aother, ids.Empty) - gomega.Ω(err).Should(gomega.BeNil()) - senderBalance, err := instancesA[0].tcli.Balance(context.Background(), sender, ids.Empty) - gomega.Ω(err).Should(gomega.BeNil()) - - parser, err := instancesA[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesA[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: auth.NewED25519Address(other.PublicKey()), - Asset: ids.Empty, - Value: sendAmount, - Return: false, - Destination: destination, - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast and wait for transaction - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, fee, err := instancesA[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp export transaction{{/}}\n") - - // Check loans and balances - amount, err := instancesA[0].tcli.Loan(context.Background(), ids.Empty, destination) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(amount).Should(gomega.Equal(sendAmount)) - aotherBalance, err := instancesA[0].tcli.Balance(context.Background(), aother, ids.Empty) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(otherBalance).Should(gomega.Equal(aotherBalance)) - asenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(asenderBalance).Should(gomega.Equal(senderBalance - sendAmount - fee)) - }) - - ginkgo.By("fund other account with native", func() { - parser, err := instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.Transfer{ - To: auth.NewED25519Address(other.PublicKey()), - Asset: ids.Empty, - Value: 500_000_000, - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID := tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast transaction (wait for after all broadcast) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - - // Confirm transaction is accepted - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, _, err := instancesB[0].tcli.WaitForTransaction(ctx, txID) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found transaction %s on B{{/}}\n", txID) - }) - - ginkgo.By("submitting an import action on destination", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - nativeOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newOtherBalance).Should(gomega.Equal(uint64(0))) - nativeSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newSenderBalance).Should(gomega.Equal(uint64(0))) - - var ( - msg *warp.Message - subnetWeight, sigWeight uint64 - ) - for { - msg, subnetWeight, sigWeight, err = instancesA[0].cli.GenerateAggregateWarpSignature( - context.Background(), - txID, - ) - if sigWeight == subnetWeight && err == nil { - break - } - if err == nil { - hutils.Outf( - "{{yellow}}waiting for signature weight:{{/}} %d {{yellow}}observed:{{/}} %d\n", - subnetWeight, - sigWeight, - ) - } else { - hutils.Outf("{{red}}found error:{{/}} %v\n", err) - } - time.Sleep(1 * time.Second) - } - hutils.Outf( - "{{green}}fetched signature weight:{{/}} %d {{green}}total weight:{{/}} %d\n", - sigWeight, - subnetWeight, - ) - gomega.Ω(subnetWeight).Should(gomega.Equal(sigWeight)) - - parser, err := instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - msg, - &actions.ImportAsset{}, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, fee, err := instancesB[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp import transaction{{/}}\n") - - // Check asset info and balance - aNativeOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(nativeOtherBalance).Should(gomega.Equal(aNativeOtherBalance)) - aNewOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewOtherBalance).Should(gomega.Equal(sendAmount)) - aNativeSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeSenderBalance).Should(gomega.Equal(nativeSenderBalance - fee)) - aNewSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewSenderBalance).Should(gomega.Equal(uint64(0))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instancesB[0].tcli.Asset(context.Background(), newAsset, false) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(exists).Should(gomega.BeTrue()) - gomega.Ω(string(symbol)).Should(gomega.Equal(consts.Symbol)) - gomega.Ω(decimals).Should(gomega.Equal(uint8(consts.Decimals))) - gomega.Ω(metadata).Should(gomega.Equal(actions.ImportedAssetMetadata(ids.Empty, bIDA))) - gomega.Ω(supply).Should(gomega.Equal(sendAmount)) - gomega.Ω(owner).Should(gomega.Equal(codec.MustAddressBech32(consts.HRP, codec.EmptyAddress))) - gomega.Ω(warp).Should(gomega.BeTrue()) - }) - - ginkgo.By("submitting an invalid export action to new destination", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - parser, err := instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: rsender, - Asset: newAsset, - Value: 100, - Return: false, - Destination: ids.GenerateTestID(), - }, - otherFactory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast and wait for transaction - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, _, err := instancesB[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeFalse()) - - // Confirm balances are unchanged - newOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newOtherBalance).Should(gomega.Equal(sendAmount)) - }) - - ginkgo.By("submitting first (2000) return export action on destination", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - parser, err := instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: rsender, - Asset: newAsset, - Value: 2000, - Return: true, - Destination: source, - Reward: 100, - }, - otherFactory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast and wait for transaction - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, _, err := instancesB[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp export transaction{{/}}\n") - - // Check balances and asset info - amount, err := instancesB[0].tcli.Loan(context.Background(), newAsset, source) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(amount).Should(gomega.Equal(uint64(0))) - otherBalance, err := instancesB[0].tcli.Balance(context.Background(), aother, newAsset) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(otherBalance).Should(gomega.Equal(uint64(2900))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instancesB[0].tcli.Asset(context.Background(), newAsset, false) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(exists).Should(gomega.BeTrue()) - gomega.Ω(string(symbol)).Should(gomega.Equal(consts.Symbol)) - gomega.Ω(decimals).Should(gomega.Equal(uint8(consts.Decimals))) - gomega.Ω(metadata).Should(gomega.Equal(actions.ImportedAssetMetadata(ids.Empty, bIDA))) - gomega.Ω(supply).Should(gomega.Equal(uint64(2900))) - gomega.Ω(owner).Should(gomega.Equal(codec.MustAddressBech32(consts.HRP, codec.EmptyAddress))) - gomega.Ω(warp).Should(gomega.BeTrue()) - }) - - ginkgo.By("submitting first import action on source", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - nativeOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newOtherBalance).Should(gomega.Equal(uint64(0))) - nativeSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newSenderBalance).Should(gomega.Equal(uint64(0))) - - var ( - msg *warp.Message - subnetWeight, sigWeight uint64 - ) - for { - msg, subnetWeight, sigWeight, err = instancesB[0].cli.GenerateAggregateWarpSignature( - context.Background(), - txID, - ) - if sigWeight == subnetWeight && err == nil { - break - } - if err == nil { - hutils.Outf( - "{{yellow}}waiting for signature weight:{{/}} %d {{yellow}}observed:{{/}} %d\n", - subnetWeight, - sigWeight, - ) - } else { - hutils.Outf("{{red}}found error:{{/}} %v\n", err) - } - time.Sleep(1 * time.Second) - } - hutils.Outf( - "{{green}}fetched signature weight:{{/}} %d {{green}}total weight:{{/}} %d\n", - sigWeight, - subnetWeight, - ) - gomega.Ω(subnetWeight).Should(gomega.Equal(sigWeight)) - - parser, err := instancesA[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesA[0].cli.GenerateTransaction( - context.Background(), - parser, - msg, - &actions.ImportAsset{}, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, fee, err := instancesA[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp import transaction{{/}}\n") - - // Check balances and loan - aNativeOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(nativeOtherBalance).Should(gomega.Equal(aNativeOtherBalance)) - aNewOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewOtherBalance).Should(gomega.Equal(uint64(0))) - aNativeSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeSenderBalance). - Should(gomega.Equal(nativeSenderBalance - fee + 2000 + 100)) - aNewSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewSenderBalance).Should(gomega.Equal(uint64(0))) - amount, err := instancesA[0].tcli.Loan(context.Background(), ids.Empty, destination) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(amount).Should(gomega.Equal(uint64(2900))) - }) - - ginkgo.By("submitting second (2900) return export action on destination", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - parser, err := instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: auth.NewED25519Address(other.PublicKey()), - Asset: newAsset, - Value: 2900, - Return: true, - Destination: source, - }, - otherFactory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast and wait for transaction - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, _, err := instancesB[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp export transaction{{/}}\n") - - // Check balances and asset info - amount, err := instancesB[0].tcli.Loan(context.Background(), newAsset, source) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(amount).Should(gomega.Equal(uint64(0))) - otherBalance, err := instancesB[0].tcli.Balance(context.Background(), aother, newAsset) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(otherBalance).Should(gomega.Equal(uint64(0))) - exists, _, _, _, _, _, _, err := instancesB[0].tcli.Asset(context.Background(), newAsset, false) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(exists).Should(gomega.BeFalse()) - }) - - ginkgo.By("submitting second import action on source", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - nativeOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newOtherBalance).Should(gomega.Equal(uint64(0))) - nativeSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newSenderBalance).Should(gomega.Equal(uint64(0))) - - var ( - msg *warp.Message - subnetWeight, sigWeight uint64 - ) - for { - msg, subnetWeight, sigWeight, err = instancesB[0].cli.GenerateAggregateWarpSignature( - context.Background(), - txID, - ) - if sigWeight == subnetWeight && err == nil { - break - } - if err == nil { - hutils.Outf( - "{{yellow}}waiting for signature weight:{{/}} %d {{yellow}}observed:{{/}} %d\n", - subnetWeight, - sigWeight, - ) - } else { - hutils.Outf("{{red}}found error:{{/}} %v\n", err) - } - time.Sleep(1 * time.Second) - } - hutils.Outf( - "{{green}}fetched signature weight:{{/}} %d {{green}}total weight:{{/}} %d\n", - sigWeight, - subnetWeight, - ) - gomega.Ω(subnetWeight).Should(gomega.Equal(sigWeight)) - - parser, err := instancesA[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesA[0].cli.GenerateTransaction( - context.Background(), - parser, - msg, - &actions.ImportAsset{}, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, fee, err := instancesA[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp import transaction{{/}}\n") - - // Check balances and loan - aNativeOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeOtherBalance).Should(gomega.Equal(nativeOtherBalance + 2900)) - aNewOtherBalance, err := instancesA[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewOtherBalance).Should(gomega.Equal(uint64(0))) - aNativeSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeSenderBalance).Should(gomega.Equal(nativeSenderBalance - fee)) - aNewSenderBalance, err := instancesA[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewSenderBalance).Should(gomega.Equal(uint64(0))) - amount, err := instancesA[0].tcli.Loan(context.Background(), ids.Empty, destination) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(amount).Should(gomega.Equal(uint64(0))) - }) - - ginkgo.By("swaping into destination", func() { - bIDA, err := ids.FromString(blockchainIDA) - gomega.Ω(err).Should(gomega.BeNil()) - newAsset := actions.ImportedAssetID(ids.Empty, bIDA) - parser, err := instancesA[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instancesA[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: auth.NewED25519Address(other.PublicKey()), - Asset: ids.Empty, // becomes newAsset - Value: 2000, - Return: false, - SwapIn: 100, - AssetOut: ids.Empty, - SwapOut: 200, - SwapExpiry: time.Now().UnixMilli() + 100_000, - Destination: destination, - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - - // Broadcast and wait for transaction - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel := context.WithTimeout(context.Background(), requestTimeout) - success, _, err := instancesA[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp export transaction{{/}}\n") - - // Record balances on destination - nativeOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newOtherBalance).Should(gomega.Equal(uint64(0))) - nativeSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - newSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(newSenderBalance).Should(gomega.Equal(uint64(0))) - - var ( - msg *warp.Message - subnetWeight, sigWeight uint64 - ) - for { - msg, subnetWeight, sigWeight, err = instancesA[0].cli.GenerateAggregateWarpSignature( - context.Background(), - txID, - ) - if sigWeight == subnetWeight && err == nil { - break - } - if err == nil { - hutils.Outf( - "{{yellow}}waiting for signature weight:{{/}} %d {{yellow}}observed:{{/}} %d\n", - subnetWeight, - sigWeight, - ) - } else { - hutils.Outf("{{red}}found error:{{/}} %v\n", err) - } - time.Sleep(1 * time.Second) - } - hutils.Outf( - "{{green}}fetched signature weight:{{/}} %d {{green}}total weight:{{/}} %d\n", - sigWeight, - subnetWeight, - ) - gomega.Ω(subnetWeight).Should(gomega.Equal(sigWeight)) - - parser, err = instancesB[0].tcli.Parser(context.TODO()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err = instancesB[0].cli.GenerateTransaction( - context.Background(), - parser, - msg, - &actions.ImportAsset{ - Fill: true, - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - txID = tx.ID() - hutils.Outf("{{yellow}}generated transaction:{{/}} %s\n", txID) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - hutils.Outf("{{yellow}}submitted transaction{{/}}\n") - ctx, cancel = context.WithTimeout(context.Background(), requestTimeout) - success, fee, err := instancesB[0].tcli.WaitForTransaction(ctx, tx.ID()) - cancel() - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(success).Should(gomega.BeTrue()) - hutils.Outf("{{yellow}}found warp import transaction{{/}}\n") - - // Check balances following swap - aNativeOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeOtherBalance).Should(gomega.Equal(nativeOtherBalance + 200)) - aNewOtherBalance, err := instancesB[0].tcli.Balance( - context.Background(), - aother, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewOtherBalance).Should(gomega.Equal(uint64(1900))) - aNativeSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - ids.Empty, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNativeSenderBalance). - Should(gomega.Equal(nativeSenderBalance - fee - 200)) - aNewSenderBalance, err := instancesB[0].tcli.Balance( - context.Background(), - sender, - newAsset, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(aNewSenderBalance).Should(gomega.Equal(uint64(100))) - }) - }) - // TODO: add custom asset test // TODO: test with only part of sig weight // TODO: attempt to mint a warp asset diff --git a/examples/tokenvm/tests/integration/integration_test.go b/examples/tokenvm/tests/integration/integration_test.go index 1a2dfebb49..5a02a9f8ee 100644 --- a/examples/tokenvm/tests/integration/integration_test.go +++ b/examples/tokenvm/tests/integration/integration_test.go @@ -27,7 +27,6 @@ import ( "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/fatih/color" ginkgo "github.com/onsi/ginkgo/v2" "github.com/onsi/gomega" @@ -210,7 +209,6 @@ var _ = ginkgo.BeforeSuite(func() { ChainDataDir: dname, Metrics: metrics.NewOptionalGatherer(), PublicKey: bls.PublicFromSecretKey(sk), - WarpSigner: warp.NewSigner(sk, networkID, chainID), ValidatorState: &validators.TestState{}, } @@ -270,7 +268,7 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(balance).Should(gomega.Equal(alloc.Balance)) csupply += alloc.Balance } - exists, symbol, decimals, metadata, supply, owner, warp, err := cli.Asset(context.Background(), ids.Empty, false) + exists, symbol, decimals, metadata, supply, owner, err := cli.Asset(context.Background(), ids.Empty, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(string(symbol)).Should(gomega.Equal(tconsts.Symbol)) @@ -278,7 +276,6 @@ var _ = ginkgo.BeforeSuite(func() { gomega.Ω(string(metadata)).Should(gomega.Equal(tconsts.Name)) gomega.Ω(supply).Should(gomega.Equal(csupply)) gomega.Ω(owner).Should(gomega.Equal(codec.MustAddressBech32(tconsts.HRP, codec.EmptyAddress))) - gomega.Ω(warp).Should(gomega.BeFalse()) } blocks = []snowman.Block{} @@ -934,7 +931,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(balance).Should(gomega.Equal(uint64(0))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -942,7 +939,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(0))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("mint a new asset", func() { @@ -973,7 +969,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(balance).Should(gomega.Equal(uint64(0))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -981,7 +977,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(15))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("mint asset from wrong owner", func() { @@ -1010,7 +1005,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(string(result.Output)). Should(gomega.ContainSubstring("wrong owner")) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -1018,7 +1013,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(15))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("burn new asset", func() { @@ -1048,7 +1042,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(balance).Should(gomega.Equal(uint64(0))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -1056,7 +1050,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(10))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("burn missing asset", func() { @@ -1082,7 +1075,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(string(result.Output)). Should(gomega.ContainSubstring("invalid balance")) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -1090,7 +1083,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(10))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("rejects empty mint", func() { @@ -1156,7 +1148,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(balance).Should(gomega.Equal(uint64(0))) - exists, symbol, decimals, metadata, supply, owner, warp, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) + exists, symbol, decimals, metadata, supply, owner, err := instances[0].tcli.Asset(context.TODO(), asset1ID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeTrue()) gomega.Ω(symbol).Should(gomega.Equal(asset1Symbol)) @@ -1164,7 +1156,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(metadata).Should(gomega.Equal(asset1)) gomega.Ω(supply).Should(gomega.Equal(uint64(10))) gomega.Ω(owner).Should(gomega.Equal(sender)) - gomega.Ω(warp).Should(gomega.BeFalse()) }) ginkgo.It("rejects mint of native token", func() { @@ -1695,260 +1686,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(orders).Should(gomega.HaveLen(0)) }) - - ginkgo.It("import warp message with nil when expected", func() { - tx := chain.NewTx( - &chain.Base{ - ChainID: instances[0].chainID, - Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), - MaxFee: 1000, - }, - nil, - &actions.ImportAsset{}, - ) - // Must do manual construction to avoid `tx.Sign` error (would fail with - // empty warp) - msg, err := tx.Digest() - gomega.Ω(err).To(gomega.BeNil()) - auth, err := factory.Sign(msg) - gomega.Ω(err).To(gomega.BeNil()) - tx.Auth = auth - p := codec.NewWriter(0, consts.MaxInt) // test codec growth - gomega.Ω(tx.Marshal(p)).To(gomega.BeNil()) - gomega.Ω(p.Err()).To(gomega.BeNil()) - _, err = instances[0].cli.SubmitTx( - context.Background(), - p.Bytes(), - ) - gomega.Ω(err.Error()).Should(gomega.ContainSubstring("expected warp message")) - }) - - ginkgo.It("import warp message empty", func() { - wm, err := warp.NewMessage(&warp.UnsignedMessage{}, &warp.BitSetSignature{}) - gomega.Ω(err).Should(gomega.BeNil()) - tx := chain.NewTx( - &chain.Base{ - ChainID: instances[0].chainID, - Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), - MaxFee: 1000, - }, - wm, - &actions.ImportAsset{}, - ) - // Must do manual construction to avoid `tx.Sign` error (would fail with - // empty warp) - msg, err := tx.Digest() - gomega.Ω(err).To(gomega.BeNil()) - auth, err := factory.Sign(msg) - gomega.Ω(err).To(gomega.BeNil()) - tx.Auth = auth - p := codec.NewWriter(0, consts.MaxInt) // test codec growth - gomega.Ω(tx.Marshal(p)).To(gomega.BeNil()) - gomega.Ω(p.Err()).To(gomega.BeNil()) - _, err = instances[0].cli.SubmitTx( - context.Background(), - p.Bytes(), - ) - gomega.Ω(err.Error()).Should(gomega.ContainSubstring("empty warp payload")) - }) - - ginkgo.It("import with wrong payload", func() { - uwm, err := warp.NewUnsignedMessage(networkID, ids.Empty, []byte("hello")) - gomega.Ω(err).Should(gomega.BeNil()) - wm, err := warp.NewMessage(uwm, &warp.BitSetSignature{}) - gomega.Ω(err).Should(gomega.BeNil()) - tx := chain.NewTx( - &chain.Base{ - ChainID: instances[0].chainID, - Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), - MaxFee: 1000, - }, - wm, - &actions.ImportAsset{}, - ) - // Must do manual construction to avoid `tx.Sign` error (would fail with - // invalid object) - msg, err := tx.Digest() - gomega.Ω(err).To(gomega.BeNil()) - auth, err := factory.Sign(msg) - gomega.Ω(err).To(gomega.BeNil()) - tx.Auth = auth - p := codec.NewWriter(0, consts.MaxInt) // test codec growth - gomega.Ω(tx.Marshal(p)).To(gomega.BeNil()) - gomega.Ω(p.Err()).To(gomega.BeNil()) - _, err = instances[0].cli.SubmitTx( - context.Background(), - p.Bytes(), - ) - gomega.Ω(err.Error()).Should(gomega.ContainSubstring("insufficient length for input")) - }) - - ginkgo.It("import with invalid payload", func() { - wt := &actions.WarpTransfer{} - wtb, err := wt.Marshal() - gomega.Ω(err).Should(gomega.BeNil()) - uwm, err := warp.NewUnsignedMessage(networkID, ids.Empty, wtb) - gomega.Ω(err).Should(gomega.BeNil()) - wm, err := warp.NewMessage(uwm, &warp.BitSetSignature{}) - gomega.Ω(err).Should(gomega.BeNil()) - tx := chain.NewTx( - &chain.Base{ - ChainID: instances[0].chainID, - Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), - MaxFee: 1000, - }, - wm, - &actions.ImportAsset{}, - ) - // Must do manual construction to avoid `tx.Sign` error (would fail with - // invalid object) - msg, err := tx.Digest() - gomega.Ω(err).To(gomega.BeNil()) - auth, err := factory.Sign(msg) - gomega.Ω(err).To(gomega.BeNil()) - tx.Auth = auth - p := codec.NewWriter(0, consts.MaxInt) // test codec growth - gomega.Ω(tx.Marshal(p)).To(gomega.BeNil()) - gomega.Ω(p.Err()).To(gomega.BeNil()) - _, err = instances[0].cli.SubmitTx( - context.Background(), - p.Bytes(), - ) - gomega.Ω(err.Error()).Should(gomega.ContainSubstring("field is not populated")) - }) - - ginkgo.It("import with wrong destination", func() { - wt := &actions.WarpTransfer{ - To: rsender, - Symbol: []byte("s"), - Decimals: 2, - Asset: ids.GenerateTestID(), - Value: 100, - Return: false, - Reward: 100, - TxID: ids.GenerateTestID(), - DestinationChainID: ids.GenerateTestID(), - } - wtb, err := wt.Marshal() - gomega.Ω(err).Should(gomega.BeNil()) - uwm, err := warp.NewUnsignedMessage(networkID, ids.Empty, wtb) - gomega.Ω(err).Should(gomega.BeNil()) - wm, err := warp.NewMessage(uwm, &warp.BitSetSignature{}) - gomega.Ω(err).Should(gomega.BeNil()) - parser, err := instances[0].tcli.Parser(context.Background()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, _, _, err := instances[0].cli.GenerateTransaction( - context.Background(), - parser, - wm, - &actions.ImportAsset{}, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - - // Build block with no context (should fail) - gomega.Ω(instances[0].vm.Builder().Force(context.TODO())).To(gomega.BeNil()) - <-instances[0].toEngine - blk, err := instances[0].vm.BuildBlock(context.TODO()) - gomega.Ω(err).To(gomega.Not(gomega.BeNil())) - gomega.Ω(blk).To(gomega.BeNil()) - - // Wait for mempool to be size 1 (txs are restored async) - for { - if instances[0].vm.Mempool().Len(context.Background()) > 0 { - break - } - log.Info("waiting for txs to be restored") - time.Sleep(100 * time.Millisecond) - } - - // Build block with context - accept := expectBlkWithContext(instances[0]) - results := accept(false) - gomega.Ω(results).Should(gomega.HaveLen(1)) - result := results[0] - gomega.Ω(result.Success).Should(gomega.BeFalse()) - gomega.Ω(string(result.Output)).Should(gomega.ContainSubstring("warp verification failed")) - }) - - ginkgo.It("export native asset", func() { - dest := ids.GenerateTestID() - loan, err := instances[0].tcli.Loan(context.TODO(), ids.Empty, dest) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(loan).Should(gomega.Equal(uint64(0))) - - parser, err := instances[0].tcli.Parser(context.Background()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, tx, _, err := instances[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: rsender, - Asset: ids.Empty, - Value: 100, - Return: false, - Reward: 10, - Destination: dest, - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - accept := expectBlk(instances[0]) - results := accept(false) - gomega.Ω(results).Should(gomega.HaveLen(1)) - result := results[0] - gomega.Ω(result.Success).Should(gomega.BeTrue()) - wt := &actions.WarpTransfer{ - To: rsender, - Symbol: []byte(tconsts.Symbol), - Decimals: tconsts.Decimals, - Asset: ids.Empty, - Value: 100, - Return: false, - Reward: 10, - TxID: tx.ID(), - DestinationChainID: dest, - } - wtb, err := wt.Marshal() - gomega.Ω(err).Should(gomega.BeNil()) - wm, err := warp.NewUnsignedMessage(networkID, instances[0].chainID, wtb) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(result.WarpMessage).Should(gomega.Equal(wm)) - - loan, err = instances[0].tcli.Loan(context.TODO(), ids.Empty, dest) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(loan).Should(gomega.Equal(uint64(110))) - }) - - ginkgo.It("export native asset (invalid return)", func() { - parser, err := instances[0].tcli.Parser(context.Background()) - gomega.Ω(err).Should(gomega.BeNil()) - submit, _, _, err := instances[0].cli.GenerateTransaction( - context.Background(), - parser, - nil, - &actions.ExportAsset{ - To: rsender, - Asset: ids.Empty, - Value: 100, - Return: true, - Reward: 10, - Destination: ids.GenerateTestID(), - }, - factory, - ) - gomega.Ω(err).Should(gomega.BeNil()) - gomega.Ω(submit(context.Background())).Should(gomega.BeNil()) - accept := expectBlk(instances[0]) - results := accept(false) - gomega.Ω(results).Should(gomega.HaveLen(1)) - result := results[0] - gomega.Ω(result.Success).Should(gomega.BeFalse()) - gomega.Ω(string(result.Output)).Should(gomega.ContainSubstring("not warp asset")) - }) }) func expectBlk(i instance) func(bool) []*chain.Result { diff --git a/rpc/dependencies.go b/rpc/dependencies.go index d9cde2e5bf..67f9e0d567 100644 --- a/rpc/dependencies.go +++ b/rpc/dependencies.go @@ -10,7 +10,6 @@ import ( "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/trace" "github.com/ava-labs/avalanchego/utils/logging" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/fees" @@ -30,8 +29,6 @@ type VM interface { ) (errs []error) LastAcceptedBlock() *chain.StatelessBlock UnitPrices(context.Context) (fees.Dimensions, error) - GetOutgoingWarpMessage(ids.ID) (*warp.UnsignedMessage, error) - GetWarpSignatures(ids.ID) ([]*chain.WarpSignature, error) CurrentValidators( context.Context, ) (map[ids.NodeID]*validators.GetValidatorOutput, map[string]struct{}) diff --git a/rpc/jsonrpc_client.go b/rpc/jsonrpc_client.go index fa14d6f91d..54c3d6f285 100644 --- a/rpc/jsonrpc_client.go +++ b/rpc/jsonrpc_client.go @@ -14,7 +14,6 @@ import ( "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "golang.org/x/exp/maps" "github.com/ava-labs/hypersdk/chain" @@ -123,41 +122,6 @@ func (cli *JSONRPCClient) SubmitTx(ctx context.Context, d []byte) (ids.ID, error return resp.TxID, err } -func (cli *JSONRPCClient) GetWarpSignatures( - ctx context.Context, - txID ids.ID, -) (*warp.UnsignedMessage, map[ids.NodeID]*validators.GetValidatorOutput, []*chain.WarpSignature, error) { - resp := new(GetWarpSignaturesReply) - if err := cli.requester.SendRequest( - ctx, - "getWarpSignatures", - &GetWarpSignaturesArgs{TxID: txID}, - resp, - ); err != nil { - return nil, nil, nil, err - } - // Ensure message is initialized - if err := resp.Message.Initialize(); err != nil { - return nil, nil, nil, err - } - m := map[ids.NodeID]*validators.GetValidatorOutput{} - for _, vdr := range resp.Validators { - vout := &validators.GetValidatorOutput{ - NodeID: vdr.NodeID, - Weight: vdr.Weight, - } - if len(vdr.PublicKey) > 0 { - pk, err := bls.PublicKeyFromBytes(vdr.PublicKey) - if err != nil { - return nil, nil, nil, err - } - vout.PublicKey = pk - } - m[vdr.NodeID] = vout - } - return resp.Message, m, resp.Signatures, nil -} - type Modifier interface { Base(*chain.Base) } @@ -165,7 +129,6 @@ type Modifier interface { func (cli *JSONRPCClient) GenerateTransaction( ctx context.Context, parser chain.Parser, - wm *warp.Message, action chain.Action, authFactory chain.AuthFactory, modifiers ...Modifier, @@ -176,7 +139,7 @@ func (cli *JSONRPCClient) GenerateTransaction( return nil, nil, 0, err } - maxUnits, err := chain.EstimateMaxUnits(parser.Rules(time.Now().UnixMilli()), action, authFactory, wm) + maxUnits, err := chain.EstimateMaxUnits(parser.Rules(time.Now().UnixMilli()), action, authFactory) if err != nil { return nil, nil, 0, err } @@ -184,7 +147,7 @@ func (cli *JSONRPCClient) GenerateTransaction( if err != nil { return nil, nil, 0, err } - f, tx, err := cli.GenerateTransactionManual(parser, wm, action, authFactory, maxFee, modifiers...) + f, tx, err := cli.GenerateTransactionManual(parser, action, authFactory, maxFee, modifiers...) if err != nil { return nil, nil, 0, err } @@ -193,7 +156,6 @@ func (cli *JSONRPCClient) GenerateTransaction( func (cli *JSONRPCClient) GenerateTransactionManual( parser chain.Parser, - wm *warp.Message, action chain.Action, authFactory chain.AuthFactory, maxFee uint64, @@ -213,16 +175,9 @@ func (cli *JSONRPCClient) GenerateTransactionManual( m.Base(base) } - // Ensure warp message is intialized before we marshal it - if wm != nil { - if err := wm.Initialize(); err != nil { - return nil, nil, err - } - } - // Build transaction actionRegistry, authRegistry := parser.Registry() - tx := chain.NewTx(base, wm, action) + tx := chain.NewTx(base, action) tx, err := tx.Sign(authFactory, actionRegistry, authRegistry) if err != nil { return nil, nil, fmt.Errorf("%w: failed to sign transaction", err) @@ -248,101 +203,3 @@ func Wait(ctx context.Context, check func(ctx context.Context) (bool, error)) er } return ctx.Err() } - -// getCanonicalValidatorSet returns the validator set of [subnetID] in a canonical ordering. -// Also returns the total weight on [subnetID]. -func getCanonicalValidatorSet( - _ context.Context, - vdrSet map[ids.NodeID]*validators.GetValidatorOutput, -) ([]*warp.Validator, uint64, error) { - var ( - vdrs = make(map[string]*warp.Validator, len(vdrSet)) - totalWeight uint64 - err error - ) - for _, vdr := range vdrSet { - totalWeight, err = math.Add64(totalWeight, vdr.Weight) - if err != nil { - return nil, 0, fmt.Errorf("%w: %v", warp.ErrWeightOverflow, err) //nolint:errorlint - } - - if vdr.PublicKey == nil { - fmt.Println("skipping validator because of empty public key", vdr.NodeID) - continue - } - - pkBytes := bls.PublicKeyToBytes(vdr.PublicKey) - uniqueVdr, ok := vdrs[string(pkBytes)] - if !ok { - uniqueVdr = &warp.Validator{ - PublicKey: vdr.PublicKey, - PublicKeyBytes: pkBytes, - } - vdrs[string(pkBytes)] = uniqueVdr - } - - uniqueVdr.Weight += vdr.Weight // Impossible to overflow here - uniqueVdr.NodeIDs = append(uniqueVdr.NodeIDs, vdr.NodeID) - } - - // Sort validators by public key - vdrList := maps.Values(vdrs) - avautils.Sort(vdrList) - return vdrList, totalWeight, nil -} - -func (cli *JSONRPCClient) GenerateAggregateWarpSignature( - ctx context.Context, - txID ids.ID, -) (*warp.Message, uint64, uint64, error) { - unsignedMessage, validators, signatures, err := cli.GetWarpSignatures(ctx, txID) - if err != nil { - return nil, 0, 0, fmt.Errorf("%w: failed to fetch warp signatures", err) - } - - // Get canonical validator ordering to generate signature bit set - canonicalValidators, weight, err := getCanonicalValidatorSet(ctx, validators) - if err != nil { - return nil, 0, 0, fmt.Errorf("%w: failed to get canonical validator set", err) - } - - // Generate map of bls.PublicKey => Signature - signatureMap := map[ids.ID][]byte{} - for _, signature := range signatures { - // Convert to hash for easy comparison (could just as easily store the raw - // public key but that would involve a number of memory copies) - signatureMap[utils.ToID(signature.PublicKey)] = signature.Signature - } - - // Generate signature - signers := set.NewBits() - var signatureWeight uint64 - orderedSignatures := []*bls.Signature{} - for i, vdr := range canonicalValidators { - sig, ok := signatureMap[utils.ToID(vdr.PublicKeyBytes)] - if !ok { - continue - } - blsSig, err := bls.SignatureFromBytes(sig) - if err != nil { - return nil, 0, 0, err - } - signers.Add(i) - signatureWeight += vdr.Weight - orderedSignatures = append(orderedSignatures, blsSig) - } - aggSignature, err := bls.AggregateSignatures(orderedSignatures) - if err != nil { - return nil, 0, 0, fmt.Errorf("%w: failed to aggregate signatures", err) - } - aggSignatureBytes := bls.SignatureToBytes(aggSignature) - signature := &warp.BitSetSignature{ - Signers: signers.Bytes(), - } - copy(signature.Signature[:], aggSignatureBytes) - message, err := warp.NewMessage(unsignedMessage, signature) - if err != nil { - return nil, 0, 0, fmt.Errorf("%w: failed to generate warp message", err) - } - return message, weight, signatureWeight, nil -} diff --git a/rpc/jsonrpc_server.go b/rpc/jsonrpc_server.go index f18211abbc..9860219fff 100644 --- a/rpc/jsonrpc_server.go +++ b/rpc/jsonrpc_server.go @@ -11,7 +11,6 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils/crypto/bls" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "go.uber.org/zap" "github.com/ava-labs/hypersdk/chain" @@ -122,82 +121,3 @@ func (j *JSONRPCServer) UnitPrices( reply.UnitPrices = unitPrices return nil } - -type GetWarpSignaturesArgs struct { - TxID ids.ID `json:"txID"` -} - -type WarpValidator struct { - NodeID ids.NodeID `json:"nodeID"` - PublicKey []byte `json:"publicKey"` - Weight uint64 `json:"weight"` -} - -type GetWarpSignaturesReply struct { - Validators []*WarpValidator `json:"validators"` - Message *warp.UnsignedMessage `json:"message"` - Signatures []*chain.WarpSignature `json:"signatures"` -} - -func (j *JSONRPCServer) GetWarpSignatures( - req *http.Request, - args *GetWarpSignaturesArgs, - reply *GetWarpSignaturesReply, -) error { - _, span := j.vm.Tracer().Start(req.Context(), "JSONRPCServer.GetWarpSignatures") - defer span.End() - - message, err := j.vm.GetOutgoingWarpMessage(args.TxID) - if err != nil { - return err - } - if message == nil { - return ErrMessageMissing - } - - signatures, err := j.vm.GetWarpSignatures(args.TxID) - if err != nil { - return err - } - - // Ensure we only return valid signatures - validSignatures := []*chain.WarpSignature{} - warpValidators := []*WarpValidator{} - validators, publicKeys := j.vm.CurrentValidators(req.Context()) - for _, sig := range signatures { - if _, ok := publicKeys[string(sig.PublicKey)]; !ok { - continue - } - validSignatures = append(validSignatures, sig) - } - for _, vdr := range validators { - wv := &WarpValidator{ - NodeID: vdr.NodeID, - Weight: vdr.Weight, - } - if vdr.PublicKey != nil { - wv.PublicKey = bls.PublicKeyToBytes(vdr.PublicKey) - } - warpValidators = append(warpValidators, wv) - } - - // Optimistically request that we gather signatures if we don't have all of them - if len(validSignatures) < len(publicKeys) { - j.vm.Logger().Info( - "fetching missing signatures", - zap.Stringer("txID", args.TxID), - zap.Int( - "previously collected", - len(signatures), - ), - zap.Int("valid", len(validSignatures)), - zap.Int("current public key count", len(publicKeys)), - ) - j.vm.GatherSignatures(context.TODO(), args.TxID, message.Bytes()) - } - - reply.Message = message - reply.Validators = warpValidators - reply.Signatures = validSignatures - return nil -} diff --git a/vm/network_warp.go b/vm/network_warp.go deleted file mode 100644 index 45b5169507..0000000000 --- a/vm/network_warp.go +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package vm - -import ( - "context" - "time" - - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/version" -) - -type WarpHandler struct { - vm *VM -} - -func NewWarpHandler(vm *VM) *WarpHandler { - return &WarpHandler{vm} -} - -func (*WarpHandler) Connected(context.Context, ids.NodeID, *version.Application) error { - return nil -} - -func (*WarpHandler) Disconnected(context.Context, ids.NodeID) error { - return nil -} - -func (*WarpHandler) AppGossip(context.Context, ids.NodeID, []byte) error { - return nil -} - -func (w *WarpHandler) AppRequest( - ctx context.Context, - nodeID ids.NodeID, - requestID uint32, - _ time.Time, - request []byte, -) error { - return w.vm.warpManager.AppRequest(ctx, nodeID, requestID, request) -} - -func (w *WarpHandler) AppRequestFailed( - _ context.Context, - _ ids.NodeID, - requestID uint32, -) error { - return w.vm.warpManager.HandleRequestFailed(requestID) -} - -func (w *WarpHandler) AppResponse( - _ context.Context, - _ ids.NodeID, - requestID uint32, - response []byte, -) error { - return w.vm.warpManager.HandleResponse(requestID, response) -} - -func (*WarpHandler) CrossChainAppRequest(context.Context, ids.ID, uint32, time.Time, []byte) error { - return nil -} - -func (*WarpHandler) CrossChainAppRequestFailed(context.Context, ids.ID, uint32) error { - return nil -} - -func (*WarpHandler) CrossChainAppResponse(context.Context, ids.ID, uint32, []byte) error { - return nil -} diff --git a/vm/resolutions.go b/vm/resolutions.go index ccd243964c..b566f0a91e 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -175,38 +175,6 @@ func (vm *VM) processAcceptedBlock(b *chain.StatelessBlock) { vm.Fatal("accepted processing failed", zap.Error(err)) } - // 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 - } - start := time.Now() - signature, err := vm.snowCtx.WarpSigner.Sign(result.WarpMessage) - if err != nil { - vm.Fatal("unable to sign warp message", zap.Error(err)) - } - if err := vm.StoreWarpSignature(tx.ID(), vm.snowCtx.PublicKey, signature); err != nil { - vm.Fatal("unable to store warp signature", zap.Error(err)) - } - vm.snowCtx.Log.Info( - "signed and stored warp message signature", - zap.Stringer("txID", tx.ID()), - zap.Duration("t", time.Since(start)), - ) - - // Kickoff job to fetch signatures from other validators in the - // background - // - // We pass bytes here so that signatures returned from validators can be - // verified before they are persisted. - vm.warpManager.GatherSignatures(context.TODO(), tx.ID(), result.WarpMessage.Bytes()) - } - // Update server if err := vm.webSocketServer.AcceptBlock(b); err != nil { vm.Fatal("unable to accept block in websocket server", zap.Error(err)) @@ -333,10 +301,6 @@ func (vm *VM) CurrentValidators( return vm.proposerMonitor.Validators(ctx) } -func (vm *VM) GatherSignatures(ctx context.Context, txID ids.ID, msg []byte) { - vm.warpManager.GatherSignatures(ctx, txID, msg) -} - func (vm *VM) NodeID() ids.NodeID { return vm.snowCtx.NodeID } diff --git a/vm/storage.go b/vm/storage.go index f89cd1385a..93013e462f 100644 --- a/vm/storage.go +++ b/vm/storage.go @@ -17,7 +17,6 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/choices" "github.com/ava-labs/avalanchego/utils/crypto/bls" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "go.uber.org/zap" "github.com/ava-labs/hypersdk/chain" @@ -38,15 +37,11 @@ const ( blockPrefix = 0x0 // TODO: move to flat files (https://github.com/ava-labs/hypersdk/issues/553) blockIDHeightPrefix = 0x1 // ID -> Height blockHeightIDPrefix = 0x2 // Height -> ID (don't always need full block from disk) - warpSignaturePrefix = 0x3 - warpFetchPrefix = 0x4 ) var ( isSyncing = []byte("is_syncing") lastAccepted = []byte("last_accepted") - - signatureLRU = &cache.LRU[string, *chain.WarpSignature]{Size: 1024} ) func PrefixBlockKey(height uint64) []byte { @@ -220,96 +215,3 @@ func (vm *VM) PutDiskIsSyncing(v bool) error { } return vm.vmDB.Put(isSyncing, []byte{0x0}) } - -func (vm *VM) GetOutgoingWarpMessage(txID ids.ID) (*warp.UnsignedMessage, error) { - p := vm.c.StateManager().OutgoingWarpKeyPrefix(txID) - k := keys.EncodeChunks(p, chain.MaxOutgoingWarpChunks) - vs, errs := vm.ReadState(context.TODO(), [][]byte{k}) - v, err := vs[0], errs[0] - if errors.Is(err, database.ErrNotFound) { - return nil, nil - } - if err != nil { - return nil, err - } - return warp.ParseUnsignedMessage(v) -} - -func PrefixWarpSignatureKey(txID ids.ID, signer *bls.PublicKey) []byte { - k := make([]byte, 1+consts.IDLen+bls.PublicKeyLen) - k[0] = warpSignaturePrefix - copy(k[1:], txID[:]) - copy(k[1+consts.IDLen:], bls.PublicKeyToBytes(signer)) - return k -} - -func (vm *VM) StoreWarpSignature(txID ids.ID, signer *bls.PublicKey, signature []byte) error { - k := PrefixWarpSignatureKey(txID, signer) - // Cache any signature we produce for later queries from peers - if bytes.Equal(vm.pkBytes, bls.PublicKeyToBytes(signer)) { - signatureLRU.Put(string(k), chain.NewWarpSignature(vm.pkBytes, signature)) - } - return vm.vmDB.Put(k, signature) -} - -func (vm *VM) GetWarpSignature(txID ids.ID, signer *bls.PublicKey) (*chain.WarpSignature, error) { - k := PrefixWarpSignatureKey(txID, signer) - if ws, ok := signatureLRU.Get(string(k)); ok { - return ws, nil - } - v, err := vm.vmDB.Get(k) - if errors.Is(err, database.ErrNotFound) { - return nil, nil - } - if err != nil { - return nil, err - } - ws := &chain.WarpSignature{ - PublicKey: bls.PublicKeyToBytes(signer), - Signature: v, - } - return ws, nil -} - -func (vm *VM) GetWarpSignatures(txID ids.ID) ([]*chain.WarpSignature, error) { - prefix := make([]byte, 1+consts.IDLen) - prefix[0] = warpSignaturePrefix - copy(prefix[1:], txID[:]) - iter := vm.vmDB.NewIteratorWithPrefix(prefix) - defer iter.Release() - - // Collect all signatures we have for a txID - signatures := []*chain.WarpSignature{} - for iter.Next() { - k := iter.Key() - signatures = append(signatures, &chain.WarpSignature{ - PublicKey: k[len(k)-bls.PublicKeyLen:], - Signature: iter.Value(), - }) - } - return signatures, iter.Error() -} - -func PrefixWarpFetchKey(txID ids.ID) []byte { - k := make([]byte, 1+consts.IDLen) - k[0] = warpFetchPrefix - copy(k[1:], txID[:]) - return k -} - -func (vm *VM) StoreWarpFetch(txID ids.ID) error { - k := PrefixWarpFetchKey(txID) - return vm.vmDB.Put(k, binary.BigEndian.AppendUint64(nil, uint64(time.Now().UnixMilli()))) -} - -func (vm *VM) GetWarpFetch(txID ids.ID) (int64, error) { - k := PrefixWarpFetchKey(txID) - v, err := vm.vmDB.Get(k) - if errors.Is(err, database.ErrNotFound) { - return -1, nil - } - if err != nil { - return -1, err - } - return int64(binary.BigEndian.Uint64(v)), nil -} diff --git a/vm/vm.go b/vm/vm.go index c47fec0f19..1867229cab 100644 --- a/vm/vm.go +++ b/vm/vm.go @@ -112,10 +112,6 @@ type VM struct { stateSyncNetworkClient avasync.NetworkClient stateSyncNetworkServer *avasync.NetworkServer - // Warp manager fetches signatures from other validators for a given accepted - // txID - warpManager *WarpManager - // Network manager routes p2p messages to pre-registered handlers networkManager *network.Manager @@ -167,10 +163,6 @@ func (vm *VM) Initialize( vm.proposerMonitor = NewProposerMonitor(vm) vm.networkManager = network.NewManager(vm.snowCtx.Log, vm.snowCtx.NodeID, appSender) - warpHandler, warpSender := vm.networkManager.Register() - vm.warpManager = NewWarpManager(vm) - vm.networkManager.SetHandler(warpHandler, NewWarpHandler(vm)) - go vm.warpManager.Run(warpSender) vm.baseDB = baseDB // Always initialize implementation first @@ -561,7 +553,6 @@ func (vm *VM) Shutdown(ctx context.Context) error { <-vm.acceptorDone // Shutdown other async VM mechanisms - vm.warpManager.Done() vm.builder.Done() vm.gossiper.Done() vm.authVerifiers.Stop() diff --git a/vm/warp_manager.go b/vm/warp_manager.go deleted file mode 100644 index 13c851e277..0000000000 --- a/vm/warp_manager.go +++ /dev/null @@ -1,360 +0,0 @@ -// Copyright (C) 2023, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package vm - -import ( - "bytes" - "context" - "encoding/hex" - "sync" - "time" - - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/utils/crypto/bls" - "github.com/ava-labs/avalanchego/utils/set" - "go.uber.org/zap" - - "github.com/ava-labs/hypersdk/chain" - "github.com/ava-labs/hypersdk/codec" - "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/heap" - "github.com/ava-labs/hypersdk/utils" -) - -const ( - maxWarpResponse = bls.PublicKeyLen + bls.SignatureLen - minGatherInterval = 30 * 60 // 30 minutes - initialBackoff = 2 // give time for others to sign - backoffIncrease = 5 - maxRetries = 10 - maxOutstanding = 8 // TODO: make a config -) - -// WarpManager takes requests to get signatures from other nodes and then -// stores the result in our DB for future usage. -type WarpManager struct { - vm *VM - appSender common.AppSender - - l sync.Mutex - requestID uint32 - - pendingJobs *heap.Heap[*signatureJob, int64] - jobs map[uint32]*signatureJob - - done chan struct{} -} - -type signatureJob struct { - id ids.ID - nodeID ids.NodeID - publicKey []byte - txID ids.ID - retry int - msg []byte -} - -func NewWarpManager(vm *VM) *WarpManager { - return &WarpManager{ - vm: vm, - pendingJobs: heap.New[*signatureJob, int64](64, true), - jobs: map[uint32]*signatureJob{}, - done: make(chan struct{}), - } -} - -func (w *WarpManager) Run(appSender common.AppSender) { - w.appSender = appSender - - w.vm.Logger().Info("starting warp manager") - defer close(w.done) - - t := time.NewTicker(1 * time.Second) - defer t.Stop() - for { - select { - case <-t.C: - w.l.Lock() - now := time.Now().Unix() - for w.pendingJobs.Len() > 0 && len(w.jobs) < maxOutstanding { - first := w.pendingJobs.First() - if first.Val > now { - break - } - w.pendingJobs.Pop() - - // Send request - job := first.Item - if err := w.request(context.Background(), job); err != nil { - w.vm.snowCtx.Log.Error( - "unable to request signature", - zap.Stringer("nodeID", job.nodeID), - zap.Error(err), - ) - } - } - l := w.pendingJobs.Len() - w.l.Unlock() - w.vm.snowCtx.Log.Debug("checked for ready jobs", zap.Int("pending", l)) - case <-w.vm.stop: - w.vm.Logger().Info("stopping warp manager") - return - } - } -} - -// GatherSignatures makes a best effort to acquire signatures from other -// validators and store them inside the vmDB. -// -// GatherSignatures may be called when a block is accepted (optimistically) or -// may be triggered by RPC (if missing signatures are detected). To prevent RPC -// abuse, we limit how frequently we attempt to gather signatures for a given -// TxID. -func (w *WarpManager) GatherSignatures(ctx context.Context, txID ids.ID, msg []byte) { - lastFetch, err := w.vm.GetWarpFetch(txID) - if err != nil { - w.vm.snowCtx.Log.Error("unable to get last fetch", zap.Error(err)) - return - } - if time.Now().Unix()-lastFetch < minGatherInterval { - w.vm.snowCtx.Log.Error("skipping fetch too recent", zap.Stringer("txID", txID)) - return - } - if err := w.vm.StoreWarpFetch(txID); err != nil { - w.vm.snowCtx.Log.Error("unable to get last fetch", zap.Error(err)) - return - } - height, err := w.vm.snowCtx.ValidatorState.GetCurrentHeight(ctx) - if err != nil { - w.vm.snowCtx.Log.Error("unable to get current p-chain height", zap.Error(err)) - return - } - validators, err := w.vm.snowCtx.ValidatorState.GetValidatorSet( - ctx, - height, - w.vm.snowCtx.SubnetID, - ) - if err != nil { - w.vm.snowCtx.Log.Error("unable to get validator set", zap.Error(err)) - return - } - for nodeID, validator := range validators { - // Only request from validators that have registered BLS public keys and - // that we have not already gotten a signature from. - if validator.PublicKey == nil { - w.vm.snowCtx.Log.Info( - "skipping fetch for validator with no registered public key", - zap.Stringer("nodeID", nodeID), - zap.Uint64("pchain height", height), - ) - continue - } - previousSignature, err := w.vm.GetWarpSignature(txID, validator.PublicKey) - if err != nil { - w.vm.snowCtx.Log.Error("unable to fetch previous signature", zap.Error(err)) - return - } - if previousSignature != nil { - continue - } - - idb := make([]byte, consts.IDLen+consts.NodeIDLen) - copy(idb, txID[:]) - copy(idb[consts.IDLen:], nodeID.Bytes()) - id := utils.ToID(idb) - w.l.Lock() - if w.pendingJobs.Has(id) { - // We may already have enqueued a job when the block was accepted. - w.l.Unlock() - continue - } - w.pendingJobs.Push(&heap.Entry[*signatureJob, int64]{ - ID: id, - Item: &signatureJob{ - id, - nodeID, - bls.PublicKeyToBytes(validator.PublicKey), - txID, - 0, - msg, - }, - Val: time.Now().Unix() + initialBackoff, - Index: w.pendingJobs.Len(), - }) - w.l.Unlock() - w.vm.snowCtx.Log.Debug( - "enqueued fetch job", - zap.Stringer("nodeID", nodeID), - zap.Stringer("txID", txID), - ) - } -} - -// you must hold [w.l] when calling this function -func (w *WarpManager) request( - ctx context.Context, - j *signatureJob, -) error { - requestID := w.requestID - w.requestID++ - w.jobs[requestID] = j - - return w.appSender.SendAppRequest( - ctx, - set.Of(j.nodeID), - requestID, - j.txID[:], - ) -} - -func (w *WarpManager) AppRequest( - ctx context.Context, - nodeID ids.NodeID, - requestID uint32, - request []byte, -) error { - rp := codec.NewReader(request, consts.IDLen) - var txID ids.ID - rp.UnpackID(true, &txID) - if err := rp.Err(); err != nil { - w.vm.snowCtx.Log.Warn("unable to unpack request", zap.Error(err)) - return nil - } - sig, err := w.vm.GetWarpSignature(txID, w.vm.snowCtx.PublicKey) - if err != nil { - w.vm.snowCtx.Log.Warn("could not fetch warp signature", zap.Error(err)) - return nil - } - if sig == nil { - // Generate and save signature if it does not exist but is in state (may - // have been offline when message was accepted) - msg, err := w.vm.GetOutgoingWarpMessage(txID) - if msg == nil || err != nil { - w.vm.snowCtx.Log.Warn("could not get outgoing warp message", zap.Error(err)) - return nil - } - rSig, err := w.vm.snowCtx.WarpSigner.Sign(msg) - if err != nil { - w.vm.snowCtx.Log.Warn("could not sign outgoing warp message", zap.Error(err)) - return nil - } - if err := w.vm.StoreWarpSignature(txID, w.vm.snowCtx.PublicKey, rSig); err != nil { - w.vm.snowCtx.Log.Warn("could not store warp signature", zap.Error(err)) - return nil - } - sig = &chain.WarpSignature{ - PublicKey: w.vm.pkBytes, - Signature: rSig, - } - } - size := len(sig.PublicKey) + len(sig.Signature) - wp := codec.NewWriter(size, maxWarpResponse) - wp.PackFixedBytes(sig.PublicKey) - wp.PackFixedBytes(sig.Signature) - if err := wp.Err(); err != nil { - w.vm.snowCtx.Log.Warn("could not encode warp signature", zap.Error(err)) - return nil - } - return w.appSender.SendAppResponse(ctx, nodeID, requestID, wp.Bytes()) -} - -func (w *WarpManager) HandleResponse(requestID uint32, msg []byte) error { - w.l.Lock() - job, ok := w.jobs[requestID] - delete(w.jobs, requestID) - w.l.Unlock() - if !ok { - return nil - } - - // Parse message - r := codec.NewReader(msg, maxWarpResponse) - publicKey := make([]byte, bls.PublicKeyLen) - r.UnpackFixedBytes(bls.PublicKeyLen, &publicKey) - signature := make([]byte, bls.SignatureLen) - r.UnpackFixedBytes(bls.SignatureLen, &signature) - if err := r.Err(); err != nil { - w.vm.snowCtx.Log.Warn("could not decode warp signature", zap.Error(err)) - return nil - } - - // Check public key is expected - if !bytes.Equal(publicKey, job.publicKey) { - w.vm.snowCtx.Log.Warn( - "public key mismatch", - zap.String("found", hex.EncodeToString(publicKey)), - zap.String("expected", hex.EncodeToString(job.publicKey)), - ) - return nil - } - - // Check signature validity - pk, err := bls.PublicKeyFromBytes(publicKey) - if err != nil { - w.vm.snowCtx.Log.Warn("could not decode public key", zap.Error(err)) - return nil - } - sig, err := bls.SignatureFromBytes(signature) - if err != nil { - w.vm.snowCtx.Log.Warn("could not decode signature", zap.Error(err)) - return nil - } - if !bls.Verify(pk, sig, job.msg) { - w.vm.snowCtx.Log.Warn("could not verify signature") - return nil - } - - // Store in DB - if err := w.vm.StoreWarpSignature(job.txID, pk, signature); err != nil { - w.vm.snowCtx.Log.Warn("could not store warp signature", zap.Error(err)) - return nil - } - - w.vm.snowCtx.Log.Info( - "fetched and stored signature", - zap.Stringer("txID", job.txID), - zap.Stringer( - "nodeID", - job.nodeID, - ), - zap.String("publicKey", hex.EncodeToString(job.publicKey)), - ) - return nil -} - -func (w *WarpManager) HandleRequestFailed(requestID uint32) error { - w.l.Lock() - job, ok := w.jobs[requestID] - delete(w.jobs, requestID) - w.l.Unlock() - if !ok { - return nil - } - - // Drop if we've already retried too many times - if job.retry >= maxRetries { - w.vm.snowCtx.Log.Info( - "fetch job failed", - zap.Stringer("nodeID", job.nodeID), - zap.Stringer("txID", job.txID), - ) - return nil - } - job.retry++ - - w.l.Lock() - w.pendingJobs.Push(&heap.Entry[*signatureJob, int64]{ - ID: job.id, - Item: job, - Val: time.Now().Unix() + int64(backoffIncrease*job.retry), - Index: w.pendingJobs.Len(), - }) - w.l.Unlock() - return nil -} - -func (w *WarpManager) Done() { - <-w.done -} diff --git a/x/programs/cmd/simulator/vm/actions/program_create.go b/x/programs/cmd/simulator/vm/actions/program_create.go index 512454108b..6433b144d3 100644 --- a/x/programs/cmd/simulator/vm/actions/program_create.go +++ b/x/programs/cmd/simulator/vm/actions/program_create.go @@ -7,7 +7,6 @@ import ( "context" "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -36,10 +35,6 @@ func (*ProgramCreate) StateKeysMaxChunks() []uint16 { return []uint16{storage.ProgramChunks} } -func (*ProgramCreate) OutputsWarpMessage() bool { - return false -} - func (t *ProgramCreate) Execute( ctx context.Context, _ chain.Rules, @@ -48,16 +43,16 @@ func (t *ProgramCreate) Execute( _ codec.Address, id ids.ID, _ bool, -) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +) (bool, uint64, []byte, error) { if len(t.Program) == 0 { - return false, 1, OutputValueZero, nil, nil + return false, 1, OutputValueZero, nil } if err := storage.SetProgram(ctx, mu, id, t.Program); err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } - return true, 1, nil, nil, nil + return true, 1, nil, nil } func (*ProgramCreate) MaxComputeUnits(chain.Rules) uint64 { @@ -72,7 +67,7 @@ func (t *ProgramCreate) Marshal(p *codec.Packer) { p.PackBytes(t.Program) } -func UnmarshalProgramCreate(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalProgramCreate(p *codec.Packer) (chain.Action, error) { var pc ProgramCreate p.UnpackBytes(-1, true, &pc.Program) return &pc, p.Err() diff --git a/x/programs/cmd/simulator/vm/actions/program_execute.go b/x/programs/cmd/simulator/vm/actions/program_execute.go index a6ea74c397..5977119a21 100644 --- a/x/programs/cmd/simulator/vm/actions/program_execute.go +++ b/x/programs/cmd/simulator/vm/actions/program_execute.go @@ -16,7 +16,6 @@ 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/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" @@ -53,10 +52,6 @@ func (*ProgramExecute) StateKeysMaxChunks() []uint16 { return []uint16{storage.ProgramChunks} } -func (*ProgramExecute) OutputsWarpMessage() bool { - return false -} - func (t *ProgramExecute) Execute( ctx context.Context, _ chain.Rules, @@ -65,40 +60,40 @@ func (t *ProgramExecute) Execute( actor codec.Address, _ ids.ID, _ bool, -) (success bool, computeUnits uint64, output []byte, warpMessage *warp.UnsignedMessage, err error) { +) (success bool, computeUnits uint64, output []byte, err error) { if len(t.Function) == 0 { - return false, 1, OutputValueZero, nil, nil + return false, 1, OutputValueZero, nil } if len(t.Params) == 0 { - return false, 1, OutputValueZero, nil, nil + return false, 1, OutputValueZero, nil } programIDStr, ok := t.Params[0].Value.(string) if !ok { - return false, 1, utils.ErrBytes(fmt.Errorf("invalid call param: must be ID")), nil, nil + return false, 1, utils.ErrBytes(fmt.Errorf("invalid call param: must be ID")), nil } // TODO: take fee out of balance? programID, err := ids.FromString(programIDStr) if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } programBytes, err := storage.GetProgram(ctx, mu, programID) if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } // TODO: get cfg from genesis cfg := runtime.NewConfig() if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } ecfg, err := engine.NewConfigBuilder(). WithDefaultCache(true). Build() if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } eng := engine.New(ecfg) @@ -121,22 +116,22 @@ func (t *ProgramExecute) Execute( t.rt = runtime.New(logging.NoLog{}, eng, imports, cfg) err = t.rt.Initialize(ctx, callContext, programBytes, t.MaxUnits) if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } defer t.rt.Stop() mem, err := t.rt.Memory() if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } params, err := WriteParams(mem, t.Params) if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } resp, err := t.rt.Call(ctx, t.Function, callContext, params[1:]...) if err != nil { - return false, 1, utils.ErrBytes(err), nil, nil + return false, 1, utils.ErrBytes(err), nil } // TODO: remove this is to support readonly response for now. @@ -145,7 +140,7 @@ func (t *ProgramExecute) Execute( p.PackInt64(r) } - return true, 1, p.Bytes(), nil, nil + return true, 1, p.Bytes(), nil } func (*ProgramExecute) MaxComputeUnits(chain.Rules) uint64 { @@ -164,7 +159,7 @@ func (t *ProgramExecute) GetBalance() (uint64, error) { return t.rt.Meter().GetBalance() } -func UnmarshalProgramExecute(p *codec.Packer, _ *warp.Message) (chain.Action, error) { +func UnmarshalProgramExecute(p *codec.Packer) (chain.Action, error) { // TODO return nil, nil } diff --git a/x/programs/cmd/simulator/vm/consts/consts.go b/x/programs/cmd/simulator/vm/consts/consts.go index bbf8a04a46..67a3270180 100644 --- a/x/programs/cmd/simulator/vm/consts/consts.go +++ b/x/programs/cmd/simulator/vm/consts/consts.go @@ -5,7 +5,6 @@ package consts import ( "github.com/ava-labs/avalanchego/ids" - "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/consts" @@ -33,6 +32,6 @@ func init() { // Instantiate registry here so it can be imported by any package. We set these // values in [controller/registry]. var ( - ActionRegistry *codec.TypeParser[chain.Action, *warp.Message, bool] - AuthRegistry *codec.TypeParser[chain.Auth, *warp.Message, bool] + ActionRegistry *codec.TypeParser[chain.Action, bool] + AuthRegistry *codec.TypeParser[chain.Auth, bool] ) diff --git a/x/programs/cmd/simulator/vm/genesis/genesis.go b/x/programs/cmd/simulator/vm/genesis/genesis.go index fcce6f63f2..24e9ed7162 100644 --- a/x/programs/cmd/simulator/vm/genesis/genesis.go +++ b/x/programs/cmd/simulator/vm/genesis/genesis.go @@ -48,9 +48,6 @@ type Genesis struct { // Tx Fee Parameters BaseComputeUnits uint64 `json:"baseUnits"` - BaseWarpComputeUnits uint64 `json:"baseWarpUnits"` - WarpComputeUnitsPerSigner uint64 `json:"warpUnitsPerSigner"` - OutgoingWarpComputeUnits uint64 `json:"outgoingWarpComputeUnits"` StorageKeyReadUnits uint64 `json:"storageKeyReadUnits"` StorageValueReadUnits uint64 `json:"storageValueReadUnits"` StorageKeyAllocateUnits uint64 `json:"storageKeyAllocateUnits"` @@ -85,9 +82,6 @@ func Default() *Genesis { // Tx Fee Compute Parameters BaseComputeUnits: 1, - BaseWarpComputeUnits: 1_024, - WarpComputeUnitsPerSigner: 128, - OutgoingWarpComputeUnits: 1_024, // Tx Fee Storage Parameters // diff --git a/x/programs/cmd/simulator/vm/genesis/rules.go b/x/programs/cmd/simulator/vm/genesis/rules.go index b3248485a5..f9b1c10d87 100644 --- a/x/programs/cmd/simulator/vm/genesis/rules.go +++ b/x/programs/cmd/simulator/vm/genesis/rules.go @@ -59,18 +59,6 @@ func (r *Rules) GetBaseComputeUnits() uint64 { return r.g.BaseComputeUnits } -func (r *Rules) GetBaseWarpComputeUnits() uint64 { - return r.g.BaseWarpComputeUnits -} - -func (r *Rules) GetWarpComputeUnitsPerSigner() uint64 { - return r.g.WarpComputeUnitsPerSigner -} - -func (r *Rules) GetOutgoingWarpComputeUnits() uint64 { - return r.g.OutgoingWarpComputeUnits -} - func (r *Rules) GetStorageKeyReadUnits() uint64 { return r.g.StorageKeyReadUnits } @@ -95,10 +83,6 @@ func (r *Rules) GetStorageValueWriteUnits() uint64 { return r.g.StorageValueWriteUnits } -func (*Rules) GetWarpConfig(ids.ID) (bool, uint64, uint64) { - return false, 0, 0 -} - func (r *Rules) NetworkID() uint32 { return r.networkID } diff --git a/x/programs/cmd/simulator/vm/registry/registry.go b/x/programs/cmd/simulator/vm/registry/registry.go index 9cfda75a8f..00df69e6a1 100644 --- a/x/programs/cmd/simulator/vm/registry/registry.go +++ b/x/programs/cmd/simulator/vm/registry/registry.go @@ -5,7 +5,6 @@ package registry import ( "github.com/ava-labs/avalanchego/utils/wrappers" - "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" @@ -15,8 +14,8 @@ import ( // Setup types func init() { - consts.ActionRegistry = codec.NewTypeParser[chain.Action, *warp.Message]() - consts.AuthRegistry = codec.NewTypeParser[chain.Auth, *warp.Message]() + consts.ActionRegistry = codec.NewTypeParser[chain.Action]() + consts.AuthRegistry = codec.NewTypeParser[chain.Auth]() errs := &wrappers.Errs{} errs.Add( diff --git a/x/programs/cmd/simulator/vm/storage/state_manager.go b/x/programs/cmd/simulator/vm/storage/state_manager.go index 05e6e38b06..678bfe3490 100644 --- a/x/programs/cmd/simulator/vm/storage/state_manager.go +++ b/x/programs/cmd/simulator/vm/storage/state_manager.go @@ -44,11 +44,3 @@ func (*StateManager) TimestampKey() []byte { func (*StateManager) FeeKey() []byte { return FeeKey() } - -func (*StateManager) IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) []byte { - return IncomingWarpKeyPrefix(sourceChainID, msgID) -} - -func (*StateManager) OutgoingWarpKeyPrefix(txID ids.ID) []byte { - return OutgoingWarpKeyPrefix(txID) -} diff --git a/x/programs/cmd/simulator/vm/storage/storage.go b/x/programs/cmd/simulator/vm/storage/storage.go index 0c22d4a878..0f0e02f565 100644 --- a/x/programs/cmd/simulator/vm/storage/storage.go +++ b/x/programs/cmd/simulator/vm/storage/storage.go @@ -28,8 +28,6 @@ const ( heightPrefix = 0x2 timestampPrefix = 0x3 feePrefix = 0x4 - incomingWarpPrefix = 0x5 - outgoingWarpPrefix = 0x6 ) var ( @@ -174,18 +172,3 @@ func TimestampKey() (k []byte) { func FeeKey() (k []byte) { return feeKey } - -func IncomingWarpKeyPrefix(sourceChainID ids.ID, msgID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen*2) - k[0] = incomingWarpPrefix - copy(k[1:], sourceChainID[:]) - copy(k[1+consts.IDLen:], msgID[:]) - return k -} - -func OutgoingWarpKeyPrefix(txID ids.ID) (k []byte) { - k = make([]byte, 1+consts.IDLen) - k[0] = outgoingWarpPrefix - copy(k[1:], txID[:]) - return k -} From ef177a2ec4c7e305ded8f25fb87822c88a63c7a1 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 14:54:25 -0400 Subject: [PATCH 02/29] fix type_parser tests --- chain/consts.go | 2 -- chain/transaction.go | 2 +- codec/type_parser.go | 20 ++++++++++---------- codec/type_parser_test.go | 10 +++++----- examples/tokenvm/actions/burn_asset.go | 14 +++++++------- 5 files changed, 23 insertions(+), 25 deletions(-) diff --git a/chain/consts.go b/chain/consts.go index 816ef9f307..4dba91a246 100644 --- a/chain/consts.go +++ b/chain/consts.go @@ -6,8 +6,6 @@ package chain import ( "time" - "github.com/ava-labs/avalanchego/utils/units" - "github.com/ava-labs/hypersdk/keys" ) diff --git a/chain/transaction.go b/chain/transaction.go index a281b75d4c..c6ff3b4488 100644 --- a/chain/transaction.go +++ b/chain/transaction.go @@ -308,7 +308,7 @@ func (t *Transaction) Execute( // are set when this function is defined. If any of them are // modified later, they will not be used here. ts.Rollback(ctx, actionStart) - return &Result{false, utils.ErrBytes(rerr), maxUnits, maxFee, nil}, nil + return &Result{false, utils.ErrBytes(rerr), maxUnits, maxFee}, nil } success, actionCUs, output, err := t.Action.Execute(ctx, r, ts, timestamp, t.Auth.Actor(), t.id) if err != nil { diff --git a/codec/type_parser.go b/codec/type_parser.go index 0614c7b151..74f975cfec 100644 --- a/codec/type_parser.go +++ b/codec/type_parser.go @@ -7,42 +7,42 @@ import ( "github.com/ava-labs/hypersdk/consts" ) -type decoder[T any, X any, Y any] struct { - f func(*Packer, X) (T, error) +type decoder[T any, Y any] struct { + f func(*Packer) (T, error) y Y } // The number of types is limited to 255. -type TypeParser[T any, X any, Y any] struct { +type TypeParser[T any, Y any] struct { typeToIndex map[string]uint8 - indexToDecoder map[uint8]*decoder[T, X, Y] + indexToDecoder map[uint8]*decoder[T, Y] } // NewTypeParser returns an instance of a Typeparser with generic type [T]. -func NewTypeParser[T any, X any, Y bool]() *TypeParser[T, X, Y] { - return &TypeParser[T, X, Y]{ +func NewTypeParser[T any, Y bool]() *TypeParser[T, Y] { + return &TypeParser[T, Y]{ typeToIndex: map[string]uint8{}, - indexToDecoder: map[uint8]*decoder[T, X, Y]{}, + indexToDecoder: map[uint8]*decoder[T, Y]{}, } } // Register registers a new type into TypeParser [p]. Registers the type by using // the string representation of [o], and sets the decoder of that index to [f]. // Returns an error if [o] has already been registered or the TypeParser is full. -func (p *TypeParser[T, X, Y]) Register(id uint8, f func(*Packer, X) (T, error), y Y) error { +func (p *TypeParser[T, Y]) Register(id uint8, f func(*Packer) (T, error), y Y) error { if len(p.indexToDecoder) == int(consts.MaxUint8)+1 { return ErrTooManyItems } if _, ok := p.indexToDecoder[id]; ok { return ErrDuplicateItem } - p.indexToDecoder[id] = &decoder[T, X, Y]{f, y} + p.indexToDecoder[id] = &decoder[T, Y]{f, y} return nil } // LookupIndex returns the decoder function and success of lookup of [index] // from Typeparser [p]. -func (p *TypeParser[T, X, Y]) LookupIndex(index uint8) (func(*Packer, X) (T, error), Y, bool) { +func (p *TypeParser[T, Y]) LookupIndex(index uint8) (func(*Packer) (T, error), Y, bool) { d, ok := p.indexToDecoder[index] if ok { return d.f, d.y, true diff --git a/codec/type_parser_test.go b/codec/type_parser_test.go index 76f6b04ad7..1cbb71b4fa 100644 --- a/codec/type_parser_test.go +++ b/codec/type_parser_test.go @@ -34,7 +34,7 @@ func (*Blah3) Bark() string { return "blah3" } func (*Blah3) GetTypeID() uint8 { return 2 } func TestTypeParser(t *testing.T) { - tp := NewTypeParser[Blah, any, bool]() + tp := NewTypeParser[Blah, bool]() t.Run("empty parser", func(t *testing.T) { require := require.New(t) @@ -54,14 +54,14 @@ func TestTypeParser(t *testing.T) { require.NoError( tp.Register( blah1.GetTypeID(), - func(*Packer, any) (Blah, error) { return nil, errBlah1 }, + func(*Packer) (Blah, error) { return nil, errBlah1 }, true, ), ) require.NoError( tp.Register( blah2.GetTypeID(), - func(*Packer, any) (Blah, error) { return nil, errBlah2 }, + func(*Packer) (Blah, error) { return nil, errBlah2 }, false, ), ) @@ -69,14 +69,14 @@ func TestTypeParser(t *testing.T) { f, b, ok := tp.LookupIndex(blah1.GetTypeID()) require.True(ok) require.True(b) - res, err := f(nil, nil) + res, err := f(nil) require.Nil(res) require.ErrorIs(err, errBlah1) f, b, ok = tp.LookupIndex(blah2.GetTypeID()) require.True(ok) require.False(b) - res, err = f(nil, nil) + res, err = f(nil) require.Nil(res) require.ErrorIs(err, errBlah2) }) diff --git a/examples/tokenvm/actions/burn_asset.go b/examples/tokenvm/actions/burn_asset.go index 8eb18e6bab..bffc8290fb 100644 --- a/examples/tokenvm/actions/burn_asset.go +++ b/examples/tokenvm/actions/burn_asset.go @@ -52,26 +52,26 @@ func (b *BurnAsset) Execute( _ bool, ) (bool, uint64, []byte, error) { if b.Value == 0 { - return false, BurnComputeUnits, OutputValueZero, nil, nil + return false, BurnComputeUnits, OutputValueZero, nil } if err := storage.SubBalance(ctx, mu, actor, b.Asset, b.Value); err != nil { - return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil + return false, BurnComputeUnits, utils.ErrBytes(err), nil } exists, symbol, decimals, metadata, supply, owner, err := storage.GetAsset(ctx, mu, b.Asset) if err != nil { - return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil + return false, BurnComputeUnits, utils.ErrBytes(err), nil } if !exists { - return false, BurnComputeUnits, OutputAssetMissing, nil, nil + return false, BurnComputeUnits, OutputAssetMissing, nil } newSupply, err := smath.Sub(supply, b.Value) if err != nil { - return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil + return false, BurnComputeUnits, utils.ErrBytes(err), nil } if err := storage.SetAsset(ctx, mu, b.Asset, symbol, decimals, metadata, newSupply, owner); err != nil { - return false, BurnComputeUnits, utils.ErrBytes(err), nil, nil + return false, BurnComputeUnits, utils.ErrBytes(err), nil } - return true, BurnComputeUnits, nil, nil, nil + return true, BurnComputeUnits, nil, nil } func (*BurnAsset) MaxComputeUnits(chain.Rules) uint64 { From 17406c4fadffa92cbdeb275259c04ebc49f99cab Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 14:57:26 -0400 Subject: [PATCH 03/29] fix chain/transaction --- chain/builder.go | 1 - chain/transaction.go | 2 -- codec/type_parser.go | 6 +++--- codec/type_parser_test.go | 9 +++------ 4 files changed, 6 insertions(+), 12 deletions(-) diff --git a/chain/builder.go b/chain/builder.go index b589c558c7..9d4f1120f2 100644 --- a/chain/builder.go +++ b/chain/builder.go @@ -129,7 +129,6 @@ func BuildBlock( txsAttempted = 0 results = []*Result{} - vdrState = vm.ValidatorState() sm = vm.StateManager() // prepareStreamLock ensures we don't overwrite stream prefetching spawned diff --git a/chain/transaction.go b/chain/transaction.go index c6ff3b4488..057f7f02a7 100644 --- a/chain/transaction.go +++ b/chain/transaction.go @@ -5,10 +5,8 @@ package chain import ( "context" - "errors" "fmt" - "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/codec" diff --git a/codec/type_parser.go b/codec/type_parser.go index 74f975cfec..dddf099281 100644 --- a/codec/type_parser.go +++ b/codec/type_parser.go @@ -42,10 +42,10 @@ func (p *TypeParser[T, Y]) Register(id uint8, f func(*Packer) (T, error), y Y) e // LookupIndex returns the decoder function and success of lookup of [index] // from Typeparser [p]. -func (p *TypeParser[T, Y]) LookupIndex(index uint8) (func(*Packer) (T, error), Y, bool) { +func (p *TypeParser[T, Y]) LookupIndex(index uint8) (func(*Packer) (T, error), bool) { d, ok := p.indexToDecoder[index] if ok { - return d.f, d.y, true + return d.f, true } - return nil, *new(Y), false + return nil, false } diff --git a/codec/type_parser_test.go b/codec/type_parser_test.go index 1cbb71b4fa..a59a965a50 100644 --- a/codec/type_parser_test.go +++ b/codec/type_parser_test.go @@ -38,10 +38,9 @@ func TestTypeParser(t *testing.T) { t.Run("empty parser", func(t *testing.T) { require := require.New(t) - f, b, ok := tp.LookupIndex(0) + f, ok := tp.LookupIndex(0) require.Nil(f) require.False(ok) - require.False(b) }) t.Run("populated parser", func(t *testing.T) { @@ -66,16 +65,14 @@ func TestTypeParser(t *testing.T) { ), ) - f, b, ok := tp.LookupIndex(blah1.GetTypeID()) + f, ok := tp.LookupIndex(blah1.GetTypeID()) require.True(ok) - require.True(b) res, err := f(nil) require.Nil(res) require.ErrorIs(err, errBlah1) - f, b, ok = tp.LookupIndex(blah2.GetTypeID()) + f, ok = tp.LookupIndex(blah2.GetTypeID()) require.True(ok) - require.False(b) res, err = f(nil) require.Nil(res) require.ErrorIs(err, errBlah2) From 22c3b71848b9e373d783d9d7f303605f144c4fe3 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 15:03:24 -0400 Subject: [PATCH 04/29] add back ShouldVerifyWithContext --- chain/block.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/chain/block.go b/chain/block.go index 687ad80db5..d915fa2e5f 100644 --- a/chain/block.go +++ b/chain/block.go @@ -256,6 +256,11 @@ func (b *StatelessBlock) initializeBuilt( // implements "snowman.Block.choices.Decidable" func (b *StatelessBlock) ID() ids.ID { return b.id } +// implements "block.WithVerifyContext" +func (b *StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { + return false, nil +} + // implements "block.WithVerifyContext" func (b *StatelessBlock) VerifyWithContext(ctx context.Context, bctx *block.Context) error { start := time.Now() From e1ea721d00978328e9964a30e280904328818f37 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 15:38:58 -0400 Subject: [PATCH 05/29] fix rpc --- chain/mock_action.go | 28 +++++---------------- chain/mock_rules.go | 58 ------------------------------------------- rpc/jsonrpc_client.go | 7 ------ rpc/jsonrpc_server.go | 3 --- 4 files changed, 6 insertions(+), 90 deletions(-) diff --git a/chain/mock_action.go b/chain/mock_action.go index 6b75860ab4..920b763dbd 100644 --- a/chain/mock_action.go +++ b/chain/mock_action.go @@ -17,7 +17,6 @@ import ( reflect "reflect" ids "github.com/ava-labs/avalanchego/ids" - warp "github.com/ava-labs/avalanchego/vms/platformvm/warp" codec "github.com/ava-labs/hypersdk/codec" state "github.com/ava-labs/hypersdk/state" gomock "go.uber.org/mock/gomock" @@ -47,21 +46,20 @@ func (m *MockAction) EXPECT() *MockActionMockRecorder { } // Execute mocks base method. -func (m *MockAction) Execute(arg0 context.Context, arg1 Rules, arg2 state.Mutable, arg3 int64, arg4 codec.Address, arg5 ids.ID, arg6 bool) (bool, uint64, []byte, *warp.UnsignedMessage, error) { +func (m *MockAction) Execute(arg0 context.Context, arg1 Rules, arg2 state.Mutable, arg3 int64, arg4 codec.Address, arg5 ids.ID) (bool, uint64, []byte, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Execute", arg0, arg1, arg2, arg3, arg4, arg5, arg6) + ret := m.ctrl.Call(m, "Execute", arg0, arg1, arg2, arg3, arg4, arg5) ret0, _ := ret[0].(bool) ret1, _ := ret[1].(uint64) ret2, _ := ret[2].([]byte) - ret3, _ := ret[3].(*warp.UnsignedMessage) - ret4, _ := ret[4].(error) - return ret0, ret1, ret2, ret3, ret4 + ret3, _ := ret[3].(error) + return ret0, ret1, ret2, ret3 } // Execute indicates an expected call of Execute. -func (mr *MockActionMockRecorder) Execute(arg0, arg1, arg2, arg3, arg4, arg5, arg6 any) *gomock.Call { +func (mr *MockActionMockRecorder) Execute(arg0, arg1, arg2, arg3, arg4, arg5 any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Execute", reflect.TypeOf((*MockAction)(nil).Execute), arg0, arg1, arg2, arg3, arg4, arg5, arg6) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Execute", reflect.TypeOf((*MockAction)(nil).Execute), arg0, arg1, arg2, arg3, arg4, arg5) } // GetTypeID mocks base method. @@ -104,20 +102,6 @@ func (mr *MockActionMockRecorder) MaxComputeUnits(arg0 any) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MaxComputeUnits", reflect.TypeOf((*MockAction)(nil).MaxComputeUnits), arg0) } -// OutputsWarpMessage mocks base method. -func (m *MockAction) OutputsWarpMessage() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "OutputsWarpMessage") - ret0, _ := ret[0].(bool) - return ret0 -} - -// OutputsWarpMessage indicates an expected call of OutputsWarpMessage. -func (mr *MockActionMockRecorder) OutputsWarpMessage() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "OutputsWarpMessage", reflect.TypeOf((*MockAction)(nil).OutputsWarpMessage)) -} - // Size mocks base method. func (m *MockAction) Size() int { m.ctrl.T.Helper() diff --git a/chain/mock_rules.go b/chain/mock_rules.go index c0ac1ba3bc..feabddc7a7 100644 --- a/chain/mock_rules.go +++ b/chain/mock_rules.go @@ -86,20 +86,6 @@ func (mr *MockRulesMockRecorder) GetBaseComputeUnits() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBaseComputeUnits", reflect.TypeOf((*MockRules)(nil).GetBaseComputeUnits)) } -// GetBaseWarpComputeUnits mocks base method. -func (m *MockRules) GetBaseWarpComputeUnits() uint64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetBaseWarpComputeUnits") - ret0, _ := ret[0].(uint64) - return ret0 -} - -// GetBaseWarpComputeUnits indicates an expected call of GetBaseWarpComputeUnits. -func (mr *MockRulesMockRecorder) GetBaseWarpComputeUnits() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBaseWarpComputeUnits", reflect.TypeOf((*MockRules)(nil).GetBaseWarpComputeUnits)) -} - // GetMaxBlockUnits mocks base method. func (m *MockRules) GetMaxBlockUnits() fees.Dimensions { m.ctrl.T.Helper() @@ -156,20 +142,6 @@ func (mr *MockRulesMockRecorder) GetMinUnitPrice() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetMinUnitPrice", reflect.TypeOf((*MockRules)(nil).GetMinUnitPrice)) } -// GetOutgoingWarpComputeUnits mocks base method. -func (m *MockRules) GetOutgoingWarpComputeUnits() uint64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetOutgoingWarpComputeUnits") - ret0, _ := ret[0].(uint64) - return ret0 -} - -// GetOutgoingWarpComputeUnits indicates an expected call of GetOutgoingWarpComputeUnits. -func (mr *MockRulesMockRecorder) GetOutgoingWarpComputeUnits() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOutgoingWarpComputeUnits", reflect.TypeOf((*MockRules)(nil).GetOutgoingWarpComputeUnits)) -} - // GetSponsorStateKeysMaxChunks mocks base method. func (m *MockRules) GetSponsorStateKeysMaxChunks() []uint16 { m.ctrl.T.Helper() @@ -296,36 +268,6 @@ func (mr *MockRulesMockRecorder) GetValidityWindow() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetValidityWindow", reflect.TypeOf((*MockRules)(nil).GetValidityWindow)) } -// GetWarpComputeUnitsPerSigner mocks base method. -func (m *MockRules) GetWarpComputeUnitsPerSigner() uint64 { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetWarpComputeUnitsPerSigner") - ret0, _ := ret[0].(uint64) - return ret0 -} - -// GetWarpComputeUnitsPerSigner indicates an expected call of GetWarpComputeUnitsPerSigner. -func (mr *MockRulesMockRecorder) GetWarpComputeUnitsPerSigner() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWarpComputeUnitsPerSigner", reflect.TypeOf((*MockRules)(nil).GetWarpComputeUnitsPerSigner)) -} - -// GetWarpConfig mocks base method. -func (m *MockRules) GetWarpConfig(arg0 ids.ID) (bool, uint64, uint64) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetWarpConfig", arg0) - ret0, _ := ret[0].(bool) - ret1, _ := ret[1].(uint64) - ret2, _ := ret[2].(uint64) - return ret0, ret1, ret2 -} - -// GetWarpConfig indicates an expected call of GetWarpConfig. -func (mr *MockRulesMockRecorder) GetWarpConfig(arg0 any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetWarpConfig", reflect.TypeOf((*MockRules)(nil).GetWarpConfig), arg0) -} - // GetWindowTargetUnits mocks base method. func (m *MockRules) GetWindowTargetUnits() fees.Dimensions { m.ctrl.T.Helper() diff --git a/rpc/jsonrpc_client.go b/rpc/jsonrpc_client.go index 54c3d6f285..fedccd7d7d 100644 --- a/rpc/jsonrpc_client.go +++ b/rpc/jsonrpc_client.go @@ -10,18 +10,11 @@ import ( "time" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow/validators" - "github.com/ava-labs/avalanchego/utils/crypto/bls" - "github.com/ava-labs/avalanchego/utils/math" - "github.com/ava-labs/avalanchego/utils/set" - "golang.org/x/exp/maps" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/fees" "github.com/ava-labs/hypersdk/requester" "github.com/ava-labs/hypersdk/utils" - - avautils "github.com/ava-labs/avalanchego/utils" ) const ( diff --git a/rpc/jsonrpc_server.go b/rpc/jsonrpc_server.go index 9860219fff..509a087e14 100644 --- a/rpc/jsonrpc_server.go +++ b/rpc/jsonrpc_server.go @@ -4,14 +4,11 @@ package rpc import ( - "context" "errors" "fmt" "net/http" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/utils/crypto/bls" - "go.uber.org/zap" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" From 3431db5e3c0f0e7b3a50dab90de781b7b9c4c2d0 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 15:45:51 -0400 Subject: [PATCH 06/29] mock gen works --- go.mod | 2 +- rpc/dependencies.go | 1 - vm/storage.go | 4 ---- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 545118f941..e7b5855d7b 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,6 @@ require ( go.uber.org/mock v0.4.0 go.uber.org/zap v1.26.0 golang.org/x/crypto v0.17.0 - golang.org/x/exp v0.0.0-20231127185646-65229373498e golang.org/x/sync v0.5.0 gopkg.in/yaml.v2 v2.4.0 ) @@ -76,6 +75,7 @@ require ( go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.11.2 // indirect go.opentelemetry.io/proto/otlp v0.19.0 // indirect go.uber.org/multierr v1.11.0 // indirect + golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.19.0 // indirect golang.org/x/sys v0.15.0 // indirect golang.org/x/term v0.15.0 // indirect diff --git a/rpc/dependencies.go b/rpc/dependencies.go index 67f9e0d567..028aaacc06 100644 --- a/rpc/dependencies.go +++ b/rpc/dependencies.go @@ -32,6 +32,5 @@ type VM interface { CurrentValidators( context.Context, ) (map[ids.NodeID]*validators.GetValidatorOutput, map[string]struct{}) - GatherSignatures(context.Context, ids.ID, []byte) GetVerifyAuth() bool } diff --git a/vm/storage.go b/vm/storage.go index 93013e462f..9c4d36df0d 100644 --- a/vm/storage.go +++ b/vm/storage.go @@ -4,7 +4,6 @@ package vm import ( - "bytes" "context" "encoding/binary" "errors" @@ -12,16 +11,13 @@ import ( "math/rand" "time" - "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/choices" - "github.com/ava-labs/avalanchego/utils/crypto/bls" "go.uber.org/zap" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/consts" - "github.com/ava-labs/hypersdk/keys" ) // compactionOffset is used to randomize the height that we compact From c7acc0db67ae9b433cdcf41d94ff66806b0c0c83 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 16:03:22 -0400 Subject: [PATCH 07/29] morpheusvm integration passes --- examples/morpheusvm/actions/transfer.go | 1 - examples/morpheusvm/storage/state_manager.go | 1 - .../tests/integration/integration_test.go | 42 ++++++------------- 3 files changed, 13 insertions(+), 31 deletions(-) diff --git a/examples/morpheusvm/actions/transfer.go b/examples/morpheusvm/actions/transfer.go index 8ff8c9c404..d478e698a6 100644 --- a/examples/morpheusvm/actions/transfer.go +++ b/examples/morpheusvm/actions/transfer.go @@ -48,7 +48,6 @@ func (t *Transfer) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if t.Value == 0 { return false, 1, OutputValueZero, nil diff --git a/examples/morpheusvm/storage/state_manager.go b/examples/morpheusvm/storage/state_manager.go index 34a3c7810e..9e27c216fc 100644 --- a/examples/morpheusvm/storage/state_manager.go +++ b/examples/morpheusvm/storage/state_manager.go @@ -6,7 +6,6 @@ package storage import ( "context" - "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/state" diff --git a/examples/morpheusvm/tests/integration/integration_test.go b/examples/morpheusvm/tests/integration/integration_test.go index d6a4cd800a..0955c9c2c2 100644 --- a/examples/morpheusvm/tests/integration/integration_test.go +++ b/examples/morpheusvm/tests/integration/integration_test.go @@ -334,7 +334,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, transferTx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 100_000, // must be more than StateLockup @@ -367,7 +366,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: 0, MaxFee: 1000, }, - nil, &actions.Transfer{ To: addr2, Value: 110, @@ -434,19 +432,19 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // read: 2 keys reads, 1 had 0 chunks // allocate: 1 key created with 1 chunk // write: 2 keys modified (new + old) - transferTxConsumed := fees.Dimensions{191, 7, 12, 25, 26} + transferTxConsumed := fees.Dimensions{187, 7, 12, 25, 26} gomega.Ω(results[0].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(261))) + gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(257))) }) ginkgo.By("ensure balance is updated", func() { balance, err := instances[1].lcli.Balance(context.Background(), addrStr) gomega.Ω(err).To(gomega.BeNil()) - gomega.Ω(balance).To(gomega.Equal(uint64(9899739))) + gomega.Ω(balance).To(gomega.Equal(uint64(9899743))) balance2, err := instances[1].lcli.Balance(context.Background(), addrStr2) gomega.Ω(err).To(gomega.BeNil()) gomega.Ω(balance2).To(gomega.Equal(uint64(100000))) @@ -460,7 +458,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 101, @@ -481,13 +478,13 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // read: 2 keys reads, 1 chunk each // allocate: 0 key created // write: 2 key modified - transferTxConsumed := fees.Dimensions{191, 7, 14, 0, 26} + transferTxConsumed := fees.Dimensions{187, 7, 14, 0, 26} gomega.Ω(results[0].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(238))) + gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(234))) balance2, err := instances[1].lcli.Balance(context.Background(), addrStr2) gomega.Ω(err).To(gomega.BeNil()) @@ -501,7 +498,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 102, @@ -513,7 +509,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 103, @@ -525,7 +520,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr3, Value: 104, @@ -537,7 +531,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr3, Value: 105, @@ -566,12 +559,12 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // allocate: 0 key created // write: 2 key modified gomega.Ω(results[0].Success).Should(gomega.BeTrue()) - transferTxConsumed := fees.Dimensions{191, 7, 14, 0, 26} + transferTxConsumed := fees.Dimensions{187, 7, 14, 0, 26} gomega.Ω(results[0].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(238))) + gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(234))) // Unit explanation // @@ -581,12 +574,12 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // allocate: 0 key created // write: 2 keys modified gomega.Ω(results[1].Success).Should(gomega.BeTrue()) - transferTxConsumed = fees.Dimensions{191, 7, 14, 0, 26} + transferTxConsumed = fees.Dimensions{187, 7, 14, 0, 26} gomega.Ω(results[1].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[1].Fee).Should(gomega.Equal(uint64(238))) + gomega.Ω(results[1].Fee).Should(gomega.Equal(uint64(234))) // Unit explanation // @@ -596,12 +589,12 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // allocate: 1 key created (1 chunk) // write: 2 key modified (1 chunk), both previously modified gomega.Ω(results[2].Success).Should(gomega.BeTrue()) - transferTxConsumed = fees.Dimensions{191, 7, 12, 25, 26} + transferTxConsumed = fees.Dimensions{187, 7, 12, 25, 26} gomega.Ω(results[2].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[2].Fee).Should(gomega.Equal(uint64(261))) + gomega.Ω(results[2].Fee).Should(gomega.Equal(uint64(257))) // Unit explanation // @@ -611,12 +604,12 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // allocate: 0 key created // write: 2 keys modified (1 chunk) gomega.Ω(results[3].Success).Should(gomega.BeTrue()) - transferTxConsumed = fees.Dimensions{191, 7, 12, 0, 26} + transferTxConsumed = fees.Dimensions{187, 7, 12, 0, 26} gomega.Ω(results[3].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[3].Fee).Should(gomega.Equal(uint64(236))) + gomega.Ω(results[3].Fee).Should(gomega.Equal(uint64(232))) // Check end balance balance2, err := instances[1].lcli.Balance(context.Background(), addrStr2) @@ -637,7 +630,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 200, @@ -651,7 +643,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 201, @@ -680,7 +671,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr2, Value: 203, @@ -771,7 +761,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, transfer, factory, ) @@ -819,7 +808,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { _, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, transfer, factory, ) @@ -868,7 +856,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: r1addr, Value: 2000, @@ -893,7 +880,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr, Value: 100, @@ -922,7 +908,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: r1addr, Value: 2000, @@ -947,7 +932,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: addr, Value: 100, From 6ec989d73f6f6625bddbf47a4de5c2288de8575b Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 16:09:12 -0400 Subject: [PATCH 08/29] fix load & e2e for morpheus --- examples/morpheusvm/tests/e2e/e2e_test.go | 3 --- examples/morpheusvm/tests/load/load_test.go | 1 - 2 files changed, 4 deletions(-) diff --git a/examples/morpheusvm/tests/e2e/e2e_test.go b/examples/morpheusvm/tests/e2e/e2e_test.go index 039b69c984..bf352848bc 100644 --- a/examples/morpheusvm/tests/e2e/e2e_test.go +++ b/examples/morpheusvm/tests/e2e/e2e_test.go @@ -424,7 +424,6 @@ var _ = ginkgo.Describe("[Test]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: aother, Value: sendAmount, @@ -717,7 +716,6 @@ func generateBlocks( submit, _, _, err := instances[cumulativeTxs%len(instances)].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: aother, Value: 1, @@ -786,7 +784,6 @@ func acceptTransaction(cli *rpc.JSONRPCClient, lcli *lrpc.JSONRPCClient) { submit, tx, maxFee, err := cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: aother, Value: 1, diff --git a/examples/morpheusvm/tests/load/load_test.go b/examples/morpheusvm/tests/load/load_test.go index 5f98fd5b68..634826ccd1 100644 --- a/examples/morpheusvm/tests/load/load_test.go +++ b/examples/morpheusvm/tests/load/load_test.go @@ -542,7 +542,6 @@ func issueSimpleTx( ChainID: i.chainID, MaxFee: maxFee, }, - nil, &actions.Transfer{ To: to, Value: amount, From 600a2cdf3808d4ab79e14fb943f02381fdd33cb1 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 16:22:26 -0400 Subject: [PATCH 09/29] tokenvm integration passes --- examples/tokenvm/actions/burn_asset.go | 1 - examples/tokenvm/actions/close_order.go | 1 - examples/tokenvm/actions/create_asset.go | 11 ++++- examples/tokenvm/actions/create_order.go | 1 - examples/tokenvm/actions/fill_order.go | 1 - examples/tokenvm/actions/mint_asset.go | 1 - examples/tokenvm/actions/transfer.go | 1 - examples/tokenvm/controller/controller.go | 4 -- examples/tokenvm/controller/resolutions.go | 2 +- examples/tokenvm/genesis/genesis.go | 1 - examples/tokenvm/rpc/dependencies.go | 2 +- examples/tokenvm/rpc/jsonrpc_client.go | 8 ++-- examples/tokenvm/storage/storage.go | 10 ++--- .../tests/integration/integration_test.go | 45 ++----------------- 14 files changed, 24 insertions(+), 65 deletions(-) diff --git a/examples/tokenvm/actions/burn_asset.go b/examples/tokenvm/actions/burn_asset.go index bffc8290fb..b72b49a577 100644 --- a/examples/tokenvm/actions/burn_asset.go +++ b/examples/tokenvm/actions/burn_asset.go @@ -49,7 +49,6 @@ func (b *BurnAsset) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if b.Value == 0 { return false, BurnComputeUnits, OutputValueZero, nil diff --git a/examples/tokenvm/actions/close_order.go b/examples/tokenvm/actions/close_order.go index 7990340f98..862d142752 100644 --- a/examples/tokenvm/actions/close_order.go +++ b/examples/tokenvm/actions/close_order.go @@ -48,7 +48,6 @@ func (c *CloseOrder) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { exists, _, _, out, _, remaining, owner, err := storage.GetOrder(ctx, mu, c.Order) if err != nil { diff --git a/examples/tokenvm/actions/create_asset.go b/examples/tokenvm/actions/create_asset.go index e3a5dba162..a044e3bacd 100644 --- a/examples/tokenvm/actions/create_asset.go +++ b/examples/tokenvm/actions/create_asset.go @@ -44,7 +44,6 @@ func (c *CreateAsset) Execute( _ int64, actor codec.Address, txID ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if len(c.Symbol) == 0 { return false, CreateAssetComputeUnits, OutputSymbolEmpty, nil @@ -63,7 +62,7 @@ func (c *CreateAsset) Execute( } // It should only be possible to overwrite an existing asset if there is // a hash collision. - if err := storage.SetAsset(ctx, mu, txID, c.Symbol, c.Decimals, c.Metadata, 0, actor, false); err != nil { + if err := storage.SetAsset(ctx, mu, txID, c.Symbol, c.Decimals, c.Metadata, 0, actor); err != nil { return false, CreateAssetComputeUnits, utils.ErrBytes(err), nil } return true, CreateAssetComputeUnits, nil, nil @@ -84,6 +83,14 @@ func (c *CreateAsset) Marshal(p *codec.Packer) { p.PackBytes(c.Metadata) } +func UnmarshalCreateAsset(p *codec.Packer) (chain.Action, error) { + var create CreateAsset + p.UnpackBytes(MaxSymbolSize, true, &create.Symbol) + create.Decimals = p.UnpackByte() + p.UnpackBytes(MaxMetadataSize, true, &create.Metadata) + return &create, p.Err() +} + func (*CreateAsset) ValidRange(chain.Rules) (int64, int64) { // Returning -1, -1 means that the action is always valid. return -1, -1 diff --git a/examples/tokenvm/actions/create_order.go b/examples/tokenvm/actions/create_order.go index 8435f05dd7..243e4ad9fb 100644 --- a/examples/tokenvm/actions/create_order.go +++ b/examples/tokenvm/actions/create_order.go @@ -68,7 +68,6 @@ func (c *CreateOrder) Execute( _ int64, actor codec.Address, txID ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if c.In == c.Out { return false, CreateOrderComputeUnits, OutputSameInOut, nil diff --git a/examples/tokenvm/actions/fill_order.go b/examples/tokenvm/actions/fill_order.go index 199d1c134b..0c16454642 100644 --- a/examples/tokenvm/actions/fill_order.go +++ b/examples/tokenvm/actions/fill_order.go @@ -63,7 +63,6 @@ func (f *FillOrder) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { exists, in, inTick, out, outTick, remaining, owner, err := storage.GetOrder(ctx, mu, f.Order) if err != nil { diff --git a/examples/tokenvm/actions/mint_asset.go b/examples/tokenvm/actions/mint_asset.go index c349b39609..52e05ca576 100644 --- a/examples/tokenvm/actions/mint_asset.go +++ b/examples/tokenvm/actions/mint_asset.go @@ -52,7 +52,6 @@ func (m *MintAsset) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if m.Asset == ids.Empty { return false, MintAssetComputeUnits, OutputAssetIsNative, nil diff --git a/examples/tokenvm/actions/transfer.go b/examples/tokenvm/actions/transfer.go index 29f05bb7ff..f1630f10c0 100644 --- a/examples/tokenvm/actions/transfer.go +++ b/examples/tokenvm/actions/transfer.go @@ -53,7 +53,6 @@ func (t *Transfer) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if t.Value == 0 { return false, TransferComputeUnits, OutputValueZero, nil diff --git a/examples/tokenvm/controller/controller.go b/examples/tokenvm/controller/controller.go index e5eecb4196..a84dcdc5d4 100644 --- a/examples/tokenvm/controller/controller.go +++ b/examples/tokenvm/controller/controller.go @@ -206,10 +206,6 @@ func (c *Controller) Accepted(ctx context.Context, blk *chain.StatelessBlock) er case *actions.CloseOrder: c.metrics.closeOrder.Inc() c.orderBook.Remove(action.Order) - case *actions.ImportAsset: - c.metrics.importAsset.Inc() - case *actions.ExportAsset: - c.metrics.exportAsset.Inc() } } } diff --git a/examples/tokenvm/controller/resolutions.go b/examples/tokenvm/controller/resolutions.go index a20510b9bc..ccdd47882e 100644 --- a/examples/tokenvm/controller/resolutions.go +++ b/examples/tokenvm/controller/resolutions.go @@ -38,7 +38,7 @@ func (c *Controller) GetTransaction( func (c *Controller) GetAssetFromState( ctx context.Context, asset ids.ID, -) (bool, []byte, uint8, []byte, uint64, codec.Address, bool, error) { +) (bool, []byte, uint8, []byte, uint64, codec.Address, error) { return storage.GetAssetFromState(ctx, c.inner.ReadState, asset) } diff --git a/examples/tokenvm/genesis/genesis.go b/examples/tokenvm/genesis/genesis.go index 983ff98763..0832cb5532 100644 --- a/examples/tokenvm/genesis/genesis.go +++ b/examples/tokenvm/genesis/genesis.go @@ -133,7 +133,6 @@ func (g *Genesis) Load(ctx context.Context, tracer trace.Tracer, mu state.Mutabl []byte(consts.Name), supply, codec.EmptyAddress, - false, ) } diff --git a/examples/tokenvm/rpc/dependencies.go b/examples/tokenvm/rpc/dependencies.go index dd07ad7c5c..fd43a854fa 100644 --- a/examples/tokenvm/rpc/dependencies.go +++ b/examples/tokenvm/rpc/dependencies.go @@ -18,7 +18,7 @@ type Controller interface { Genesis() *genesis.Genesis Tracer() trace.Tracer GetTransaction(context.Context, ids.ID) (bool, int64, bool, fees.Dimensions, uint64, error) - GetAssetFromState(context.Context, ids.ID) (bool, []byte, uint8, []byte, uint64, codec.Address, bool, error) + GetAssetFromState(context.Context, ids.ID) (bool, []byte, uint8, []byte, uint64, codec.Address, error) GetBalanceFromState(context.Context, codec.Address, ids.ID) (uint64, error) Orders(pair string, limit int) []*orderbook.Order GetOrderFromState(context.Context, ids.ID) ( diff --git a/examples/tokenvm/rpc/jsonrpc_client.go b/examples/tokenvm/rpc/jsonrpc_client.go index de55235e1b..934ef3700f 100644 --- a/examples/tokenvm/rpc/jsonrpc_client.go +++ b/examples/tokenvm/rpc/jsonrpc_client.go @@ -86,7 +86,7 @@ func (cli *JSONRPCClient) Asset( ctx context.Context, asset ids.ID, useCache bool, -) (bool, []byte, uint8, []byte, uint64, string, bool, error) { +) (bool, []byte, uint8, []byte, uint64, string, error) { cli.assetsL.Lock() r, ok := cli.assets[asset] cli.assetsL.Unlock() @@ -106,9 +106,9 @@ func (cli *JSONRPCClient) Asset( // We use string parsing here because the JSON-RPC library we use may not // allows us to perform errors.Is. case err != nil && strings.Contains(err.Error(), ErrAssetNotFound.Error()): - return false, nil, 0, nil, 0, "", false, nil + return false, nil, 0, nil, 0, "", nil case err != nil: - return false, nil, 0, nil, 0, "", false, err + return false, nil, 0, nil, 0, "", err } cli.assetsL.Lock() cli.assets[asset] = resp @@ -180,7 +180,7 @@ func (cli *JSONRPCClient) WaitForBalance( asset ids.ID, min uint64, ) error { - exists, symbol, decimals, _, _, _, _, err := cli.Asset(ctx, asset, true) + exists, symbol, decimals, _, _, _, err := cli.Asset(ctx, asset, true) if err != nil { return err } diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index 35e1b9c64e..70e192ae5a 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -294,7 +294,7 @@ func GetAssetFromState( ctx context.Context, f ReadState, asset ids.ID, -) (bool, []byte, uint8, []byte, uint64, codec.Address, bool, error) { +) (bool, []byte, uint8, []byte, uint64, codec.Address, error) { values, errs := f(ctx, [][]byte{AssetKey(asset)}) return innerGetAsset(values[0], errs[0]) } @@ -303,7 +303,7 @@ func GetAsset( ctx context.Context, im state.Immutable, asset ids.ID, -) (bool, []byte, uint8, []byte, uint64, codec.Address, bool, error) { +) (bool, []byte, uint8, []byte, uint64, codec.Address, error) { k := AssetKey(asset) return innerGetAsset(im.GetValue(ctx, k)) } @@ -311,12 +311,12 @@ func GetAsset( func innerGetAsset( v []byte, err error, -) (bool, []byte, uint8, []byte, uint64, codec.Address, bool, error) { +) (bool, []byte, uint8, []byte, uint64, codec.Address, error) { if errors.Is(err, database.ErrNotFound) { - return false, nil, 0, nil, 0, codec.EmptyAddress, false, nil + return false, nil, 0, nil, 0, codec.EmptyAddress, nil } if err != nil { - return false, nil, 0, nil, 0, codec.EmptyAddress, false, err + return false, nil, 0, nil, 0, codec.EmptyAddress, err } symbolLen := binary.BigEndian.Uint16(v) symbol := v[consts.Uint16Len : consts.Uint16Len+symbolLen] diff --git a/examples/tokenvm/tests/integration/integration_test.go b/examples/tokenvm/tests/integration/integration_test.go index 5a02a9f8ee..0369048947 100644 --- a/examples/tokenvm/tests/integration/integration_test.go +++ b/examples/tokenvm/tests/integration/integration_test.go @@ -334,7 +334,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, transferTx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: rsender2, Value: 100_000, // must be more than StateLockup @@ -367,7 +366,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: 0, MaxFee: 1000, }, - nil, &actions.Transfer{ To: rsender2, Value: 110, @@ -434,19 +432,19 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { // read: 2 keys reads, 1 had 0 chunks // allocate: 1 key created // write: 1 key modified, 1 key new - transferTxConsumed := fees.Dimensions{227, 7, 12, 25, 26} + transferTxConsumed := fees.Dimensions{223, 7, 12, 25, 26} gomega.Ω(results[0].Consumed).Should(gomega.Equal(transferTxConsumed)) // Fee explanation // // Multiply all unit consumption by 1 and sum - gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(297))) + gomega.Ω(results[0].Fee).Should(gomega.Equal(uint64(293))) }) ginkgo.By("ensure balance is updated", func() { balance, err := instances[1].tcli.Balance(context.Background(), sender, ids.Empty) gomega.Ω(err).To(gomega.BeNil()) - gomega.Ω(balance).To(gomega.Equal(uint64(9899703))) + gomega.Ω(balance).To(gomega.Equal(uint64(9899707))) balance2, err := instances[1].tcli.Balance(context.Background(), sender2, ids.Empty) gomega.Ω(err).To(gomega.BeNil()) gomega.Ω(balance2).To(gomega.Equal(uint64(100000))) @@ -460,7 +458,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: rsender2, Value: 101, @@ -490,7 +487,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: rsender2, Value: 200, @@ -505,7 +501,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: rsender2, Value: 201, @@ -535,7 +530,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[1].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: rsender2, Value: 203, @@ -637,7 +631,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, transfer, factory, ) @@ -686,7 +679,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { _, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, transfer, factory, ) @@ -730,7 +722,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: auth.NewED25519Address(other.PublicKey()), Value: 10, @@ -756,7 +747,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1001, }, - nil, &actions.Transfer{ To: auth.NewED25519Address(other.PublicKey()), Value: 10, @@ -789,7 +779,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: auth.NewED25519Address(other.PublicKey()), Asset: assetID, @@ -807,7 +796,7 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { gomega.Ω(string(result.Output)). Should(gomega.ContainSubstring("asset missing")) - exists, _, _, _, _, _, _, err := instances[0].tcli.Asset(context.TODO(), assetID, false) + exists, _, _, _, _, _, err := instances[0].tcli.Asset(context.TODO(), assetID, false) gomega.Ω(err).Should(gomega.BeNil()) gomega.Ω(exists).Should(gomega.BeFalse()) }) @@ -819,7 +808,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1001, }, - nil, &actions.CreateAsset{ Symbol: []byte("s0"), Decimals: 0, @@ -850,7 +838,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1001, }, - nil, &actions.CreateAsset{ Symbol: nil, Decimals: 0, @@ -881,7 +868,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1000, }, - nil, &actions.CreateAsset{ Symbol: []byte("s0"), Decimals: 0, @@ -911,7 +897,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateAsset{ Symbol: asset1Symbol, Decimals: asset1Decimals, @@ -947,7 +932,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: rsender2, Asset: asset1ID, @@ -987,7 +971,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: auth.NewED25519Address(other.PublicKey()), Asset: asset1ID, @@ -1021,7 +1004,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.BurnAsset{ Asset: asset1ID, Value: 5, @@ -1058,7 +1040,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.BurnAsset{ Asset: asset1ID, Value: 10, @@ -1094,7 +1075,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1000, }, - nil, &actions.MintAsset{ To: auth.NewED25519Address(other.PublicKey()), Asset: asset1ID, @@ -1123,7 +1103,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: rsender2, Asset: asset1ID, @@ -1167,7 +1146,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { Timestamp: hutils.UnixRMilli(-1, 5*consts.MillisecondsPerSecond), MaxFee: 1000, }, - nil, &actions.MintAsset{ To: auth.NewED25519Address(other.PublicKey()), Value: 10, @@ -1196,7 +1174,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateAsset{ Symbol: asset2Symbol, Decimals: asset2Decimals, @@ -1215,7 +1192,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: rsender, Asset: asset2ID, @@ -1241,7 +1217,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateAsset{ Symbol: asset3Symbol, Decimals: asset3Decimals, @@ -1260,7 +1235,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err = instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.MintAsset{ To: rsender2, Asset: asset3ID, @@ -1286,7 +1260,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateOrder{ In: asset3ID, InTick: 1, @@ -1324,7 +1297,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateOrder{ In: asset2ID, InTick: 4, @@ -1351,7 +1323,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateOrder{ In: asset2ID, InTick: 4, @@ -1389,7 +1360,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateOrder{ In: asset2ID, InTick: 5, @@ -1422,7 +1392,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.FillOrder{ Order: order.ID, Owner: owner, @@ -1455,7 +1424,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.FillOrder{ Order: order.ID, Owner: owner, @@ -1488,7 +1456,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.FillOrder{ Order: order.ID, Owner: owner, @@ -1535,7 +1502,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CloseOrder{ Order: order.ID, Out: asset3ID, @@ -1563,7 +1529,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CloseOrder{ Order: order.ID, Out: asset3ID, @@ -1602,7 +1567,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, tx, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.CreateOrder{ In: asset2ID, InTick: 2, @@ -1646,7 +1610,6 @@ var _ = ginkgo.Describe("[Tx Processing]", func() { submit, _, _, err := instances[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.FillOrder{ Order: order.ID, Owner: owner, From 1cb818063e23095e6fd5759003ffc4b22de92982 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 16:23:16 -0400 Subject: [PATCH 10/29] fix tokenvm e2e and load --- examples/tokenvm/tests/e2e/e2e_test.go | 3 --- examples/tokenvm/tests/load/load_test.go | 1 - 2 files changed, 4 deletions(-) diff --git a/examples/tokenvm/tests/e2e/e2e_test.go b/examples/tokenvm/tests/e2e/e2e_test.go index 5b73854c1a..60532dea6d 100644 --- a/examples/tokenvm/tests/e2e/e2e_test.go +++ b/examples/tokenvm/tests/e2e/e2e_test.go @@ -526,7 +526,6 @@ var _ = ginkgo.Describe("[Test]", func() { submit, tx, maxFee, err := instancesA[0].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: aother, Value: sendAmount, @@ -792,7 +791,6 @@ func generateBlocks( submit, _, _, err := instances[cumulativeTxs%len(instances)].cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: auth.NewED25519Address(other.PublicKey()), Value: 1, @@ -860,7 +858,6 @@ func acceptTransaction(cli *rpc.JSONRPCClient, tcli *trpc.JSONRPCClient) { submit, tx, maxFee, err := cli.GenerateTransaction( context.Background(), parser, - nil, &actions.Transfer{ To: auth.NewED25519Address(other.PublicKey()), Value: sendAmount, diff --git a/examples/tokenvm/tests/load/load_test.go b/examples/tokenvm/tests/load/load_test.go index 23aefec630..c4278ef280 100644 --- a/examples/tokenvm/tests/load/load_test.go +++ b/examples/tokenvm/tests/load/load_test.go @@ -533,7 +533,6 @@ func issueSimpleTx( ChainID: i.chainID, MaxFee: maxFee, }, - nil, &actions.Transfer{ To: to, Value: amount, From a9fb3367a44e4adb6adaf6aa4ea7e5badb9b3074 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 16:46:50 -0400 Subject: [PATCH 11/29] fix tokenvm --- chain/block.go | 6 +- chain/builder.go | 2 +- chain/consts.go | 8 +-- chain/transaction.go | 14 ++--- cli/spam.go | 8 +-- examples/morpheusvm/genesis/genesis.go | 2 +- examples/morpheusvm/storage/storage.go | 8 +-- examples/tokenvm/actions/outputs.go | 58 +++++++++---------- examples/tokenvm/cmd/token-cli/cmd/action.go | 21 +++---- examples/tokenvm/cmd/token-cli/cmd/handler.go | 1 - .../tokenvm/cmd/token-cli/cmd/resolutions.go | 26 ++++----- examples/tokenvm/cmd/token-cli/cmd/root.go | 3 - examples/tokenvm/cmd/token-cli/cmd/spam.go | 2 +- .../cmd/token-faucet/manager/manager.go | 2 +- .../cmd/token-wallet/backend/backend.go | 52 ++++++++--------- examples/tokenvm/genesis/genesis.go | 2 +- examples/tokenvm/storage/storage.go | 14 ++--- .../tests/integration/integration_test.go | 37 ------------ .../cmd/simulator/vm/genesis/genesis.go | 2 +- .../cmd/simulator/vm/storage/storage.go | 8 +-- 20 files changed, 114 insertions(+), 162 deletions(-) diff --git a/chain/block.go b/chain/block.go index d915fa2e5f..1a062b7f1e 100644 --- a/chain/block.go +++ b/chain/block.go @@ -50,7 +50,7 @@ type StatefulBlock struct { // or [Verify], which reduces the amount of time we are // blocking the consensus engine from voting on the block, // starting the verification of another block, etc. - StateRoot ids.ID `json:"stateRoot"` + StateRoot ids.ID `json:"stateRoot"` size int @@ -100,8 +100,8 @@ type StatelessBlock struct { bytes []byte txsSet set.Set[ids.ID] - bctx *block.Context - vdrState validators.State + bctx *block.Context + vdrState validators.State results []*Result feeManager *fees.Manager diff --git a/chain/builder.go b/chain/builder.go index 9d4f1120f2..ea12487d6f 100644 --- a/chain/builder.go +++ b/chain/builder.go @@ -129,7 +129,7 @@ func BuildBlock( txsAttempted = 0 results = []*Result{} - sm = vm.StateManager() + sm = vm.StateManager() // prepareStreamLock ensures we don't overwrite stream prefetching spawned // asynchronously. diff --git a/chain/consts.go b/chain/consts.go index 4dba91a246..568f2a3f14 100644 --- a/chain/consts.go +++ b/chain/consts.go @@ -16,10 +16,10 @@ const ( // This value should be (much) less than the value of [ProposerWindow], otherwise honest // nodes may not build during their allocated window to avoid increasing the skew of the // chain time. - FutureBound = 1 * time.Second - HeightKeyChunks = 1 - TimestampKeyChunks = 1 - FeeKeyChunks = 8 // 96 (per dimension) * 5 (num dimensions) + FutureBound = 1 * time.Second + HeightKeyChunks = 1 + TimestampKeyChunks = 1 + FeeKeyChunks = 8 // 96 (per dimension) * 5 (num dimensions) // MaxKeyDependencies must be greater than the maximum number of key dependencies // any single task could have when executing a task. diff --git a/chain/transaction.go b/chain/transaction.go index 057f7f02a7..5dafb4cb04 100644 --- a/chain/transaction.go +++ b/chain/transaction.go @@ -27,23 +27,23 @@ var ( ) type Transaction struct { - Base *Base `json:"base"` + Base *Base `json:"base"` // TODO: turn [Action] into an array (#335) Action Action `json:"action"` Auth Auth `json:"auth"` - digest []byte - bytes []byte - size int - id ids.ID + digest []byte + bytes []byte + size int + id ids.ID stateKeys state.Keys } func NewTx(base *Base, act Action) *Transaction { return &Transaction{ - Base: base, - Action: act, + Base: base, + Action: act, } } diff --git a/cli/spam.go b/cli/spam.go index 90afdbba82..16978d538f 100644 --- a/cli/spam.go +++ b/cli/spam.go @@ -111,7 +111,7 @@ func (h *Handler) Spam( return err } action := getTransfer(keys[0].Address, 0) - maxUnits, err := chain.EstimateMaxUnits(parser.Rules(time.Now().UnixMilli()), action, factory, nil) + maxUnits, err := chain.EstimateMaxUnits(parser.Rules(time.Now().UnixMilli()), action, factory) if err != nil { return err } @@ -167,7 +167,7 @@ func (h *Handler) Spam( accounts[i] = pk // Send funds - _, tx, err := cli.GenerateTransactionManual(parser, nil, getTransfer(pk.Address, distAmount), factory, feePerTx) + _, tx, err := cli.GenerateTransactionManual(parser, getTransfer(pk.Address, distAmount), factory, feePerTx) if err != nil { return err } @@ -316,7 +316,7 @@ func (h *Handler) Spam( if maxFee != nil { fee = *maxFee } - _, tx, err := issuer.c.GenerateTransactionManual(parser, nil, action, factory, fee, tm) + _, tx, err := issuer.c.GenerateTransactionManual(parser, action, factory, fee, tm) if err != nil { utils.Outf("{{orange}}failed to generate tx:{{/}} %v\n", err) continue @@ -424,7 +424,7 @@ func (h *Handler) Spam( if err != nil { return err } - _, tx, err := cli.GenerateTransactionManual(parser, nil, getTransfer(key.Address, returnAmt), f, feePerTx) + _, tx, err := cli.GenerateTransactionManual(parser, getTransfer(key.Address, returnAmt), f, feePerTx) if err != nil { return err } diff --git a/examples/morpheusvm/genesis/genesis.go b/examples/morpheusvm/genesis/genesis.go index 6ebf7b0361..bfe536e32b 100644 --- a/examples/morpheusvm/genesis/genesis.go +++ b/examples/morpheusvm/genesis/genesis.go @@ -77,7 +77,7 @@ func Default() *Genesis { ValidityWindow: 60 * hconsts.MillisecondsPerSecond, // ms // Tx Fee Compute Parameters - BaseComputeUnits: 1, + BaseComputeUnits: 1, // Tx Fee Storage Parameters // diff --git a/examples/morpheusvm/storage/storage.go b/examples/morpheusvm/storage/storage.go index c4763eeb51..7db1a7eee6 100644 --- a/examples/morpheusvm/storage/storage.go +++ b/examples/morpheusvm/storage/storage.go @@ -40,10 +40,10 @@ const ( txPrefix = 0x0 // stateDB - balancePrefix = 0x0 - heightPrefix = 0x1 - timestampPrefix = 0x2 - feePrefix = 0x3 + balancePrefix = 0x0 + heightPrefix = 0x1 + timestampPrefix = 0x2 + feePrefix = 0x3 ) const BalanceChunks uint16 = 1 diff --git a/examples/tokenvm/actions/outputs.go b/examples/tokenvm/actions/outputs.go index 424443e823..1c4fd03622 100644 --- a/examples/tokenvm/actions/outputs.go +++ b/examples/tokenvm/actions/outputs.go @@ -4,33 +4,33 @@ package actions var ( - OutputValueZero = []byte("value is zero") - OutputMemoTooLarge = []byte("memo is too large") - OutputAssetIsNative = []byte("cannot mint native asset") - OutputAssetAlreadyExists = []byte("asset already exists") - OutputAssetMissing = []byte("asset missing") - OutputInTickZero = []byte("in rate is zero") - OutputOutTickZero = []byte("out rate is zero") - OutputSupplyZero = []byte("supply is zero") - OutputSupplyMisaligned = []byte("supply is misaligned") - OutputOrderMissing = []byte("order is missing") - OutputUnauthorized = []byte("unauthorized") - OutputWrongIn = []byte("wrong in asset") - OutputWrongOut = []byte("wrong out asset") - OutputWrongOwner = []byte("wrong owner") - OutputInsufficientInput = []byte("insufficient input") - OutputInsufficientOutput = []byte("insufficient output") - OutputValueMisaligned = []byte("value is misaligned") - OutputSymbolEmpty = []byte("symbol is empty") - OutputSymbolIncorrect = []byte("symbol is incorrect") - OutputSymbolTooLarge = []byte("symbol is too large") - OutputDecimalsIncorrect = []byte("decimal is incorrect") - OutputDecimalsTooLarge = []byte("decimal is too large") - OutputMetadataEmpty = []byte("metadata is empty") - OutputMetadataTooLarge = []byte("metadata is too large") - OutputSameInOut = []byte("same asset used for in and out") - OutputAnycast = []byte("anycast output") - OutputWrongDestination = []byte("wrong destination") - OutputMustFill = []byte("must fill request") - OutputInvalidDestination = []byte("invalid destination") + OutputValueZero = []byte("value is zero") + OutputMemoTooLarge = []byte("memo is too large") + OutputAssetIsNative = []byte("cannot mint native asset") + OutputAssetAlreadyExists = []byte("asset already exists") + OutputAssetMissing = []byte("asset missing") + OutputInTickZero = []byte("in rate is zero") + OutputOutTickZero = []byte("out rate is zero") + OutputSupplyZero = []byte("supply is zero") + OutputSupplyMisaligned = []byte("supply is misaligned") + OutputOrderMissing = []byte("order is missing") + OutputUnauthorized = []byte("unauthorized") + OutputWrongIn = []byte("wrong in asset") + OutputWrongOut = []byte("wrong out asset") + OutputWrongOwner = []byte("wrong owner") + OutputInsufficientInput = []byte("insufficient input") + OutputInsufficientOutput = []byte("insufficient output") + OutputValueMisaligned = []byte("value is misaligned") + OutputSymbolEmpty = []byte("symbol is empty") + OutputSymbolIncorrect = []byte("symbol is incorrect") + OutputSymbolTooLarge = []byte("symbol is too large") + OutputDecimalsIncorrect = []byte("decimal is incorrect") + OutputDecimalsTooLarge = []byte("decimal is too large") + OutputMetadataEmpty = []byte("metadata is empty") + OutputMetadataTooLarge = []byte("metadata is too large") + OutputSameInOut = []byte("same asset used for in and out") + OutputAnycast = []byte("anycast output") + OutputWrongDestination = []byte("wrong destination") + OutputMustFill = []byte("must fill request") + OutputInvalidDestination = []byte("invalid destination") ) diff --git a/examples/tokenvm/cmd/token-cli/cmd/action.go b/examples/tokenvm/cmd/token-cli/cmd/action.go index eeb991e602..1bacc811c5 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/action.go +++ b/examples/tokenvm/cmd/token-cli/cmd/action.go @@ -6,20 +6,13 @@ package cmd import ( "context" - "errors" - "time" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/consts" "github.com/ava-labs/hypersdk/examples/tokenvm/actions" frpc "github.com/ava-labs/hypersdk/examples/tokenvm/cmd/token-faucet/rpc" tconsts "github.com/ava-labs/hypersdk/examples/tokenvm/consts" - trpc "github.com/ava-labs/hypersdk/examples/tokenvm/rpc" - "github.com/ava-labs/hypersdk/pubsub" - "github.com/ava-labs/hypersdk/rpc" hutils "github.com/ava-labs/hypersdk/utils" "github.com/spf13/cobra" ) @@ -76,7 +69,7 @@ var fundFaucetCmd = &cobra.Command{ if err != nil { return err } - if _, _, err = sendAndWait(ctx, nil, &actions.Transfer{ + if _, err = sendAndWait(ctx, &actions.Transfer{ To: addr, Asset: ids.Empty, Value: amount, @@ -126,7 +119,7 @@ var transferCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.Transfer{ + _, err = sendAndWait(ctx, &actions.Transfer{ To: recipient, Asset: assetID, Value: amount, @@ -169,7 +162,7 @@ var createAssetCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.CreateAsset{ + _, err = sendAndWait(ctx, &actions.CreateAsset{ Symbol: []byte(symbol), Decimals: uint8(decimals), // already constrain above to prevent overflow Metadata: []byte(metadata), @@ -233,7 +226,7 @@ var mintAssetCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.MintAsset{ + _, err = sendAndWait(ctx, &actions.MintAsset{ Asset: assetID, To: recipient, Value: amount, @@ -270,7 +263,7 @@ var closeOrderCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.CloseOrder{ + _, err = sendAndWait(ctx, &actions.CloseOrder{ Order: orderID, Out: outAssetID, }, cli, scli, tcli, factory, true) @@ -356,7 +349,7 @@ var createOrderCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.CreateOrder{ + _, err = sendAndWait(ctx, &actions.CreateOrder{ In: inAssetID, InTick: inTick, Out: outAssetID, @@ -473,7 +466,7 @@ var fillOrderCmd = &cobra.Command{ if err != nil { return err } - _, _, err = sendAndWait(ctx, nil, &actions.FillOrder{ + _, err = sendAndWait(ctx, &actions.FillOrder{ Order: order.ID, Owner: owner, In: inAssetID, diff --git a/examples/tokenvm/cmd/token-cli/cmd/handler.go b/examples/tokenvm/cmd/token-cli/cmd/handler.go index 2c303adc82..697189d4be 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/handler.go +++ b/examples/tokenvm/cmd/token-cli/cmd/handler.go @@ -10,7 +10,6 @@ import ( "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/cli" "github.com/ava-labs/hypersdk/codec" - hconsts "github.com/ava-labs/hypersdk/consts" "github.com/ava-labs/hypersdk/crypto/ed25519" "github.com/ava-labs/hypersdk/examples/tokenvm/auth" "github.com/ava-labs/hypersdk/examples/tokenvm/consts" diff --git a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go index 54dce659f6..c3939a3d24 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go +++ b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go @@ -23,27 +23,27 @@ import ( func sendAndWait( ctx context.Context, action chain.Action, cli *rpc.JSONRPCClient, scli *rpc.WebSocketClient, tcli *trpc.JSONRPCClient, factory chain.AuthFactory, printStatus bool, -) (bool, ids.ID, error) { +) (ids.ID, error) { parser, err := tcli.Parser(ctx) if err != nil { - return false, ids.Empty, err + return ids.Empty, err } _, tx, _, err := cli.GenerateTransaction(ctx, parser, action, factory) if err != nil { - return false, ids.Empty, err + return ids.Empty, err } if err := scli.RegisterTx(tx); err != nil { - return false, ids.Empty, err + return ids.Empty, err } var res *chain.Result for { txID, dErr, result, err := scli.ListenTx(ctx) if dErr != nil { - return false, ids.Empty, dErr + return ids.Empty, dErr } if err != nil { - return false, ids.Empty, err + return ids.Empty, err } if txID == tx.ID() { res = result @@ -54,7 +54,7 @@ func sendAndWait( if printStatus { handler.Root().PrintStatus(tx.ID(), res.Success) } - return res.Success, tx.ID(), nil + return tx.ID(), nil } func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result) { @@ -67,7 +67,7 @@ func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result case *actions.CreateAsset: summaryStr = fmt.Sprintf("assetID: %s symbol: %s decimals: %d metadata: %s", tx.ID(), action.Symbol, action.Decimals, action.Metadata) case *actions.MintAsset: - _, symbol, decimals, _, _, _, _, err := c.Asset(context.TODO(), action.Asset, true) + _, symbol, decimals, _, _, _, err := c.Asset(context.TODO(), action.Asset, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return @@ -78,7 +78,7 @@ func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result summaryStr = fmt.Sprintf("%d %s -> 🔥", action.Value, action.Asset) case *actions.Transfer: - _, symbol, decimals, _, _, _, _, err := c.Asset(context.TODO(), action.Asset, true) + _, symbol, decimals, _, _, _, err := c.Asset(context.TODO(), action.Asset, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return @@ -90,13 +90,13 @@ func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result } case *actions.CreateOrder: - _, inSymbol, inDecimals, _, _, _, _, err := c.Asset(context.TODO(), action.In, true) + _, inSymbol, inDecimals, _, _, _, err := c.Asset(context.TODO(), action.In, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return } inTickStr := utils.FormatBalance(action.InTick, inDecimals) - _, outSymbol, outDecimals, _, _, _, _, err := c.Asset(context.TODO(), action.Out, true) + _, outSymbol, outDecimals, _, _, _, err := c.Asset(context.TODO(), action.Out, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return @@ -106,13 +106,13 @@ func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result summaryStr = fmt.Sprintf("%s %s -> %s %s (supply: %s %s)", inTickStr, inSymbol, outTickStr, outSymbol, supplyStr, outSymbol) case *actions.FillOrder: or, _ := actions.UnmarshalOrderResult(result.Output) - _, inSymbol, inDecimals, _, _, _, _, err := c.Asset(context.TODO(), action.In, true) + _, inSymbol, inDecimals, _, _, _, err := c.Asset(context.TODO(), action.In, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return } inAmtStr := utils.FormatBalance(or.In, inDecimals) - _, outSymbol, outDecimals, _, _, _, _, err := c.Asset(context.TODO(), action.Out, true) + _, outSymbol, outDecimals, _, _, _, err := c.Asset(context.TODO(), action.Out, true) if err != nil { utils.Outf("{{red}}could not fetch asset info:{{/}} %v", err) return diff --git a/examples/tokenvm/cmd/token-cli/cmd/root.go b/examples/tokenvm/cmd/token-cli/cmd/root.go index e49a5e3822..e55fd77b53 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/root.go +++ b/examples/tokenvm/cmd/token-cli/cmd/root.go @@ -162,9 +162,6 @@ func init() { createOrderCmd, fillOrderCmd, closeOrderCmd, - - importAssetCmd, - exportAssetCmd, ) // spam diff --git a/examples/tokenvm/cmd/token-cli/cmd/spam.go b/examples/tokenvm/cmd/token-cli/cmd/spam.go index 3360ecd99a..647c22118f 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/spam.go +++ b/examples/tokenvm/cmd/token-cli/cmd/spam.go @@ -87,7 +87,7 @@ var runSpamCmd = &cobra.Command{ }, func(cli *rpc.JSONRPCClient, priv *cli.PrivateKey) func(context.Context, uint64) error { // submitDummy return func(ictx context.Context, count uint64) error { - _, _, err := sendAndWait(ictx, nil, &actions.Transfer{ + _, err := sendAndWait(ictx, &actions.Transfer{ To: priv.Address, Value: count, // prevent duplicate txs }, cli, sclient, tclient, auth.NewED25519Factory(ed25519.PrivateKey(priv.Bytes)), false) diff --git a/examples/tokenvm/cmd/token-faucet/manager/manager.go b/examples/tokenvm/cmd/token-faucet/manager/manager.go index b445faad52..0c6586f518 100644 --- a/examples/tokenvm/cmd/token-faucet/manager/manager.go +++ b/examples/tokenvm/cmd/token-faucet/manager/manager.go @@ -122,7 +122,7 @@ func (m *Manager) sendFunds(ctx context.Context, destination codec.Address, amou if err != nil { return ids.Empty, 0, err } - submit, tx, maxFee, err := m.cli.GenerateTransaction(ctx, parser, nil, &actions.Transfer{ + submit, tx, maxFee, err := m.cli.GenerateTransaction(ctx, parser, &actions.Transfer{ To: destination, Asset: ids.Empty, Value: amount, diff --git a/examples/tokenvm/cmd/token-wallet/backend/backend.go b/examples/tokenvm/cmd/token-wallet/backend/backend.go index 2b8e8596f1..61c1fa3608 100644 --- a/examples/tokenvm/cmd/token-wallet/backend/backend.go +++ b/examples/tokenvm/cmd/token-wallet/backend/backend.go @@ -226,7 +226,7 @@ func (b *Backend) collectBlocks() { continue } - _, symbol, decimals, _, _, owner, _, err := b.tcli.Asset(b.ctx, action.Asset, true) + _, symbol, decimals, _, _, owner, err := b.tcli.Asset(b.ctx, action.Asset, true) if err != nil { b.fatal(err) return @@ -309,7 +309,7 @@ func (b *Backend) collectBlocks() { continue } - _, symbol, decimals, _, _, owner, _, err := b.tcli.Asset(b.ctx, action.Asset, true) + _, symbol, decimals, _, _, owner, err := b.tcli.Asset(b.ctx, action.Asset, true) if err != nil { b.fatal(err) return @@ -361,12 +361,12 @@ func (b *Backend) collectBlocks() { continue } - _, inSymbol, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, action.In, true) + _, inSymbol, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, action.In, true) if err != nil { b.fatal(err) return } - _, outSymbol, outDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, action.Out, true) + _, outSymbol, outDecimals, _, _, _, err := b.tcli.Asset(b.ctx, action.Out, true) if err != nil { b.fatal(err) return @@ -402,12 +402,12 @@ func (b *Backend) collectBlocks() { continue } - _, inSymbol, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, action.In, true) + _, inSymbol, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, action.In, true) if err != nil { b.fatal(err) return } - _, outSymbol, outDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, action.Out, true) + _, outSymbol, outDecimals, _, _, _, err := b.tcli.Asset(b.ctx, action.Out, true) if err != nil { b.fatal(err) return @@ -608,7 +608,7 @@ func (b *Backend) GetMyAssets() []*AssetInfo { if !owned[i] { continue } - _, symbol, decimals, metadata, supply, owner, _, err := b.tcli.Asset(b.ctx, asset, false) + _, symbol, decimals, metadata, supply, owner, err := b.tcli.Asset(b.ctx, asset, false) if err != nil { b.fatal(err) return nil @@ -639,7 +639,7 @@ func (b *Backend) CreateAsset(symbol string, decimals string, metadata string) e if err != nil { return err } - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.CreateAsset{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.CreateAsset{ Symbol: []byte(symbol), Decimals: uint8(udecimals), Metadata: []byte(metadata), @@ -674,7 +674,7 @@ func (b *Backend) MintAsset(asset string, address string, amount string) error { if err != nil { return err } - _, _, decimals, _, _, _, _, err := b.tcli.Asset(b.ctx, assetID, true) + _, _, decimals, _, _, _, err := b.tcli.Asset(b.ctx, assetID, true) if err != nil { return err } @@ -694,7 +694,7 @@ func (b *Backend) MintAsset(asset string, address string, amount string) error { } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.MintAsset{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.MintAsset{ To: to, Asset: assetID, Value: value, @@ -729,7 +729,7 @@ func (b *Backend) Transfer(asset string, address string, amount string, memo str if err != nil { return err } - _, symbol, decimals, _, _, _, _, err := b.tcli.Asset(b.ctx, assetID, true) + _, symbol, decimals, _, _, _, err := b.tcli.Asset(b.ctx, assetID, true) if err != nil { return err } @@ -758,7 +758,7 @@ func (b *Backend) Transfer(asset string, address string, amount string, memo str } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.Transfer{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.Transfer{ To: to, Asset: assetID, Value: value, @@ -805,7 +805,7 @@ func (b *Backend) GetBalance() ([]*BalanceInfo, error) { } balances := []*BalanceInfo{} for _, asset := range assets { - _, symbol, decimals, _, _, _, _, err := b.tcli.Asset(b.ctx, asset, true) + _, symbol, decimals, _, _, _, err := b.tcli.Asset(b.ctx, asset, true) if err != nil { return nil, err } @@ -937,7 +937,7 @@ func (b *Backend) GetAllAssets() []*AssetInfo { } assets := []*AssetInfo{} for _, asset := range arr { - _, symbol, decimals, metadata, supply, owner, _, err := b.tcli.Asset(b.ctx, asset, false) + _, symbol, decimals, metadata, supply, owner, err := b.tcli.Asset(b.ctx, asset, false) if err != nil { b.fatal(err) return nil @@ -968,7 +968,7 @@ func (b *Backend) AddAsset(asset string) error { if hasAsset { return nil } - exists, _, _, _, _, owner, _, err := b.tcli.Asset(b.ctx, assetID, true) + exists, _, _, _, _, owner, err := b.tcli.Asset(b.ctx, assetID, true) if err != nil { return err } @@ -993,12 +993,12 @@ func (b *Backend) GetMyOrders() ([]*Order, error) { continue } inID := order.InAsset - _, inSymbol, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) + _, inSymbol, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) if err != nil { return nil, err } outID := order.OutAsset - _, outSymbol, outDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) + _, outSymbol, outDecimals, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) if err != nil { return nil, err } @@ -1035,7 +1035,7 @@ func (b *Backend) GetOrders(pair string) ([]*Order, error) { if err != nil { return nil, err } - _, inSymbol, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) + _, inSymbol, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) if err != nil { return nil, err } @@ -1044,7 +1044,7 @@ func (b *Backend) GetOrders(pair string) ([]*Order, error) { if err != nil { return nil, err } - _, outSymbol, outDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) + _, outSymbol, outDecimals, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) if err != nil { return nil, err } @@ -1080,11 +1080,11 @@ func (b *Backend) CreateOrder(assetIn string, inTick string, assetOut string, ou if err != nil { return err } - _, _, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) + _, _, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) if err != nil { return err } - _, outSymbol, outDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) + _, outSymbol, outDecimals, _, _, _, err := b.tcli.Asset(b.ctx, outID, true) if err != nil { return err } @@ -1112,7 +1112,7 @@ func (b *Backend) CreateOrder(assetIn string, inTick string, assetOut string, ou } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.CreateOrder{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.CreateOrder{ In: inID, InTick: iTick, Out: outID, @@ -1171,7 +1171,7 @@ func (b *Backend) FillOrder(orderID string, orderOwner string, assetIn string, i if err != nil { return err } - _, inSymbol, inDecimals, _, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) + _, inSymbol, inDecimals, _, _, _, err := b.tcli.Asset(b.ctx, inID, true) if err != nil { return err } @@ -1198,7 +1198,7 @@ func (b *Backend) FillOrder(orderID string, orderOwner string, assetIn string, i } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.FillOrder{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.FillOrder{ Order: oID, Owner: owner, In: inID, @@ -1255,7 +1255,7 @@ func (b *Backend) CloseOrder(orderID string, assetOut string) error { } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.CloseOrder{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.CloseOrder{ Order: oID, Out: outID, }, b.factory) @@ -1395,7 +1395,7 @@ func (b *Backend) Message(message string, url string) error { } // Generate transaction - _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, nil, &actions.Transfer{ + _, tx, maxFee, err := b.cli.GenerateTransaction(b.ctx, b.parser, &actions.Transfer{ To: recipientAddr, Asset: ids.Empty, Value: fee, diff --git a/examples/tokenvm/genesis/genesis.go b/examples/tokenvm/genesis/genesis.go index 0832cb5532..4632bd332f 100644 --- a/examples/tokenvm/genesis/genesis.go +++ b/examples/tokenvm/genesis/genesis.go @@ -78,7 +78,7 @@ func Default() *Genesis { ValidityWindow: 60 * hconsts.MillisecondsPerSecond, // ms // Tx Fee Compute Parameters - BaseComputeUnits: 1, + BaseComputeUnits: 1, // Tx Fee Storage Parameters // diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index 70e192ae5a..4e79c005c5 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -44,13 +44,13 @@ const ( txPrefix = 0x0 // stateDB - balancePrefix = 0x0 - assetPrefix = 0x1 - orderPrefix = 0x2 - loanPrefix = 0x3 - heightPrefix = 0x4 - timestampPrefix = 0x5 - feePrefix = 0x6 + balancePrefix = 0x0 + assetPrefix = 0x1 + orderPrefix = 0x2 + loanPrefix = 0x3 + heightPrefix = 0x4 + timestampPrefix = 0x5 + feePrefix = 0x6 ) const ( diff --git a/examples/tokenvm/tests/integration/integration_test.go b/examples/tokenvm/tests/integration/integration_test.go index 0369048947..34e3a10be3 100644 --- a/examples/tokenvm/tests/integration/integration_test.go +++ b/examples/tokenvm/tests/integration/integration_test.go @@ -22,7 +22,6 @@ import ( "github.com/ava-labs/avalanchego/snow/choices" "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/logging" @@ -1684,42 +1683,6 @@ func expectBlk(i instance) func(bool) []*chain.Result { } } -// TODO: unify with expectBlk -func expectBlkWithContext(i instance) func(bool) []*chain.Result { - ctx := context.TODO() - - // manually signal ready - gomega.Ω(i.vm.Builder().Force(ctx)).To(gomega.BeNil()) - // manually ack ready sig as in engine - <-i.toEngine - - bctx := &block.Context{PChainHeight: 1} - blk, err := i.vm.BuildBlockWithContext(ctx, bctx) - gomega.Ω(err).To(gomega.BeNil()) - gomega.Ω(blk).To(gomega.Not(gomega.BeNil())) - cblk := blk.(block.WithVerifyContext) - - gomega.Ω(cblk.VerifyWithContext(ctx, bctx)).To(gomega.BeNil()) - gomega.Ω(blk.Status()).To(gomega.Equal(choices.Processing)) - - err = i.vm.SetPreference(ctx, blk.ID()) - gomega.Ω(err).To(gomega.BeNil()) - - return func(add bool) []*chain.Result { - gomega.Ω(blk.Accept(ctx)).To(gomega.BeNil()) - gomega.Ω(blk.Status()).To(gomega.Equal(choices.Accepted)) - - if add { - blocks = append(blocks, blk) - } - - lastAccepted, err := i.vm.LastAccepted(ctx) - gomega.Ω(err).To(gomega.BeNil()) - gomega.Ω(lastAccepted).To(gomega.Equal(blk.ID())) - return blk.(*chain.StatelessBlock).Results() - } -} - var _ common.AppSender = &appSender{} type appSender struct { diff --git a/x/programs/cmd/simulator/vm/genesis/genesis.go b/x/programs/cmd/simulator/vm/genesis/genesis.go index 24e9ed7162..958ca59817 100644 --- a/x/programs/cmd/simulator/vm/genesis/genesis.go +++ b/x/programs/cmd/simulator/vm/genesis/genesis.go @@ -81,7 +81,7 @@ func Default() *Genesis { MaxBlockUnits: fees.Dimensions{1_800_000, 2_000, 2_000, 2_000, 2_000}, // Tx Fee Compute Parameters - BaseComputeUnits: 1, + BaseComputeUnits: 1, // Tx Fee Storage Parameters // diff --git a/x/programs/cmd/simulator/vm/storage/storage.go b/x/programs/cmd/simulator/vm/storage/storage.go index 0f0e02f565..c3a6b70728 100644 --- a/x/programs/cmd/simulator/vm/storage/storage.go +++ b/x/programs/cmd/simulator/vm/storage/storage.go @@ -24,10 +24,10 @@ const ( // stateDB keyPrefix = 0x0 - programPrefix = 0x1 - heightPrefix = 0x2 - timestampPrefix = 0x3 - feePrefix = 0x4 + programPrefix = 0x1 + heightPrefix = 0x2 + timestampPrefix = 0x3 + feePrefix = 0x4 ) var ( From 1c8074031393c1fb8a0b5e5764108f4bd6e8b058 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 22:01:36 -0400 Subject: [PATCH 12/29] fix morpheus --- examples/morpheusvm/cmd/morpheus-cli/cmd/action.go | 2 +- examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/action.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/action.go index 51f6e9a1dd..9fd10c3d28 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/action.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/action.go @@ -52,7 +52,7 @@ var transferCmd = &cobra.Command{ } // Generate transaction - _, _, err = sendAndWait(ctx, nil, &actions.Transfer{ + _, _, err = sendAndWait(ctx, &actions.Transfer{ To: recipient, Value: amount, }, cli, bcli, ws, factory, true) diff --git a/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go b/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go index 43482b51a1..1ac060733c 100644 --- a/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go +++ b/examples/morpheusvm/cmd/morpheus-cli/cmd/spam.go @@ -106,7 +106,7 @@ var runSpamCmd = &cobra.Command{ if err != nil { return err } - _, _, err = sendAndWait(ictx, nil, &actions.Transfer{ + _, _, err = sendAndWait(ictx, &actions.Transfer{ To: priv.Address, Value: count, // prevent duplicate txs }, cli, bclient, wclient, factory, false) From 276b3da1766446b873fafb51fd32f482f1188455 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 22:04:54 -0400 Subject: [PATCH 13/29] fix lint --- chain/block.go | 8 +++----- examples/tokenvm/cmd/token-cli/cmd/action.go | 14 +++++++------- examples/tokenvm/cmd/token-cli/cmd/resolutions.go | 15 +++++++-------- examples/tokenvm/cmd/token-cli/cmd/spam.go | 3 +-- vm/resolutions.go | 8 -------- 5 files changed, 18 insertions(+), 30 deletions(-) diff --git a/chain/block.go b/chain/block.go index 1a062b7f1e..c78e01ebdc 100644 --- a/chain/block.go +++ b/chain/block.go @@ -13,7 +13,6 @@ import ( "github.com/ava-labs/avalanchego/snow/choices" "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" - "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/x/merkledb" "go.opentelemetry.io/otel/attribute" @@ -100,8 +99,7 @@ type StatelessBlock struct { bytes []byte txsSet set.Set[ids.ID] - bctx *block.Context - vdrState validators.State + bctx *block.Context results []*Result feeManager *fees.Manager @@ -257,7 +255,7 @@ func (b *StatelessBlock) initializeBuilt( func (b *StatelessBlock) ID() ids.ID { return b.id } // implements "block.WithVerifyContext" -func (b *StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { +func (*StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { return false, nil } @@ -451,7 +449,7 @@ func (b *StatelessBlock) innerVerify(ctx context.Context, vctx VerifyContext) er feeKey := FeeKey(b.vm.StateManager().FeeKey()) feeRaw, err := parentView.GetValue(ctx, feeKey) if err != nil { - return err //nolint:spancheck + return err } parentFeeManager := fees.NewManager(feeRaw) feeManager, err := parentFeeManager.ComputeNext(parentTimestamp, b.Tmstmp, r) diff --git a/examples/tokenvm/cmd/token-cli/cmd/action.go b/examples/tokenvm/cmd/token-cli/cmd/action.go index 1bacc811c5..d8cf34fa2c 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/action.go +++ b/examples/tokenvm/cmd/token-cli/cmd/action.go @@ -69,7 +69,7 @@ var fundFaucetCmd = &cobra.Command{ if err != nil { return err } - if _, err = sendAndWait(ctx, &actions.Transfer{ + if err = sendAndWait(ctx, &actions.Transfer{ To: addr, Asset: ids.Empty, Value: amount, @@ -119,7 +119,7 @@ var transferCmd = &cobra.Command{ } // Generate transaction - _, err = sendAndWait(ctx, &actions.Transfer{ + err = sendAndWait(ctx, &actions.Transfer{ To: recipient, Asset: assetID, Value: amount, @@ -162,7 +162,7 @@ var createAssetCmd = &cobra.Command{ } // Generate transaction - _, err = sendAndWait(ctx, &actions.CreateAsset{ + err = sendAndWait(ctx, &actions.CreateAsset{ Symbol: []byte(symbol), Decimals: uint8(decimals), // already constrain above to prevent overflow Metadata: []byte(metadata), @@ -226,7 +226,7 @@ var mintAssetCmd = &cobra.Command{ } // Generate transaction - _, err = sendAndWait(ctx, &actions.MintAsset{ + err = sendAndWait(ctx, &actions.MintAsset{ Asset: assetID, To: recipient, Value: amount, @@ -263,7 +263,7 @@ var closeOrderCmd = &cobra.Command{ } // Generate transaction - _, err = sendAndWait(ctx, &actions.CloseOrder{ + err = sendAndWait(ctx, &actions.CloseOrder{ Order: orderID, Out: outAssetID, }, cli, scli, tcli, factory, true) @@ -349,7 +349,7 @@ var createOrderCmd = &cobra.Command{ } // Generate transaction - _, err = sendAndWait(ctx, &actions.CreateOrder{ + err = sendAndWait(ctx, &actions.CreateOrder{ In: inAssetID, InTick: inTick, Out: outAssetID, @@ -466,7 +466,7 @@ var fillOrderCmd = &cobra.Command{ if err != nil { return err } - _, err = sendAndWait(ctx, &actions.FillOrder{ + err = sendAndWait(ctx, &actions.FillOrder{ Order: order.ID, Owner: owner, In: inAssetID, diff --git a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go index c3939a3d24..598151c7c9 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/resolutions.go +++ b/examples/tokenvm/cmd/token-cli/cmd/resolutions.go @@ -8,7 +8,6 @@ import ( "fmt" "reflect" - "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/chain" "github.com/ava-labs/hypersdk/cli" "github.com/ava-labs/hypersdk/codec" @@ -23,27 +22,27 @@ import ( func sendAndWait( ctx context.Context, action chain.Action, cli *rpc.JSONRPCClient, scli *rpc.WebSocketClient, tcli *trpc.JSONRPCClient, factory chain.AuthFactory, printStatus bool, -) (ids.ID, error) { +) error { parser, err := tcli.Parser(ctx) if err != nil { - return ids.Empty, err + return err } _, tx, _, err := cli.GenerateTransaction(ctx, parser, action, factory) if err != nil { - return ids.Empty, err + return err } if err := scli.RegisterTx(tx); err != nil { - return ids.Empty, err + return err } var res *chain.Result for { txID, dErr, result, err := scli.ListenTx(ctx) if dErr != nil { - return ids.Empty, dErr + return dErr } if err != nil { - return ids.Empty, err + return err } if txID == tx.ID() { res = result @@ -54,7 +53,7 @@ func sendAndWait( if printStatus { handler.Root().PrintStatus(tx.ID(), res.Success) } - return tx.ID(), nil + return nil } func handleTx(c *trpc.JSONRPCClient, tx *chain.Transaction, result *chain.Result) { diff --git a/examples/tokenvm/cmd/token-cli/cmd/spam.go b/examples/tokenvm/cmd/token-cli/cmd/spam.go index 647c22118f..f3bf3b6660 100644 --- a/examples/tokenvm/cmd/token-cli/cmd/spam.go +++ b/examples/tokenvm/cmd/token-cli/cmd/spam.go @@ -87,11 +87,10 @@ var runSpamCmd = &cobra.Command{ }, func(cli *rpc.JSONRPCClient, priv *cli.PrivateKey) func(context.Context, uint64) error { // submitDummy return func(ictx context.Context, count uint64) error { - _, err := sendAndWait(ictx, &actions.Transfer{ + return sendAndWait(ictx, &actions.Transfer{ To: priv.Address, Value: count, // prevent duplicate txs }, cli, sclient, tclient, auth.NewED25519Factory(ed25519.PrivateKey(priv.Bytes)), false) - return err } }, ) diff --git a/vm/resolutions.go b/vm/resolutions.go index b566f0a91e..0b8ec49a3d 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -417,14 +417,6 @@ func (vm *VM) GetAuthBatchVerifier(authTypeID uint8, cores int, count int) (chai 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) -} - func (vm *VM) RecordBlockVerify(t time.Duration) { vm.metrics.blockVerify.Observe(float64(t)) } From 32f442264fc12aad4bc166cc1fb72f49b00e2b73 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 22:20:49 -0400 Subject: [PATCH 14/29] fix x/programs --- x/programs/cmd/simulator/cmd/program.go | 4 ++-- x/programs/cmd/simulator/vm/actions/program_create.go | 1 - x/programs/cmd/simulator/vm/actions/program_execute.go | 1 - x/programs/cmd/simulator/vm/storage/state_manager.go | 1 - 4 files changed, 2 insertions(+), 5 deletions(-) diff --git a/x/programs/cmd/simulator/cmd/program.go b/x/programs/cmd/simulator/cmd/program.go index 48409d2495..d0c9a98307 100644 --- a/x/programs/cmd/simulator/cmd/program.go +++ b/x/programs/cmd/simulator/cmd/program.go @@ -114,7 +114,7 @@ func programCreateFunc(ctx context.Context, db *state.SimpleMutable, path string } // execute the action - success, _, output, _, err := programCreateAction.Execute(ctx, nil, db, 0, codec.EmptyAddress, programID, false) + success, _, output, err := programCreateAction.Execute(ctx, nil, db, 0, codec.EmptyAddress, programID) if output != nil { fmt.Println(string(output)) } @@ -157,7 +157,7 @@ func programExecuteFunc( } // execute the action - success, _, resp, _, err := programExecuteAction.Execute(ctx, nil, db, 0, codec.EmptyAddress, programTxID, false) + success, _, resp, err := programExecuteAction.Execute(ctx, nil, db, 0, codec.EmptyAddress, programTxID) if !success { return ids.Empty, nil, 0, fmt.Errorf("program execution failed: %s", string(resp)) diff --git a/x/programs/cmd/simulator/vm/actions/program_create.go b/x/programs/cmd/simulator/vm/actions/program_create.go index 6433b144d3..f010f1cb31 100644 --- a/x/programs/cmd/simulator/vm/actions/program_create.go +++ b/x/programs/cmd/simulator/vm/actions/program_create.go @@ -42,7 +42,6 @@ func (t *ProgramCreate) Execute( _ int64, _ codec.Address, id ids.ID, - _ bool, ) (bool, uint64, []byte, error) { if len(t.Program) == 0 { return false, 1, OutputValueZero, nil diff --git a/x/programs/cmd/simulator/vm/actions/program_execute.go b/x/programs/cmd/simulator/vm/actions/program_execute.go index 5977119a21..cb7c51b988 100644 --- a/x/programs/cmd/simulator/vm/actions/program_execute.go +++ b/x/programs/cmd/simulator/vm/actions/program_execute.go @@ -59,7 +59,6 @@ func (t *ProgramExecute) Execute( _ int64, actor codec.Address, _ ids.ID, - _ bool, ) (success bool, computeUnits uint64, output []byte, err error) { if len(t.Function) == 0 { return false, 1, OutputValueZero, nil diff --git a/x/programs/cmd/simulator/vm/storage/state_manager.go b/x/programs/cmd/simulator/vm/storage/state_manager.go index 678bfe3490..ea4d62ea29 100644 --- a/x/programs/cmd/simulator/vm/storage/state_manager.go +++ b/x/programs/cmd/simulator/vm/storage/state_manager.go @@ -6,7 +6,6 @@ package storage import ( "context" - "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/hypersdk/codec" "github.com/ava-labs/hypersdk/state" ) From 8ead7463129d3f0856395aa7df74baffe9cd3a93 Mon Sep 17 00:00:00 2001 From: William Law Date: Tue, 23 Apr 2024 23:02:53 -0400 Subject: [PATCH 15/29] remove StatelessBlock.bctx --- chain/block.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/chain/block.go b/chain/block.go index c78e01ebdc..ee4f7dd9f8 100644 --- a/chain/block.go +++ b/chain/block.go @@ -99,8 +99,6 @@ type StatelessBlock struct { bytes []byte txsSet set.Set[ids.ID] - bctx *block.Context - results []*Result feeManager *fees.Manager @@ -260,7 +258,7 @@ func (*StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { } // implements "block.WithVerifyContext" -func (b *StatelessBlock) VerifyWithContext(ctx context.Context, bctx *block.Context) error { +func (b *StatelessBlock) VerifyWithContext(ctx context.Context, _ *block.Context) error { start := time.Now() defer func() { b.vm.RecordBlockVerify(time.Since(start)) @@ -273,15 +271,11 @@ func (b *StatelessBlock) VerifyWithContext(ctx context.Context, bctx *block.Cont attribute.Int("txs", len(b.Txs)), attribute.Int64("height", int64(b.Hght)), attribute.Bool("stateReady", stateReady), - attribute.Int64("pchainHeight", int64(bctx.PChainHeight)), attribute.Bool("built", b.Processed()), ), ) defer span.End() - // Persist the context in case we need it during Accept - b.bctx = bctx - // Proceed with normal verification return b.verify(ctx, stateReady) } From 727330034ccae47d7479b2e1a811693781abb9bf Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 10:32:52 -0400 Subject: [PATCH 16/29] remove block functions --- chain/block.go | 33 ++------------------------------- vm/resolutions.go | 11 +++++------ vm/vm.go | 13 ------------- 3 files changed, 7 insertions(+), 50 deletions(-) diff --git a/chain/block.go b/chain/block.go index ee4f7dd9f8..c2c8067682 100644 --- a/chain/block.go +++ b/chain/block.go @@ -29,9 +29,8 @@ import ( ) var ( - _ snowman.Block = &StatelessBlock{} - _ block.WithVerifyContext = &StatelessBlock{} - _ block.StateSummary = &SyncableBlock{} + _ snowman.Block = &StatelessBlock{} + _ block.StateSummary = &SyncableBlock{} ) type StatefulBlock struct { @@ -252,34 +251,6 @@ func (b *StatelessBlock) initializeBuilt( // implements "snowman.Block.choices.Decidable" func (b *StatelessBlock) ID() ids.ID { return b.id } -// implements "block.WithVerifyContext" -func (*StatelessBlock) ShouldVerifyWithContext(context.Context) (bool, error) { - return false, nil -} - -// implements "block.WithVerifyContext" -func (b *StatelessBlock) VerifyWithContext(ctx context.Context, _ *block.Context) error { - start := time.Now() - defer func() { - b.vm.RecordBlockVerify(time.Since(start)) - }() - - stateReady := b.vm.StateReady() - ctx, span := b.vm.Tracer().Start( - ctx, "StatelessBlock.VerifyWithContext", - trace.WithAttributes( - attribute.Int("txs", len(b.Txs)), - attribute.Int64("height", int64(b.Hght)), - attribute.Bool("stateReady", stateReady), - attribute.Bool("built", b.Processed()), - ), - ) - defer span.End() - - // Proceed with normal verification - return b.verify(ctx, stateReady) -} - // implements "snowman.Block" func (b *StatelessBlock) Verify(ctx context.Context) error { start := time.Now() diff --git a/vm/resolutions.go b/vm/resolutions.go index 0b8ec49a3d..972ed67e3c 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -26,12 +26,11 @@ import ( ) var ( - _ chain.VM = (*VM)(nil) - _ gossiper.VM = (*VM)(nil) - _ builder.VM = (*VM)(nil) - _ block.ChainVM = (*VM)(nil) - _ block.StateSyncableVM = (*VM)(nil) - _ block.BuildBlockWithContextChainVM = (*VM)(nil) + _ chain.VM = (*VM)(nil) + _ gossiper.VM = (*VM)(nil) + _ builder.VM = (*VM)(nil) + _ block.ChainVM = (*VM)(nil) + _ block.StateSyncableVM = (*VM)(nil) ) func (vm *VM) ChainID() ids.ID { diff --git a/vm/vm.go b/vm/vm.go index 1867229cab..d4b4c78237 100644 --- a/vm/vm.go +++ b/vm/vm.go @@ -753,19 +753,6 @@ func (vm *VM) BuildBlock(ctx context.Context) (snowman.Block, error) { return vm.buildBlock(ctx, nil) } -// implements "block.BuildBlockWithContextChainVM" -func (vm *VM) BuildBlockWithContext(ctx context.Context, blockContext *block.Context) (snowman.Block, error) { - start := time.Now() - defer func() { - vm.metrics.blockBuild.Observe(float64(time.Since(start))) - }() - - ctx, span := vm.tracer.Start(ctx, "VM.BuildBlockWithContext") - defer span.End() - - return vm.buildBlock(ctx, blockContext) -} - func (vm *VM) Submit( ctx context.Context, verifyAuth bool, From 8842b0d22cd8498c41443d1197844d9a987497e0 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 10:36:04 -0400 Subject: [PATCH 17/29] update readme --- README.md | 131 ++---------------------------------------------------- 1 file changed, 4 insertions(+), 127 deletions(-) diff --git a/README.md b/README.md index 9f5b8efbb3..b4f614cf5a 100644 --- a/README.md +++ b/README.md @@ -200,7 +200,6 @@ type StatefulBlock struct { Txs []*Transaction `json:"txs"` StateRoot ids.ID `json:"stateRoot"` - WarpResults set.Bits64 `json:"warpResults"` } ``` @@ -276,9 +275,6 @@ GetWindowTargetUnits() Dimensions GetMaxBlockUnits() Dimensions GetBaseComputeUnits() uint64 -GetBaseWarpComputeUnits() uint64 -GetWarpComputeUnitsPerSigner() uint64 -GetOutgoingWarpComputeUnits() uint64 GetStorageKeyReadUnits() uint64 GetStorageValueReadUnits() uint64 // per chunk @@ -296,9 +292,6 @@ WindowTargetUnits: chain.Dimensions{20_000_000, 1_000, 1_000, 1_000, 1_ MaxBlockUnits: chain.Dimensions{1_800_000, 2_000, 2_000, 2_000, 2_000}, BaseComputeUnits: 1, -BaseWarpComputeUnits: 1_024, -WarpComputeUnitsPerSigner: 128, -OutgoingWarpComputeUnits: 1_024, StorageKeyReadUnits: 5, StorageValueReadUnits: 2, @@ -399,36 +392,6 @@ for a single account and ensure they are ordered) and makes the network layer more efficient (we can gossip any valid transaction to any node instead of just the transactions for each account that can be executed at the moment). -### Avalanche Warp Messaging Support -`hypersdk` provides support for Avalanche Warp Messaging (AWM) out-of-the-box. AWM enables any -Avalanche Subnet to send arbitrary messages to any other Avalanche Subnet in just a few -seconds (or less) without relying on a trusted relayer or bridge (just the validators of the Subnet sending the message). -You can learn more about AWM and how it works -[here](https://docs.google.com/presentation/d/1eV4IGMB7qNV7Fc4hp7NplWxK_1cFycwCMhjrcnsE9mU/edit). - -

- warp -

- -AWM is a primitive provided by the Avalanche Network used to verify that -a particular [BLS Multi-Signatures](https://crypto.stanford.edu/~dabo/pubs/papers/BLSmultisig.html) -is valid and signed by some % of the stake weight of a particular Avalanche -Subnet (typically the Subnet where the message originated). Specifying when an -Avalanche Custom VM produces a Warp Message for signing, defining the format -of Warp Messages sent between Subnets, implementing some mechanism to gather -individual signatures from validators (to aggregate into a BLS -Multi-Signature) over this user-defined message, articulating how an imported -Warp Message from another Subnet is handled on a destination (if the -destination chooses to even accept the message), and enabling retries in the -case that a message is dropped or the BLS Multi-Signature expires are just a few of the items -left to the implementer. - -The `hypersdk` handles all of the above items for you except for defining when -you should emit a Warp Message to send to another Subnet (i.e. what an export looks like on-chain), -what this Warp Message should look like (i.e. what do you want to send to another Subnet), and -what you should do if you receive a Warp Message (i.e. mint assets if you -receive an import). - ### Easy Functionality Upgrades Every object that can appear on-chain (i.e. `Actions` and/or `Auth`) and every chain parameter (i.e. `Unit Price`) is scoped by block timestamp. This makes it @@ -630,8 +593,8 @@ You can view what this looks like in the `tokenvm` by clicking this #### Registry ```golang -ActionRegistry *codec.TypeParser[Action, *warp.Message, bool] -AuthRegistry *codec.TypeParser[Auth, *warp.Message, bool] +ActionRegistry *codec.TypeParser[Action, bool] +AuthRegistry *codec.TypeParser[Auth, bool] ``` The `ActionRegistry` and `AuthRegistry` inform the `hypersdk` how to @@ -702,12 +665,7 @@ type Action interface { timestamp int64, actor codec.Address, txID ids.ID, - warpVerified bool, - ) (success bool, computeUnits uint64, output []byte, warpMessage *warp.UnsignedMessage, err error) - - // OutputsWarpMessage indicates whether an [Action] will produce a warp message. The max size - // of any warp message is [MaxOutgoingWarpChunks]. - OutputsWarpMessage() bool + ) (success bool, computeUnits uint64, output []byte, err error) } ``` @@ -727,16 +685,13 @@ type Result struct { Consumed Dimensions Fee uint64 - - WarpMessage *warp.UnsignedMessage } ``` `Actions` emit a `Result` at the end of their execution. This `Result` indicates if the execution was a `Success` (if not, all effects are rolled back), how many `Units` were used (failed execution may not use all units an -`Action` requested), an `Output` (arbitrary bytes specific to the `hypervm`), -and optionally a `WarpMessage` (which Subnet Validators will sign). +`Action` requested), an `Output` (arbitrary bytes specific to the `hypervm`). ### Auth ```golang @@ -795,9 +750,6 @@ type Rules interface { GetMaxBlockUnits() Dimensions GetBaseComputeUnits() uint64 - GetBaseWarpComputeUnits() uint64 - GetWarpComputeUnitsPerSigner() uint64 - GetOutgoingWarpComputeUnits() uint64 // Invariants: // * Controllers must manage the max key length and max value length (max network @@ -819,8 +771,6 @@ type Rules interface { GetStorageKeyWriteUnits() uint64 GetStorageValueWriteUnits() uint64 // per chunk - GetWarpConfig(sourceChainID ids.ID) (bool, uint64, uint64) - FetchCustom(string) (any, bool) } ``` @@ -836,79 +786,6 @@ You can view what this looks like in the `indexvm` by clicking case of the `indexvm`, the custom rule support is used to set the cost for adding anything to state (which is a very `hypervm-specific` value). -### Avalanche Warp Messaging -To add AWM support to a `hypervm`, an implementer first specifies whether a -particular `Action`/`Auth` item expects a `*warp.Message` when registering -them with their corresponding registry (`false` if no expected and `true` if -so): -```golang -ActionRegistry.Register(&actions.Transfer{}, actions.UnmarshalTransfer, false) -ActionRegistry.Register(&actions.ImportAsset{}, actions.UnmarshalImportAsset, true) -``` - -You can view what this looks like in the `tokenvm` by clicking -[here](./examples/tokenvm/controller/registry.go). The `hypersdk` uses this -boolean to enforce the existence/non-existence of a `*warp.Message` on the -`chain.Transaction` that wraps the `Action` (marking a block as invalid if there is -something unexpected). - -`Actions` can use the provided `*warp.Message` in their registered unmarshaler -(in this case, the provided `*warp.Message` is parsed into a format specified -by the `tokenvm`): -```golang -func UnmarshalImportAsset(p *codec.Packer, wm *warp.Message) (chain.Action, error) { - var ( - imp ImportAsset - err error - ) - imp.Fill = p.UnpackBool() - if err := p.Err(); err != nil { - return nil, err - } - imp.warpMessage = wm - imp.warpTransfer, err = UnmarshalWarpTransfer(imp.warpMessage.Payload) - if err != nil { - return nil, err - } - // Ensure we can fill the swap if it exists - if imp.Fill && imp.warpTransfer.SwapIn == 0 { - return nil, ErrNoSwapToFill - } - return &imp, nil -} -``` - -This `WarpTransfer` object looks like: -```golang -type WarpTransfer struct { - To crypto.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. - Return bool `json:"return"` - - // Reward is the amount of [Asset] to send the [Actor] that submits this - // transaction. - Reward uint64 `json:"reward"` - - // SwapIn is the amount of [Asset] we are willing to swap for [AssetOut]. - SwapIn uint64 `json:"swapIn"` - // AssetOut is the asset we are seeking to get for [SwapIn]. - AssetOut ids.ID `json:"assetOut"` - // SwapOut is the amount of [AssetOut] we are seeking. - SwapOut uint64 `json:"swapOut"` - // SwapExpiry is the unix timestamp at which the swap becomes invalid (and - // the message can be processed without a swap. - SwapExpiry int64 `json:"swapExpiry"` - - // TxID is the transaction that created this message. This is used to ensure - // there is WarpID uniqueness. - TxID ids.ID `json:"txID"` -} -``` - You can view what the import `Action` associated with the above examples looks like [here](./examples/tokenvm/actions/import_asset.go) From 04226e33901cbfddd609f0e933b4cc9b8ee0da55 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 10:37:33 -0400 Subject: [PATCH 18/29] remove unused outputs --- examples/tokenvm/actions/outputs.go | 1 - 1 file changed, 1 deletion(-) diff --git a/examples/tokenvm/actions/outputs.go b/examples/tokenvm/actions/outputs.go index 1c4fd03622..c9535fc315 100644 --- a/examples/tokenvm/actions/outputs.go +++ b/examples/tokenvm/actions/outputs.go @@ -29,7 +29,6 @@ var ( OutputMetadataEmpty = []byte("metadata is empty") OutputMetadataTooLarge = []byte("metadata is too large") OutputSameInOut = []byte("same asset used for in and out") - OutputAnycast = []byte("anycast output") OutputWrongDestination = []byte("wrong destination") OutputMustFill = []byte("must fill request") OutputInvalidDestination = []byte("invalid destination") From 9a51964a7dad21ef5c26e75de0368199257295d6 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 10:41:58 -0400 Subject: [PATCH 19/29] fix Result.Size() --- chain/result.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/chain/result.go b/chain/result.go index 7154576acd..6949376946 100644 --- a/chain/result.go +++ b/chain/result.go @@ -18,9 +18,7 @@ type Result struct { } func (r *Result) Size() int { - size := consts.BoolLen + codec.BytesLen(r.Output) + fees.DimensionsLen + consts.Uint64Len - size += codec.BytesLen(nil) - return size + return consts.BoolLen + codec.BytesLen(r.Output) + fees.DimensionsLen + consts.Uint64Len } func (r *Result) Marshal(p *codec.Packer) error { From 688fec97a2108527e45829984e2433939aa11cf8 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 10:43:04 -0400 Subject: [PATCH 20/29] inline size into NewWriter --- chain/transaction.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/chain/transaction.go b/chain/transaction.go index 5dafb4cb04..5357692cea 100644 --- a/chain/transaction.go +++ b/chain/transaction.go @@ -52,8 +52,7 @@ func (t *Transaction) Digest() ([]byte, error) { return t.digest, nil } actionID := t.Action.GetTypeID() - size := t.Base.Size() + consts.ByteLen + t.Action.Size() - p := codec.NewWriter(size, consts.NetworkSizeLimit) + p := codec.NewWriter(t.Base.Size()+consts.ByteLen+t.Action.Size(), consts.NetworkSizeLimit) t.Base.Marshal(p) p.PackByte(actionID) t.Action.Marshal(p) From a3ef92e8abf680b3b29a7a368ed599ebeb0a2a9e Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 11:07:50 -0400 Subject: [PATCH 21/29] keep vm.cacheAuth --- vm/resolutions.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/vm/resolutions.go b/vm/resolutions.go index 972ed67e3c..5d26039f3f 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -174,6 +174,11 @@ func (vm *VM) processAcceptedBlock(b *chain.StatelessBlock) { vm.Fatal("accepted processing failed", zap.Error(err)) } + for i, tx := range b.Txs { + // Only cache auth for accepted blocks to prevent cache manipulation from RPC submissions + vm.cacheAuth(tx.Auth) + } + // Update server if err := vm.webSocketServer.AcceptBlock(b); err != nil { vm.Fatal("unable to accept block in websocket server", zap.Error(err)) @@ -416,6 +421,14 @@ func (vm *VM) GetAuthBatchVerifier(authTypeID uint8, cores int, count int) (chai 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) +} + func (vm *VM) RecordBlockVerify(t time.Duration) { vm.metrics.blockVerify.Observe(float64(t)) } From dc82c617f1767d95249695420ecd431f07d5116b Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 11:14:58 -0400 Subject: [PATCH 22/29] storage remove unnecessary alloc --- examples/tokenvm/storage/storage.go | 2 -- vm/resolutions.go | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index 4e79c005c5..59b89a61f7 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -350,8 +350,6 @@ func SetAsset( copy(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len:], metadata) binary.BigEndian.PutUint64(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen:], supply) copy(v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len:], owner[:]) - b := byte(0x0) - v[consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len+codec.AddressLen] = b return mu.Insert(ctx, k, v) } diff --git a/vm/resolutions.go b/vm/resolutions.go index 5d26039f3f..c7b17ceafc 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -174,7 +174,7 @@ func (vm *VM) processAcceptedBlock(b *chain.StatelessBlock) { vm.Fatal("accepted processing failed", zap.Error(err)) } - for i, tx := range b.Txs { + for _, tx := range b.Txs { // Only cache auth for accepted blocks to prevent cache manipulation from RPC submissions vm.cacheAuth(tx.Auth) } From 86ee5c2a87c7280ee7519fb1f64dcc2378210fde Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 11:19:32 -0400 Subject: [PATCH 23/29] check no bytes left to read --- chain/result.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/chain/result.go b/chain/result.go index 6949376946..9e9964608d 100644 --- a/chain/result.go +++ b/chain/result.go @@ -58,6 +58,9 @@ func UnmarshalResult(p *codec.Packer) (*Result, error) { } result.Consumed = consumed result.Fee = p.UnpackUint64(false) + if !p.Empty() { + return nil, p.Err() + } return result, p.Err() } From f873f852a3287251a729da9fd0589108d91b871b Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 13:53:00 -0400 Subject: [PATCH 24/29] update tokenvm readme --- examples/tokenvm/README.md | 45 +++----------------------------------- 1 file changed, 3 insertions(+), 42 deletions(-) diff --git a/examples/tokenvm/README.md b/examples/tokenvm/README.md index 52bc0ff5a6..3210c6208b 100644 --- a/examples/tokenvm/README.md +++ b/examples/tokenvm/README.md @@ -105,41 +105,6 @@ be valid only until a particular time. This enables you to go for orders as you see fit at the time and not have to worry about your fill sitting around until you explicitly cancel it/replace it. -### Avalanche Warp Support -We take advantage of the Avalanche Warp Messaging (AWM) support provided by the -`hypersdk` to enable any `tokenvm` to send assets to any other `tokenvm` without -relying on a trusted relayer or bridge (just the validators of the `tokenvm` -sending the message). - -By default, a `tokenvm` will accept a message from another `tokenvm` if 80% of -the stake weight of the source has signed it. Because each imported asset is -given a unique `AssetID` (hash of `sourceChainID + sourceAssetID`), it is not -possible for a malicious/rogue Subnet to corrupt token balances imported from -other Subnets with this default import setting. `tokenvms` also track the -amount of assets exported to all other `tokenvms` and ensure that more assets -can't be brought back from a `tokenvm` than were exported to it (prevents -infinite minting). - -To limit "contagion" in the case of a `tokenvm` failure, we ONLY allow the -export of natively minted assets to another `tokenvm`. This means you can -transfer an asset between two `tokenvms` A and B but you can't export from -`tokenvm` A to `tokenvm` B to `tokenvm` C. This ensures that the import policy -for an external `tokenvm` is always transparent and is never inherited -implicitly by the transfers between other `tokenvms`. The ability to impose -this restriction (without massively driving up the cost of each transfer) is -possible because AWM does not impose an additional overhead per Subnet -connection (no "per connection" state to maintain). This means it is just as -cheap/scalable to communicate with every other `tokenvm` as it is to only -communicate with one. - -Lastly, the `tokenvm` allows users to both tip relayers (whoever sends -a transaction that imports their message) and to swap for another asset when -their message is imported (so they can acquire fee-paying tokens right when -they arrive). - -You can see how this works by checking out the [E2E test suite](./tests/e2e/e2e_test.go) that -runs through these flows. - ## Demos Someone: "Seems cool but I need to see it to really get it." Me: "Look no further." @@ -237,7 +202,7 @@ database: .token-cli address: token1rvzhmceq997zntgvravfagsks6w0ryud3rylh4cdvayry0dl97nsjzf3yp chainID: Em2pZtHr7rDCzii43an2bBi1M2mTFyLN33QP1Xfjy7BcWtaH9 assetID (use TKN for native token): 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug -metadata: MarioCoin supply: 10000 warp: false +metadata: MarioCoin supply: 10000 balance: 10000 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug ``` @@ -257,7 +222,7 @@ chainID: Em2pZtHr7rDCzii43an2bBi1M2mTFyLN33QP1Xfjy7BcWtaH9 in assetID (use TKN for native token): TKN ✔ in tick: 1█ out assetID (use TKN for native token): 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug -metadata: MarioCoin supply: 10000 warp: false +metadata: MarioCoin supply: 10000 balance: 10000 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug out tick: 10 supply (must be multiple of out tick): 100 @@ -287,7 +252,7 @@ chainID: Em2pZtHr7rDCzii43an2bBi1M2mTFyLN33QP1Xfjy7BcWtaH9 in assetID (use TKN for native token): TKN balance: 997.999993843 TKN out assetID (use TKN for native token): 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug -metadata: MarioCoin supply: 10000 warp: false +metadata: MarioCoin supply: 10000 available orders: 1 0) Rate(in/out): 100000000.0000 InTick: 1.000000000 TKN OutTick: 10 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug Remaining: 100 27grFs9vE2YP9kwLM5hQJGLDvqEY9ii71zzdoRHNGC4Appavug select order: 0 @@ -484,10 +449,6 @@ out on the Avalanche Discord._ * Add expiring order support (can't fill an order after some point in time but still need to explicitly close it to get your funds back -> async cleanup is not a good idea) -* Add lockup fee for creating a Warp Message and ability to reclaim the lockup - with a refund action (this will allow for "user-driven" acks on - messages, which will remain signable and in state until a refund action is - issued)

From b1a8d57ede04ae95033cdd60996b44378fd27134 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 14:02:34 -0400 Subject: [PATCH 25/29] reduce alloc size of v --- examples/tokenvm/storage/storage.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/tokenvm/storage/storage.go b/examples/tokenvm/storage/storage.go index 59b89a61f7..e8382a8e16 100644 --- a/examples/tokenvm/storage/storage.go +++ b/examples/tokenvm/storage/storage.go @@ -342,7 +342,7 @@ func SetAsset( k := AssetKey(asset) symbolLen := len(symbol) metadataLen := len(metadata) - v := make([]byte, consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len+codec.AddressLen+1) + v := make([]byte, consts.Uint16Len+symbolLen+consts.Uint8Len+consts.Uint16Len+metadataLen+consts.Uint64Len+codec.AddressLen) binary.BigEndian.PutUint16(v, uint16(symbolLen)) copy(v[consts.Uint16Len:], symbol) v[consts.Uint16Len+symbolLen] = decimals From 6529b6ee4ed45bf6a468c3ed4d39c1956b9f7410 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 14:07:06 -0400 Subject: [PATCH 26/29] remove vm.buildBlock blockCtx --- chain/builder.go | 3 --- vm/vm.go | 7 +++---- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/chain/builder.go b/chain/builder.go index ea12487d6f..c6122563b1 100644 --- a/chain/builder.go +++ b/chain/builder.go @@ -13,7 +13,6 @@ import ( "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/utils/set" "go.opentelemetry.io/otel/attribute" @@ -65,7 +64,6 @@ func BuildBlock( ctx context.Context, vm VM, parent *StatelessBlock, - blockContext *block.Context, ) (*StatelessBlock, error) { ctx, span := vm.Tracer().Start(ctx, "chain.BuildBlock") defer span.End() @@ -457,7 +455,6 @@ func BuildBlock( log.Info( "built block", - zap.Bool("context", blockContext != nil), zap.Uint64("hght", b.Hght), zap.Int("attempted", txsAttempted), zap.Int("added", len(b.Txs)), diff --git a/vm/vm.go b/vm/vm.go index d4b4c78237..f763575fe3 100644 --- a/vm/vm.go +++ b/vm/vm.go @@ -17,7 +17,6 @@ import ( "github.com/ava-labs/avalanchego/snow/choices" "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/profiler" "github.com/ava-labs/avalanchego/utils/set" @@ -699,7 +698,7 @@ func (vm *VM) ParseBlock(ctx context.Context, source []byte) (snowman.Block, err return newBlk, nil } -func (vm *VM) buildBlock(ctx context.Context, blockContext *block.Context) (snowman.Block, error) { +func (vm *VM) buildBlock(ctx context.Context) (snowman.Block, error) { // If the node isn't ready, we should exit. // // We call [QueueNotify] when the VM becomes ready, so exiting @@ -729,7 +728,7 @@ func (vm *VM) buildBlock(ctx context.Context, blockContext *block.Context) (snow vm.snowCtx.Log.Warn("unable to get preferred block", zap.Error(err)) return nil, err } - blk, err := chain.BuildBlock(ctx, vm, preferredBlk, blockContext) + blk, err := chain.BuildBlock(ctx, vm, preferredBlk) if err != nil { // This is a DEBUG log because BuildBlock may fail before // the min build gap (especially when there are no transactions). @@ -750,7 +749,7 @@ func (vm *VM) BuildBlock(ctx context.Context) (snowman.Block, error) { ctx, span := vm.tracer.Start(ctx, "VM.BuildBlock") defer span.End() - return vm.buildBlock(ctx, nil) + return vm.buildBlock(ctx) } func (vm *VM) Submit( From cf42708523fcbcf49d1e50ad46bb1d93dfcecdf7 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 14:07:51 -0400 Subject: [PATCH 27/29] add todo to remove cachAuth --- vm/resolutions.go | 1 + 1 file changed, 1 insertion(+) diff --git a/vm/resolutions.go b/vm/resolutions.go index c7b17ceafc..f12d892224 100644 --- a/vm/resolutions.go +++ b/vm/resolutions.go @@ -174,6 +174,7 @@ func (vm *VM) processAcceptedBlock(b *chain.StatelessBlock) { vm.Fatal("accepted processing failed", zap.Error(err)) } + // TODO: consider removing this (unused and requires an extra iteration) for _, tx := range b.Txs { // Only cache auth for accepted blocks to prevent cache manipulation from RPC submissions vm.cacheAuth(tx.Auth) From 0335fe369eadb55198267d18f01b7ea5901690ca Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 14:10:13 -0400 Subject: [PATCH 28/29] unify verify and Verify --- chain/block.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/chain/block.go b/chain/block.go index c2c8067682..fd97d86f1d 100644 --- a/chain/block.go +++ b/chain/block.go @@ -270,10 +270,6 @@ func (b *StatelessBlock) Verify(ctx context.Context) error { ) defer span.End() - return b.verify(ctx, stateReady) -} - -func (b *StatelessBlock) verify(ctx context.Context, stateReady bool) error { log := b.vm.Logger() switch { case !stateReady: From f5dc77fb6a2ace1bcb462268b844d71da8cb9a45 Mon Sep 17 00:00:00 2001 From: William Law Date: Wed, 24 Apr 2024 14:12:52 -0400 Subject: [PATCH 29/29] unify buildBlock and BuildBlock --- vm/vm.go | 24 ++++++++++-------------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/vm/vm.go b/vm/vm.go index f763575fe3..ab91d2970f 100644 --- a/vm/vm.go +++ b/vm/vm.go @@ -698,7 +698,16 @@ func (vm *VM) ParseBlock(ctx context.Context, source []byte) (snowman.Block, err return newBlk, nil } -func (vm *VM) buildBlock(ctx context.Context) (snowman.Block, error) { +// implements "block.ChainVM" +func (vm *VM) BuildBlock(ctx context.Context) (snowman.Block, error) { + start := time.Now() + defer func() { + vm.metrics.blockBuild.Observe(float64(time.Since(start))) + }() + + ctx, span := vm.tracer.Start(ctx, "VM.BuildBlock") + defer span.End() + // If the node isn't ready, we should exit. // // We call [QueueNotify] when the VM becomes ready, so exiting @@ -739,19 +748,6 @@ func (vm *VM) buildBlock(ctx context.Context) (snowman.Block, error) { return blk, nil } -// implements "block.ChainVM" -func (vm *VM) BuildBlock(ctx context.Context) (snowman.Block, error) { - start := time.Now() - defer func() { - vm.metrics.blockBuild.Observe(float64(time.Since(start))) - }() - - ctx, span := vm.tracer.Start(ctx, "VM.BuildBlock") - defer span.End() - - return vm.buildBlock(ctx) -} - func (vm *VM) Submit( ctx context.Context, verifyAuth bool,