diff --git a/tests/e2e/p/l1.go b/tests/e2e/p/l1.go index e9c8ec89a6c3..73c1e08eed1e 100644 --- a/tests/e2e/p/l1.go +++ b/tests/e2e/p/l1.go @@ -4,33 +4,58 @@ package p import ( + "context" + "errors" "math" + "slices" "time" "github.com/onsi/ginkgo/v2" + "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/proto" "github.com/ava-labs/avalanchego/api/info" "github.com/ava-labs/avalanchego/config" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/peer" + "github.com/ava-labs/avalanchego/proto/pb/sdk" + "github.com/ava-labs/avalanchego/snow/networking/router" "github.com/ava-labs/avalanchego/tests/fixture/e2e" "github.com/ava-labs/avalanchego/tests/fixture/tmpnet" + "github.com/ava-labs/avalanchego/utils" + "github.com/ava-labs/avalanchego/utils/buffer" "github.com/ava-labs/avalanchego/utils/constants" "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/vms/example/xsvm/genesis" "github.com/ava-labs/avalanchego/vms/platformvm" "github.com/ava-labs/avalanchego/vms/platformvm/txs" + "github.com/ava-labs/avalanchego/vms/platformvm/warp" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" "github.com/ava-labs/avalanchego/vms/secp256k1fx" + p2pmessage "github.com/ava-labs/avalanchego/message" + p2psdk "github.com/ava-labs/avalanchego/network/p2p" + p2ppb "github.com/ava-labs/avalanchego/proto/pb/p2p" snowvalidators "github.com/ava-labs/avalanchego/snow/validators" + platformvmvalidators "github.com/ava-labs/avalanchego/vms/platformvm/validators" warpmessage "github.com/ava-labs/avalanchego/vms/platformvm/warp/message" ) const ( - genesisWeight = units.Schmeckle - genesisBalance = units.Avax + genesisWeight = units.Schmeckle + genesisBalance = units.Avax + registerWeight = genesisWeight / 10 + registerBalance = 0 + + // Validator registration attempts expire 5 minutes after they are created + expiryDelay = 5 * time.Minute + // P2P message requests timeout after 10 seconds + p2pTimeout = 10 * time.Second ) var _ = e2e.DescribePChain("[L1]", func() { @@ -164,6 +189,22 @@ var _ = e2e.DescribePChain("[L1]", func() { genesisNodePK, err := bls.PublicKeyFromCompressedBytes(genesisNodePoP.PublicKey[:]) require.NoError(err) + tc.By("connecting to the genesis validator") + var ( + networkID = env.GetNetwork().GetNetworkID() + genesisPeerMessages = buffer.NewUnboundedBlockingDeque[p2pmessage.InboundMessage](1) + ) + genesisPeer, err := peer.StartTestPeer( + tc.DefaultContext(), + subnetGenesisNode.StakingAddress, + networkID, + router.InboundHandlerFunc(func(_ context.Context, m p2pmessage.InboundMessage) { + tc.Outf("received %s %s from %s\n", m.Op(), m.Message(), m.NodeID()) + genesisPeerMessages.PushRight(m) + }), + ) + require.NoError(err) + address := []byte{} tc.By("issuing a ConvertSubnetTx", func() { _, err := pWallet.IssueConvertSubnetTx( @@ -227,6 +268,200 @@ var _ = e2e.DescribePChain("[L1]", func() { }) }) + advanceProposerVMPChainHeight := func() { + // We must wait at least [RecentlyAcceptedWindowTTL] to ensure the + // next block will reference the last accepted P-chain height. + time.Sleep((5 * platformvmvalidators.RecentlyAcceptedWindowTTL) / 4) + } + tc.By("advancing the proposervm P-chain height", advanceProposerVMPChainHeight) + + tc.By("creating the validator to register") + subnetRegisterNode := e2e.AddEphemeralNode(tc, env.GetNetwork(), tmpnet.FlagsMap{ + config.TrackSubnetsKey: subnetID.String(), + }) + + registerNodePoP, err := subnetRegisterNode.GetProofOfPossession() + require.NoError(err) + + tc.By("ensuring the subnet nodes are healthy", func() { + e2e.WaitForHealthy(tc, subnetGenesisNode) + e2e.WaitForHealthy(tc, subnetRegisterNode) + }) + + tc.By("creating the RegisterSubnetValidatorMessage") + expiry := uint64(time.Now().Add(expiryDelay).Unix()) // This message will expire in 5 minutes + registerSubnetValidatorMessage, err := warpmessage.NewRegisterSubnetValidator( + subnetID, + subnetRegisterNode.NodeID, + registerNodePoP.PublicKey, + expiry, + warpmessage.PChainOwner{}, + warpmessage.PChainOwner{}, + registerWeight, + ) + require.NoError(err) + + tc.By("registering the validator", func() { + tc.By("creating the unsigned warp message") + unsignedRegisterSubnetValidator := must[*warp.UnsignedMessage](tc)(warp.NewUnsignedMessage( + networkID, + chainID, + must[*payload.AddressedCall](tc)(payload.NewAddressedCall( + address, + registerSubnetValidatorMessage.Bytes(), + )).Bytes(), + )) + + tc.By("sending the request to sign the warp message", func() { + registerSubnetValidatorRequest, err := wrapWarpSignatureRequest( + unsignedRegisterSubnetValidator, + nil, + ) + require.NoError(err) + + require.True(genesisPeer.Send(tc.DefaultContext(), registerSubnetValidatorRequest)) + }) + + tc.By("getting the signature response") + registerSubnetValidatorSignature, ok, err := findMessage(genesisPeerMessages, unwrapWarpSignature) + require.NoError(err) + require.True(ok) + + tc.By("creating the signed warp message to register the validator") + registerSubnetValidator, err := warp.NewMessage( + unsignedRegisterSubnetValidator, + &warp.BitSetSignature{ + Signers: set.NewBits(0).Bytes(), // [signers] has weight from the genesis peer + Signature: ([bls.SignatureLen]byte)( + bls.SignatureToBytes(registerSubnetValidatorSignature), + ), + }, + ) + require.NoError(err) + + tc.By("issuing a RegisterSubnetValidatorTx", func() { + _, err := pWallet.IssueRegisterSubnetValidatorTx( + registerBalance, + registerNodePoP.ProofOfPossession, + registerSubnetValidator.Bytes(), + ) + require.NoError(err) + }) + }) + + tc.By("verifying the validator was registered", func() { + tc.By("verifying the validator set was updated", func() { + verifyValidatorSet(map[ids.NodeID]*snowvalidators.GetValidatorOutput{ + subnetGenesisNode.NodeID: { + NodeID: subnetGenesisNode.NodeID, + PublicKey: genesisNodePK, + Weight: genesisWeight, + }, + ids.EmptyNodeID: { // The validator is not active + NodeID: ids.EmptyNodeID, + Weight: registerWeight, + }, + }) + }) + }) + + genesisPeerMessages.Close() + genesisPeer.StartClose() + require.NoError(genesisPeer.AwaitClosed(tc.DefaultContext())) + _ = e2e.CheckBootstrapIsPossible(tc, env.GetNetwork()) }) }) + +func wrapWarpSignatureRequest( + msg *warp.UnsignedMessage, + justification []byte, +) (p2pmessage.OutboundMessage, error) { + p2pMessageFactory, err := p2pmessage.NewCreator( + logging.NoLog{}, + prometheus.NewRegistry(), + constants.DefaultNetworkCompressionType, + p2pTimeout, + ) + if err != nil { + return nil, err + } + + request := sdk.SignatureRequest{ + Message: msg.Bytes(), + Justification: justification, + } + requestBytes, err := proto.Marshal(&request) + if err != nil { + return nil, err + } + + return p2pMessageFactory.AppRequest( + msg.SourceChainID, + 0, + time.Hour, + p2psdk.PrefixMessage( + p2psdk.ProtocolPrefix(p2psdk.SignatureRequestHandlerID), + requestBytes, + ), + ) +} + +func findMessage[T any]( + q buffer.BlockingDeque[p2pmessage.InboundMessage], + parser func(p2pmessage.InboundMessage) (T, bool, error), +) (T, bool, error) { + var messagesToReprocess []p2pmessage.InboundMessage + defer func() { + slices.Reverse(messagesToReprocess) + for _, msg := range messagesToReprocess { + q.PushLeft(msg) + } + }() + + for { + msg, ok := q.PopLeft() + if !ok { + return utils.Zero[T](), false, nil + } + + parsed, ok, err := parser(msg) + if err != nil { + return utils.Zero[T](), false, err + } + if ok { + return parsed, true, nil + } + + messagesToReprocess = append(messagesToReprocess, msg) + } +} + +// unwrapWarpSignature assumes the only type of AppResponses that will be +// received are ACP-118 compliant responses. +func unwrapWarpSignature(msg p2pmessage.InboundMessage) (*bls.Signature, bool, error) { + var appResponse *p2ppb.AppResponse + switch msg := msg.Message().(type) { + case *p2ppb.AppResponse: + appResponse = msg + case *p2ppb.AppError: + return nil, false, errors.New(msg.ErrorMessage) + default: + return nil, false, nil + } + + var response sdk.SignatureResponse + if err := proto.Unmarshal(appResponse.AppBytes, &response); err != nil { + return nil, false, err + } + + warpSignature, err := bls.SignatureFromBytes(response.Signature) + return warpSignature, true, err +} + +func must[T any](t require.TestingT) func(T, error) T { + return func(val T, err error) T { + require.NoError(t, err) + return val + } +} diff --git a/vms/platformvm/metrics/tx_metrics.go b/vms/platformvm/metrics/tx_metrics.go index 7957cb6ab2e9..1da84206cf09 100644 --- a/vms/platformvm/metrics/tx_metrics.go +++ b/vms/platformvm/metrics/tx_metrics.go @@ -145,3 +145,10 @@ func (m *txMetrics) ConvertSubnetTx(*txs.ConvertSubnetTx) error { }).Inc() return nil } + +func (m *txMetrics) RegisterSubnetValidatorTx(*txs.RegisterSubnetValidatorTx) error { + m.numTxs.With(prometheus.Labels{ + txLabel: "register_subnet_validator", + }).Inc() + return nil +} diff --git a/vms/platformvm/signer/proof_of_possession.go b/vms/platformvm/signer/proof_of_possession.go index 245d2a96c6f0..f63365d985f4 100644 --- a/vms/platformvm/signer/proof_of_possession.go +++ b/vms/platformvm/signer/proof_of_possession.go @@ -14,7 +14,7 @@ import ( var ( _ Signer = (*ProofOfPossession)(nil) - errInvalidProofOfPossession = errors.New("invalid proof of possession") + ErrInvalidProofOfPossession = errors.New("invalid proof of possession") ) type ProofOfPossession struct { @@ -52,7 +52,7 @@ func (p *ProofOfPossession) Verify() error { return err } if !bls.VerifyProofOfPossession(publicKey, signature, p.PublicKey[:]) { - return errInvalidProofOfPossession + return ErrInvalidProofOfPossession } p.publicKey = publicKey diff --git a/vms/platformvm/signer/proof_of_possession_test.go b/vms/platformvm/signer/proof_of_possession_test.go index 9f4f3feefa3c..9674d63985fc 100644 --- a/vms/platformvm/signer/proof_of_possession_test.go +++ b/vms/platformvm/signer/proof_of_possession_test.go @@ -35,7 +35,7 @@ func TestProofOfPossession(t *testing.T) { require.NoError(err) newBLSPOP.ProofOfPossession = blsPOP.ProofOfPossession err = newBLSPOP.Verify() - require.ErrorIs(err, errInvalidProofOfPossession) + require.ErrorIs(err, ErrInvalidProofOfPossession) } func TestNewProofOfPossessionDeterministic(t *testing.T) { diff --git a/vms/platformvm/txs/codec.go b/vms/platformvm/txs/codec.go index 123da91b8dea..4c3892753555 100644 --- a/vms/platformvm/txs/codec.go +++ b/vms/platformvm/txs/codec.go @@ -123,5 +123,6 @@ func RegisterDurangoTypes(targetCodec linearcodec.Codec) error { func RegisterEtnaTypes(targetCodec linearcodec.Codec) error { return errors.Join( targetCodec.RegisterType(&ConvertSubnetTx{}), + targetCodec.RegisterType(&RegisterSubnetValidatorTx{}), ) } diff --git a/vms/platformvm/txs/executor/atomic_tx_executor.go b/vms/platformvm/txs/executor/atomic_tx_executor.go index 1977608d09c1..95c56b24c4e6 100644 --- a/vms/platformvm/txs/executor/atomic_tx_executor.go +++ b/vms/platformvm/txs/executor/atomic_tx_executor.go @@ -112,6 +112,10 @@ func (*atomicTxExecutor) ConvertSubnetTx(*txs.ConvertSubnetTx) error { return ErrWrongTxType } +func (*atomicTxExecutor) RegisterSubnetValidatorTx(*txs.RegisterSubnetValidatorTx) error { + return ErrWrongTxType +} + func (e *atomicTxExecutor) ImportTx(*txs.ImportTx) error { return e.atomicTx() } diff --git a/vms/platformvm/txs/executor/proposal_tx_executor.go b/vms/platformvm/txs/executor/proposal_tx_executor.go index 03b03c889a23..aa2e949b312f 100644 --- a/vms/platformvm/txs/executor/proposal_tx_executor.go +++ b/vms/platformvm/txs/executor/proposal_tx_executor.go @@ -131,6 +131,10 @@ func (*proposalTxExecutor) ConvertSubnetTx(*txs.ConvertSubnetTx) error { return ErrWrongTxType } +func (*proposalTxExecutor) RegisterSubnetValidatorTx(*txs.RegisterSubnetValidatorTx) error { + return ErrWrongTxType +} + func (e *proposalTxExecutor) AddValidatorTx(tx *txs.AddValidatorTx) error { // AddValidatorTx is a proposal transaction until the Banff fork // activation. Following the activation, AddValidatorTxs must be issued into diff --git a/vms/platformvm/txs/executor/standard_tx_executor.go b/vms/platformvm/txs/executor/standard_tx_executor.go index 3ccc059bfa8e..b64972452eba 100644 --- a/vms/platformvm/txs/executor/standard_tx_executor.go +++ b/vms/platformvm/txs/executor/standard_tx_executor.go @@ -4,6 +4,7 @@ package executor import ( + "bytes" "context" "errors" "fmt" @@ -20,21 +21,40 @@ import ( "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/components/gas" "github.com/ava-labs/avalanchego/vms/components/verify" + "github.com/ava-labs/avalanchego/vms/platformvm/signer" "github.com/ava-labs/avalanchego/vms/platformvm/state" "github.com/ava-labs/avalanchego/vms/platformvm/txs" "github.com/ava-labs/avalanchego/vms/platformvm/txs/fee" + "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/vms/platformvm/warp/message" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" +) + +// TODO: Before Etna, ensure that the maximum number of expiries to track is +// limited to a reasonable number by this window. +const ( + second = 1 + minute = 60 * second + hour = 60 * minute + day = 24 * hour + RegisterSubnetValidatorTxExpiryWindow = day ) var ( _ txs.Visitor = (*standardTxExecutor)(nil) - errEmptyNodeID = errors.New("validator nodeID cannot be empty") - errMaxStakeDurationTooLarge = errors.New("max stake duration must be less than or equal to the global max stake duration") - errMissingStartTimePreDurango = errors.New("staker transactions must have a StartTime pre-Durango") - errEtnaUpgradeNotActive = errors.New("attempting to use an Etna-upgrade feature prior to activation") - errTransformSubnetTxPostEtna = errors.New("TransformSubnetTx is not permitted post-Etna") - errMaxNumActiveValidators = errors.New("already at the max number of active validators") + errEmptyNodeID = errors.New("validator nodeID cannot be empty") + errMaxStakeDurationTooLarge = errors.New("max stake duration must be less than or equal to the global max stake duration") + errMissingStartTimePreDurango = errors.New("staker transactions must have a StartTime pre-Durango") + errEtnaUpgradeNotActive = errors.New("attempting to use an Etna-upgrade feature prior to activation") + errTransformSubnetTxPostEtna = errors.New("TransformSubnetTx is not permitted post-Etna") + errMaxNumActiveValidators = errors.New("already at the max number of active validators") + errCouldNotLoadSubnetConversion = errors.New("could not load subnet conversion") + errWrongWarpMessageSourceChainID = errors.New("wrong warp message source chain ID") + errWrongWarpMessageSourceAddress = errors.New("wrong warp message source address") + errWarpMessageExpired = errors.New("warp message expired") + errWarpMessageNotYetAllowed = errors.New("warp message not yet allowed") + errWarpMessageAlreadyIssued = errors.New("warp message already issued") ) // StandardTx executes the standard transaction [tx]. @@ -780,6 +800,165 @@ func (e *standardTxExecutor) ConvertSubnetTx(tx *txs.ConvertSubnetTx) error { return nil } +func (e *standardTxExecutor) RegisterSubnetValidatorTx(tx *txs.RegisterSubnetValidatorTx) error { + var ( + currentTimestamp = e.state.GetTimestamp() + upgrades = e.backend.Config.UpgradeConfig + ) + if !upgrades.IsEtnaActivated(currentTimestamp) { + return errEtnaUpgradeNotActive + } + + if err := e.tx.SyntacticVerify(e.backend.Ctx); err != nil { + return err + } + + if err := avax.VerifyMemoFieldLength(tx.Memo, true /*=isDurangoActive*/); err != nil { + return err + } + + // Verify the flowcheck + fee, err := e.feeCalculator.CalculateFee(tx) + if err != nil { + return err + } + fee, err = math.Add(fee, tx.Balance) + if err != nil { + return err + } + + if err := e.backend.FlowChecker.VerifySpend( + tx, + e.state, + tx.Ins, + tx.Outs, + e.tx.Creds, + map[ids.ID]uint64{ + e.backend.Ctx.AVAXAssetID: fee, + }, + ); err != nil { + return err + } + + // Parse the warp message. + warpMessage, err := warp.ParseMessage(tx.Message) + if err != nil { + return err + } + addressedCall, err := payload.ParseAddressedCall(warpMessage.Payload) + if err != nil { + return err + } + msg, err := message.ParseRegisterSubnetValidator(addressedCall.Payload) + if err != nil { + return err + } + if err := msg.Verify(); err != nil { + return err + } + + // Verify that the warp message was sent from the expected chain and + // address. + subnetConversion, err := e.state.GetSubnetConversion(msg.SubnetID) + if err != nil { + return fmt.Errorf("%w for %s with: %w", errCouldNotLoadSubnetConversion, msg.SubnetID, err) + } + if warpMessage.SourceChainID != subnetConversion.ChainID { + return fmt.Errorf("%w expected %s but had %s", errWrongWarpMessageSourceChainID, subnetConversion.ChainID, warpMessage.SourceChainID) + } + if !bytes.Equal(addressedCall.SourceAddress, subnetConversion.Addr) { + return fmt.Errorf("%w expected 0x%x but got 0x%x", errWrongWarpMessageSourceAddress, subnetConversion.Addr, addressedCall.SourceAddress) + } + + // Verify that the message contains a valid expiry time. + currentTimestampUnix := uint64(currentTimestamp.Unix()) + if msg.Expiry <= currentTimestampUnix { + return fmt.Errorf("%w at %d and it is currently %d", errWarpMessageExpired, msg.Expiry, currentTimestampUnix) + } + if secondsUntilExpiry := msg.Expiry - currentTimestampUnix; secondsUntilExpiry > RegisterSubnetValidatorTxExpiryWindow { + return fmt.Errorf("%w because time is %d seconds in the future but the limit is %d", errWarpMessageNotYetAllowed, secondsUntilExpiry, RegisterSubnetValidatorTxExpiryWindow) + } + + // Verify that this warp message isn't being replayed. + validationID := msg.ValidationID() + expiry := state.ExpiryEntry{ + Timestamp: msg.Expiry, + ValidationID: validationID, + } + isDuplicate, err := e.state.HasExpiry(expiry) + if err != nil { + return err + } + if isDuplicate { + return fmt.Errorf("%w for validationID %s", errWarpMessageAlreadyIssued, validationID) + } + + // Verify proof of possession provided by the transaction against the public + // key provided by the warp message. + pop := signer.ProofOfPossession{ + PublicKey: msg.BLSPublicKey, + ProofOfPossession: tx.ProofOfPossession, + } + if err := pop.Verify(); err != nil { + return err + } + + // Create the SoV. + nodeID, err := ids.ToNodeID(msg.NodeID) + if err != nil { + return err + } + remainingBalanceOwner, err := txs.Codec.Marshal(txs.CodecVersion, &msg.RemainingBalanceOwner) + if err != nil { + return err + } + deactivationOwner, err := txs.Codec.Marshal(txs.CodecVersion, &msg.DisableOwner) + if err != nil { + return err + } + sov := state.SubnetOnlyValidator{ + ValidationID: validationID, + SubnetID: msg.SubnetID, + NodeID: nodeID, + PublicKey: bls.PublicKeyToUncompressedBytes(pop.Key()), + RemainingBalanceOwner: remainingBalanceOwner, + DeactivationOwner: deactivationOwner, + StartTime: currentTimestampUnix, + Weight: msg.Weight, + MinNonce: 0, + EndAccumulatedFee: 0, // If Balance is 0, this is will remain 0 + } + + // If the balance is non-zero, this validator should be initially active. + if tx.Balance != 0 { + // Verify that there is space for an active validator. + if gas.Gas(e.state.NumActiveSubnetOnlyValidators()) >= e.backend.Config.ValidatorFeeConfig.Capacity { + return errMaxNumActiveValidators + } + + // Mark the validator as active. + currentFees := e.state.GetAccruedFees() + sov.EndAccumulatedFee, err = math.Add(tx.Balance, currentFees) + if err != nil { + return err + } + } + + if err := e.state.PutSubnetOnlyValidator(sov); err != nil { + return err + } + + txID := e.tx.ID() + + // Consume the UTXOS + avax.Consume(e.state, tx.Ins) + // Produce the UTXOS + avax.Produce(e.state, txID, tx.Outs) + // Prevent this warp message from being replayed + e.state.PutExpiry(expiry) + return nil +} + // Creates the staker as defined in [stakerTx] and adds it to [e.State]. func (e *standardTxExecutor) putStaker(stakerTx txs.Staker) error { var ( diff --git a/vms/platformvm/txs/executor/standard_tx_executor_test.go b/vms/platformvm/txs/executor/standard_tx_executor_test.go index 8a2e574ccfc5..c392e2dfe030 100644 --- a/vms/platformvm/txs/executor/standard_tx_executor_test.go +++ b/vms/platformvm/txs/executor/standard_tx_executor_test.go @@ -13,6 +13,7 @@ import ( "github.com/stretchr/testify/require" "go.uber.org/mock/gomock" + "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/genesis" "github.com/ava-labs/avalanchego/ids" @@ -25,6 +26,7 @@ import ( "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" "github.com/ava-labs/avalanchego/utils/hashing" "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/components/verify" @@ -40,10 +42,13 @@ import ( "github.com/ava-labs/avalanchego/vms/platformvm/txs/txstest" "github.com/ava-labs/avalanchego/vms/platformvm/utxo" "github.com/ava-labs/avalanchego/vms/platformvm/utxo/utxomock" + "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/vms/platformvm/warp/message" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" "github.com/ava-labs/avalanchego/vms/secp256k1fx" "github.com/ava-labs/avalanchego/wallet/subnet/primary/common" + safemath "github.com/ava-labs/avalanchego/utils/math" txfee "github.com/ava-labs/avalanchego/vms/platformvm/txs/fee" validatorfee "github.com/ava-labs/avalanchego/vms/platformvm/validators/fee" ) @@ -2670,3 +2675,542 @@ func TestStandardExecutorConvertSubnetTx(t *testing.T) { }) } } + +func TestStandardExecutorRegisterSubnetValidatorTx(t *testing.T) { + var ( + fx = &secp256k1fx.Fx{} + vm = &secp256k1fx.TestVM{ + Log: logging.NoLog{}, + } + ) + require.NoError(t, fx.InitializeVM(vm)) + + var ( + ctx = snowtest.Context(t, constants.PlatformChainID) + defaultConfig = &config.Config{ + DynamicFeeConfig: genesis.LocalParams.DynamicFeeConfig, + ValidatorFeeConfig: genesis.LocalParams.ValidatorFeeConfig, + UpgradeConfig: upgradetest.GetConfig(upgradetest.Latest), + } + baseState = statetest.New(t, statetest.Config{ + Upgrades: defaultConfig.UpgradeConfig, + Context: ctx, + }) + wallet = txstest.NewWallet( + t, + ctx, + defaultConfig, + baseState, + secp256k1fx.NewKeychain(genesistest.DefaultFundedKeys...), + nil, // subnetIDs + nil, // chainIDs + ) + flowChecker = utxo.NewVerifier( + ctx, + &vm.Clk, + fx, + ) + + backend = &Backend{ + Config: defaultConfig, + Bootstrapped: utils.NewAtomic(true), + Fx: fx, + FlowChecker: flowChecker, + Ctx: ctx, + } + feeCalculator = state.PickFeeCalculator(defaultConfig, baseState) + ) + + // Create the initial state + diff, err := state.NewDiffOn(baseState) + require.NoError(t, err) + + // Create the subnet + createSubnetTx, err := wallet.IssueCreateSubnetTx( + &secp256k1fx.OutputOwners{}, + ) + require.NoError(t, err) + + // Execute the subnet creation + _, _, _, err = StandardTx( + backend, + feeCalculator, + createSubnetTx, + diff, + ) + require.NoError(t, err) + + // Create the subnet conversion + initialSK, err := bls.NewSecretKey() + require.NoError(t, err) + + const ( + initialWeight = 1 + initialBalance = units.Avax + ) + var ( + subnetID = createSubnetTx.ID() + chainID = ids.GenerateTestID() + address = utils.RandomBytes(32) + initialNodeID = ids.GenerateTestNodeID() + initialPoP = signer.NewProofOfPossession(initialSK) + validator = &txs.ConvertSubnetValidator{ + NodeID: initialNodeID.Bytes(), + Weight: initialWeight, + Balance: initialBalance, + Signer: *initialPoP, + RemainingBalanceOwner: message.PChainOwner{}, + DeactivationOwner: message.PChainOwner{}, + } + ) + convertSubnetTx, err := wallet.IssueConvertSubnetTx( + subnetID, + chainID, + address, + []*txs.ConvertSubnetValidator{ + validator, + }, + ) + require.NoError(t, err) + + // Execute the subnet conversion + _, _, _, err = StandardTx( + backend, + feeCalculator, + convertSubnetTx, + diff, + ) + require.NoError(t, err) + require.NoError(t, diff.Apply(baseState)) + require.NoError(t, baseState.Commit()) + + var ( + nodeID = ids.GenerateTestNodeID() + lastAcceptedTime = baseState.GetTimestamp() + expiryTime = lastAcceptedTime.Add(5 * time.Minute) + expiry = uint64(expiryTime.Unix()) // The warp message will expire in 5 minutes + ) + + const weight = 1 + + // Create the Warp message + sk, err := bls.NewSecretKey() + require.NoError(t, err) + pop := signer.NewProofOfPossession(sk) + pk := bls.PublicFromSecretKey(sk) + pkBytes := bls.PublicKeyToUncompressedBytes(pk) + + remainingBalanceOwner := message.PChainOwner{} + remainingBalanceOwnerBytes, err := txs.Codec.Marshal(txs.CodecVersion, &remainingBalanceOwner) + require.NoError(t, err) + + deactivationOwner := message.PChainOwner{} + deactivationOwnerBytes, err := txs.Codec.Marshal(txs.CodecVersion, &deactivationOwner) + require.NoError(t, err) + + addressedCallPayload := must[*message.RegisterSubnetValidator](t)(message.NewRegisterSubnetValidator( + subnetID, + nodeID, + pop.PublicKey, + expiry, + remainingBalanceOwner, + deactivationOwner, + weight, + )) + unsignedWarp := must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + addressedCallPayload.Bytes(), + )).Bytes(), + )) + warpSignature := &warp.BitSetSignature{ + Signers: set.NewBits(0).Bytes(), + Signature: ([bls.SignatureLen]byte)(bls.SignatureToBytes( + bls.Sign( + sk, + unsignedWarp.Bytes(), + ), + )), + } + warpMessage := must[*warp.Message](t)(warp.NewMessage( + unsignedWarp, + warpSignature, + )) + + validationID := addressedCallPayload.ValidationID() + tests := []struct { + name string + balance uint64 + message []byte + builderOptions []common.Option + updateTx func(*txs.RegisterSubnetValidatorTx) + updateExecutor func(*standardTxExecutor) error + expectedErr error + }{ + { + name: "invalid prior to E-Upgrade", + updateExecutor: func(e *standardTxExecutor) error { + e.backend.Config = &config.Config{ + UpgradeConfig: upgradetest.GetConfig(upgradetest.Durango), + } + return nil + }, + expectedErr: errEtnaUpgradeNotActive, + }, + { + name: "tx fails syntactic verification", + updateExecutor: func(e *standardTxExecutor) error { + e.backend.Ctx = snowtest.Context(t, ids.GenerateTestID()) + return nil + }, + expectedErr: avax.ErrWrongChainID, + }, + { + name: "invalid memo length", + builderOptions: []common.Option{ + common.WithMemo([]byte("memo!")), + }, + expectedErr: avax.ErrMemoTooLarge, + }, + { + name: "invalid fee calculation", + updateExecutor: func(e *standardTxExecutor) error { + e.feeCalculator = txfee.NewStaticCalculator(e.backend.Config.StaticFeeConfig) + return nil + }, + expectedErr: txfee.ErrUnsupportedTx, + }, + { + name: "fee calculation overflow", + updateTx: func(tx *txs.RegisterSubnetValidatorTx) { + tx.Balance = math.MaxUint64 + }, + expectedErr: safemath.ErrOverflow, + }, + { + name: "insufficient fee", + updateExecutor: func(e *standardTxExecutor) error { + e.feeCalculator = txfee.NewDynamicCalculator( + e.backend.Config.DynamicFeeConfig.Weights, + 100*genesis.LocalParams.DynamicFeeConfig.MinPrice, + ) + return nil + }, + expectedErr: utxo.ErrInsufficientUnlockedFunds, + }, + { + name: "invalid warp message", + message: []byte{}, + expectedErr: codec.ErrCantUnpackVersion, + }, + { + name: "invalid warp payload", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.Hash](t)(payload.NewHash(ids.Empty)).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: payload.ErrWrongType, + }, + { + name: "invalid addressed call", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + must[*message.SubnetConversion](t)(message.NewSubnetConversion(ids.Empty)).Bytes(), + )).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: message.ErrWrongType, + }, + { + name: "invalid addressed call payload", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + must[*message.RegisterSubnetValidator](t)(message.NewRegisterSubnetValidator( + subnetID, + nodeID, + pop.PublicKey, + expiry, + remainingBalanceOwner, + deactivationOwner, + 0, // weight = 0 is invalid + )).Bytes(), + )).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: message.ErrInvalidWeight, + }, + { + name: "subnet conversion not found", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + must[*message.RegisterSubnetValidator](t)(message.NewRegisterSubnetValidator( + ids.GenerateTestID(), // invalid subnetID + nodeID, + pop.PublicKey, + expiry, + remainingBalanceOwner, + deactivationOwner, + weight, + )).Bytes(), + )).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: errCouldNotLoadSubnetConversion, + }, + { + name: "invalid source chain", + updateExecutor: func(e *standardTxExecutor) error { + e.state.SetSubnetConversion(subnetID, state.SubnetConversion{}) + return nil + }, + expectedErr: errWrongWarpMessageSourceChainID, + }, + { + name: "invalid source address", + updateExecutor: func(e *standardTxExecutor) error { + e.state.SetSubnetConversion(subnetID, state.SubnetConversion{ + ChainID: chainID, + }) + return nil + }, + expectedErr: errWrongWarpMessageSourceAddress, + }, + { + name: "message expired", + updateExecutor: func(e *standardTxExecutor) error { + e.state.SetTimestamp(expiryTime) + return nil + }, + expectedErr: errWarpMessageExpired, + }, + { + name: "message expiry too far in the future", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + must[*message.RegisterSubnetValidator](t)(message.NewRegisterSubnetValidator( + subnetID, + nodeID, + pop.PublicKey, + math.MaxUint64, // expiry too far in the future + remainingBalanceOwner, + deactivationOwner, + weight, + )).Bytes(), + )).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: errWarpMessageNotYetAllowed, + }, + { + name: "SoV previously registered", + balance: 1, + updateExecutor: func(e *standardTxExecutor) error { + e.state.PutExpiry(state.ExpiryEntry{ + Timestamp: expiry, + ValidationID: validationID, + }) + return nil + }, + expectedErr: errWarpMessageAlreadyIssued, + }, + { + name: "invalid PoP", + message: must[*warp.Message](t)(warp.NewMessage( + must[*warp.UnsignedMessage](t)(warp.NewUnsignedMessage( + ctx.NetworkID, + chainID, + must[*payload.AddressedCall](t)(payload.NewAddressedCall( + address, + must[*message.RegisterSubnetValidator](t)(message.NewRegisterSubnetValidator( + subnetID, + nodeID, + initialPoP.PublicKey, // Wrong public key + expiry, + remainingBalanceOwner, + deactivationOwner, + weight, + )).Bytes(), + )).Bytes(), + )), + warpSignature, + )).Bytes(), + expectedErr: signer.ErrInvalidProofOfPossession, + }, + { + name: "too many active validators", + balance: 1, + updateExecutor: func(e *standardTxExecutor) error { + e.backend.Config = &config.Config{ + DynamicFeeConfig: genesis.LocalParams.DynamicFeeConfig, + ValidatorFeeConfig: validatorfee.Config{ + Capacity: 0, + Target: genesis.LocalParams.ValidatorFeeConfig.Target, + MinPrice: genesis.LocalParams.ValidatorFeeConfig.MinPrice, + ExcessConversionConstant: genesis.LocalParams.ValidatorFeeConfig.ExcessConversionConstant, + }, + UpgradeConfig: upgradetest.GetConfig(upgradetest.Latest), + } + return nil + }, + expectedErr: errMaxNumActiveValidators, + }, + { + name: "accrued fees overflow", + balance: 1, + updateExecutor: func(e *standardTxExecutor) error { + e.state.SetAccruedFees(math.MaxUint64) + return nil + }, + expectedErr: safemath.ErrOverflow, + }, + { + name: "duplicate subnetID + nodeID pair", + updateExecutor: func(e *standardTxExecutor) error { + return e.state.PutSubnetOnlyValidator(state.SubnetOnlyValidator{ + ValidationID: ids.GenerateTestID(), + SubnetID: subnetID, + NodeID: nodeID, + PublicKey: bls.PublicKeyToUncompressedBytes(bls.PublicFromSecretKey(initialSK)), + Weight: 1, + }) + }, + expectedErr: state.ErrDuplicateSubnetOnlyValidator, + }, + { + name: "valid tx", + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + require := require.New(t) + + // Create the RegisterSubnetValidatorTx + wallet := txstest.NewWallet( + t, + ctx, + defaultConfig, + baseState, + secp256k1fx.NewKeychain(genesistest.DefaultFundedKeys...), + nil, // subnetIDs + nil, // chainIDs + ) + + message := test.message + if message == nil { + message = warpMessage.Bytes() + } + registerSubnetValidatorTx, err := wallet.IssueRegisterSubnetValidatorTx( + test.balance, + pop.ProofOfPossession, + message, + test.builderOptions..., + ) + require.NoError(err) + + if test.updateTx != nil { + unsignedTx := registerSubnetValidatorTx.Unsigned.(*txs.RegisterSubnetValidatorTx) + test.updateTx(unsignedTx) + } + + diff, err := state.NewDiffOn(baseState) + require.NoError(err) + + executor := &standardTxExecutor{ + backend: &Backend{ + Config: defaultConfig, + Bootstrapped: utils.NewAtomic(true), + Fx: fx, + FlowChecker: flowChecker, + Ctx: ctx, + }, + feeCalculator: state.PickFeeCalculator(defaultConfig, baseState), + tx: registerSubnetValidatorTx, + state: diff, + } + if test.updateExecutor != nil { + require.NoError(test.updateExecutor(executor)) + } + + err = registerSubnetValidatorTx.Unsigned.Visit(executor) + require.ErrorIs(err, test.expectedErr) + if err != nil { + return + } + + for utxoID := range registerSubnetValidatorTx.InputIDs() { + _, err := diff.GetUTXO(utxoID) + require.ErrorIs(err, database.ErrNotFound) + } + + for _, expectedUTXO := range registerSubnetValidatorTx.UTXOs() { + utxoID := expectedUTXO.InputID() + utxo, err := diff.GetUTXO(utxoID) + require.NoError(err) + require.Equal(expectedUTXO, utxo) + } + + sov, err := diff.GetSubnetOnlyValidator(validationID) + require.NoError(err) + + var expectedEndAccumulatedFee uint64 + if test.balance != 0 { + expectedEndAccumulatedFee = test.balance + diff.GetAccruedFees() + } + require.Equal( + state.SubnetOnlyValidator{ + ValidationID: validationID, + SubnetID: subnetID, + NodeID: nodeID, + PublicKey: pkBytes, + RemainingBalanceOwner: remainingBalanceOwnerBytes, + DeactivationOwner: deactivationOwnerBytes, + StartTime: uint64(diff.GetTimestamp().Unix()), + Weight: weight, + MinNonce: 0, + EndAccumulatedFee: expectedEndAccumulatedFee, + }, + sov, + ) + + hasExpiry, err := diff.HasExpiry(state.ExpiryEntry{ + Timestamp: expiry, + ValidationID: validationID, + }) + require.NoError(err) + require.True(hasExpiry) + }) + } +} + +func must[T any](t require.TestingT) func(T, error) T { + return func(val T, err error) T { + require.NoError(t, err) + return val + } +} diff --git a/vms/platformvm/txs/fee/calculator_test.go b/vms/platformvm/txs/fee/calculator_test.go index 5039dc557345..dd71649fecd3 100644 --- a/vms/platformvm/txs/fee/calculator_test.go +++ b/vms/platformvm/txs/fee/calculator_test.go @@ -231,5 +231,17 @@ var ( }, expectedDynamicFee: 365_600, }, + { + name: "RegisterSubnetValidatorTx", + tx: "00000000002400003039000000000000000000000000000000000000000000000000000000000000000000000001dbcf890f77f49b96857648b72b77f9f82937f28a68704af05da0dc12ba53f2db00000007002386f1f88b552a000000000000000000000001000000013cb7d3842e8cee6a0ebd09f1fe884f6861e1b29c00000001ca44ad45a63381b07074be7f82005c41550c989b967f40020f3bedc4b02191f300000000dbcf890f77f49b96857648b72b77f9f82937f28a68704af05da0dc12ba53f2db00000005002386f234262404000000010000000000000000000000003b9aca00ab5cb0516b7afdb13727f766185b2b8da44e2653eef63c85f196701083e649289cce1a23c39eb471b2473bc6872aa3ea190de0fe66296cbdd4132c92c3430ff22f28f0b341b15905a005bbd66cc0f4056bc4be5934e4f3a57151a60060f429190000012f000000003039705f3d4415f990225d3df5ce437d7af2aa324b1bbce854ee34ab6f39882250d20000009c000000000001000000000000008e000000000001a0673b4ee5ec44e57c8ab250dd7cd7b68d04421f64bd6559a4284a3ee358ff2b000000145efc86a11c5b12cc95b2cf527c023f9cf6e0e8f6b62034315c5d11cea4190f6ea8997821c02483d29adb5e4567843f7a44c39b2ffa20c8520dc358702fb1ec29f2746dcc000000006705af280000000000000000000000000000000000000000000000010000000000000001018e99dc6ed736089c03b9a1275e0cf801524ed341fb10111f29c0390fa2f96cf6aa78539ec767e5cd523c606c7ede50e60ba6065a3685e770d979b0df74e3541b61ed63f037463776098576e385767a695de59352b44e515831c5ee7a8cc728f9000000010000000900000001a0950b9e6e866130f0d09e2a7bfdd0246513295237258afa942b1850dab79824605c796bbfc9223cf91935fb29c66f8b927690220b9b1c24d6f078054a3e346201", + expectedStaticFeeErr: ErrUnsupportedTx, + expectedComplexity: gas.Dimensions{ + gas.Bandwidth: 710, // The length of the tx in bytes + gas.DBRead: IntrinsicRegisterSubnetValidatorTxComplexities[gas.DBRead] + intrinsicInputDBRead, + gas.DBWrite: IntrinsicRegisterSubnetValidatorTxComplexities[gas.DBWrite] + intrinsicInputDBWrite + intrinsicOutputDBWrite, + gas.Compute: 0, // TODO: implement + }, + expectedDynamicFee: 151_000, + }, } ) diff --git a/vms/platformvm/txs/fee/complexity.go b/vms/platformvm/txs/fee/complexity.go index e1891a3d51a1..5083296d2936 100644 --- a/vms/platformvm/txs/fee/complexity.go +++ b/vms/platformvm/txs/fee/complexity.go @@ -1,6 +1,8 @@ // Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. +// TODO: Before Etna, address all TODOs in this package and ensure ACP-103 +// compliance. package fee import ( @@ -196,6 +198,15 @@ var ( gas.DBWrite: 2, // manager + weight gas.Compute: 0, } + IntrinsicRegisterSubnetValidatorTxComplexities = gas.Dimensions{ + gas.Bandwidth: IntrinsicBaseTxComplexities[gas.Bandwidth] + + wrappers.LongLen + // balance + bls.SignatureLen + // proof of possession + wrappers.IntLen, // message length + gas.DBRead: 0, // TODO + gas.DBWrite: 0, // TODO + gas.Compute: 0, // TODO: Include PoP verification time + } errUnsupportedOutput = errors.New("unsupported output type") errUnsupportedInput = errors.New("unsupported input type") @@ -436,6 +447,14 @@ func SignerComplexity(s signer.Signer) (gas.Dimensions, error) { } } +// WarpComplexity returns the complexity a warp message adds to a transaction. +func WarpComplexity(message []byte) (gas.Dimensions, error) { + // TODO: Implement me + return gas.Dimensions{ + gas.Bandwidth: uint64(len(message)), + }, nil +} + type complexityVisitor struct { output gas.Dimensions } @@ -686,6 +705,22 @@ func (c *complexityVisitor) ConvertSubnetTx(tx *txs.ConvertSubnetTx) error { return err } +func (c *complexityVisitor) RegisterSubnetValidatorTx(tx *txs.RegisterSubnetValidatorTx) error { + baseTxComplexity, err := baseTxComplexity(&tx.BaseTx) + if err != nil { + return err + } + warpComplexity, err := WarpComplexity(tx.Message) + if err != nil { + return err + } + c.output, err = IntrinsicRegisterSubnetValidatorTxComplexities.Add( + &baseTxComplexity, + &warpComplexity, + ) + return err +} + func baseTxComplexity(tx *txs.BaseTx) (gas.Dimensions, error) { outputsComplexity, err := OutputComplexity(tx.Outs...) if err != nil { diff --git a/vms/platformvm/txs/fee/static_calculator.go b/vms/platformvm/txs/fee/static_calculator.go index 1b97349ce2cb..be4651cbb911 100644 --- a/vms/platformvm/txs/fee/static_calculator.go +++ b/vms/platformvm/txs/fee/static_calculator.go @@ -51,6 +51,10 @@ func (*staticVisitor) ConvertSubnetTx(*txs.ConvertSubnetTx) error { return ErrUnsupportedTx } +func (*staticVisitor) RegisterSubnetValidatorTx(*txs.RegisterSubnetValidatorTx) error { + return ErrUnsupportedTx +} + func (c *staticVisitor) AddValidatorTx(*txs.AddValidatorTx) error { c.fee = c.config.AddPrimaryNetworkValidatorFee return nil diff --git a/vms/platformvm/txs/register_subnet_validator_tx.go b/vms/platformvm/txs/register_subnet_validator_tx.go new file mode 100644 index 000000000000..a5a7a0192329 --- /dev/null +++ b/vms/platformvm/txs/register_subnet_validator_tx.go @@ -0,0 +1,45 @@ +// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package txs + +import ( + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/utils/crypto/bls" + "github.com/ava-labs/avalanchego/vms/types" +) + +var _ UnsignedTx = (*RegisterSubnetValidatorTx)(nil) + +type RegisterSubnetValidatorTx struct { + // Metadata, inputs and outputs + BaseTx `serialize:"true"` + // Balance <= sum($AVAX inputs) - sum($AVAX outputs) - TxFee. + Balance uint64 `serialize:"true" json:"balance"` + // ProofOfPossession of the BLS key that is included in the Message. + ProofOfPossession [bls.SignatureLen]byte `serialize:"true" json:"proofOfPossession"` + // Message is expected to be a signed Warp message containing an + // AddressedCall payload with the RegisterSubnetValidator message. + Message types.JSONByteSlice `serialize:"true" json:"message"` +} + +func (tx *RegisterSubnetValidatorTx) SyntacticVerify(ctx *snow.Context) error { + switch { + case tx == nil: + return ErrNilTx + case tx.SyntacticallyVerified: + // already passed syntactic verification + return nil + } + + if err := tx.BaseTx.SyntacticVerify(ctx); err != nil { + return err + } + + tx.SyntacticallyVerified = true + return nil +} + +func (tx *RegisterSubnetValidatorTx) Visit(visitor Visitor) error { + return visitor.RegisterSubnetValidatorTx(tx) +} diff --git a/vms/platformvm/txs/register_subnet_validator_tx_test.go b/vms/platformvm/txs/register_subnet_validator_tx_test.go new file mode 100644 index 000000000000..64055b6171e0 --- /dev/null +++ b/vms/platformvm/txs/register_subnet_validator_tx_test.go @@ -0,0 +1,393 @@ +// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package txs + +import ( + "encoding/hex" + "encoding/json" + "strings" + "testing" + + "github.com/stretchr/testify/require" + + _ "embed" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/snowtest" + "github.com/ava-labs/avalanchego/utils/constants" + "github.com/ava-labs/avalanchego/utils/crypto/bls" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/ava-labs/avalanchego/vms/components/avax" + "github.com/ava-labs/avalanchego/vms/platformvm/signer" + "github.com/ava-labs/avalanchego/vms/platformvm/stakeable" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + "github.com/ava-labs/avalanchego/vms/types" +) + +//go:embed register_subnet_validator_tx_test.json +var registerSubnetValidatorTxJSON []byte + +func TestRegisterSubnetValidatorTxSerialization(t *testing.T) { + require := require.New(t) + + const balance = units.Avax + + skBytes, err := hex.DecodeString("6668fecd4595b81e4d568398c820bbf3f073cb222902279fa55ebb84764ed2e3") + require.NoError(err) + sk, err := bls.SecretKeyFromBytes(skBytes) + require.NoError(err) + + var ( + pop = signer.NewProofOfPossession(sk) + message = []byte("message") + addr = ids.ShortID{ + 0x44, 0x55, 0x66, 0x77, 0x88, 0x99, 0xaa, 0xbb, + 0x44, 0x55, 0x66, 0x77, 0x88, 0x99, 0xaa, 0xbb, + 0x44, 0x55, 0x66, 0x77, + } + avaxAssetID = ids.ID{ + 0x21, 0xe6, 0x73, 0x17, 0xcb, 0xc4, 0xbe, 0x2a, + 0xeb, 0x00, 0x67, 0x7a, 0xd6, 0x46, 0x27, 0x78, + 0xa8, 0xf5, 0x22, 0x74, 0xb9, 0xd6, 0x05, 0xdf, + 0x25, 0x91, 0xb2, 0x30, 0x27, 0xa8, 0x7d, 0xff, + } + customAssetID = ids.ID{ + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + } + txID = ids.ID{ + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + } + ) + + var unsignedTx UnsignedTx = &RegisterSubnetValidatorTx{ + BaseTx: BaseTx{ + BaseTx: avax.BaseTx{ + NetworkID: constants.UnitTestID, + BlockchainID: constants.PlatformChainID, + Outs: []*avax.TransferableOutput{ + { + Asset: avax.Asset{ + ID: avaxAssetID, + }, + Out: &stakeable.LockOut{ + Locktime: 87654321, + TransferableOut: &secp256k1fx.TransferOutput{ + Amt: 1, + OutputOwners: secp256k1fx.OutputOwners{ + Locktime: 12345678, + Threshold: 0, + Addrs: []ids.ShortID{}, + }, + }, + }, + }, + { + Asset: avax.Asset{ + ID: customAssetID, + }, + Out: &stakeable.LockOut{ + Locktime: 876543210, + TransferableOut: &secp256k1fx.TransferOutput{ + Amt: 0xffffffffffffffff, + OutputOwners: secp256k1fx.OutputOwners{ + Locktime: 0, + Threshold: 1, + Addrs: []ids.ShortID{ + addr, + }, + }, + }, + }, + }, + }, + Ins: []*avax.TransferableInput{ + { + UTXOID: avax.UTXOID{ + TxID: txID, + OutputIndex: 1, + }, + Asset: avax.Asset{ + ID: avaxAssetID, + }, + In: &secp256k1fx.TransferInput{ + Amt: units.Avax, + Input: secp256k1fx.Input{ + SigIndices: []uint32{2, 5}, + }, + }, + }, + { + UTXOID: avax.UTXOID{ + TxID: txID, + OutputIndex: 2, + }, + Asset: avax.Asset{ + ID: customAssetID, + }, + In: &stakeable.LockIn{ + Locktime: 876543210, + TransferableIn: &secp256k1fx.TransferInput{ + Amt: 0xefffffffffffffff, + Input: secp256k1fx.Input{ + SigIndices: []uint32{0}, + }, + }, + }, + }, + { + UTXOID: avax.UTXOID{ + TxID: txID, + OutputIndex: 3, + }, + Asset: avax.Asset{ + ID: customAssetID, + }, + In: &secp256k1fx.TransferInput{ + Amt: 0x1000000000000000, + Input: secp256k1fx.Input{ + SigIndices: []uint32{}, + }, + }, + }, + }, + Memo: types.JSONByteSlice("😅\nwell that's\x01\x23\x45!"), + }, + }, + Balance: balance, + ProofOfPossession: pop.ProofOfPossession, + Message: message, + } + txBytes, err := Codec.Marshal(CodecVersion, &unsignedTx) + require.NoError(err) + + expectedBytes := []byte{ + // Codec version + 0x00, 0x00, + // RegisterSubnetValidatorTx Type ID + 0x00, 0x00, 0x00, 0x24, + // Network ID + 0x00, 0x00, 0x00, 0x0a, + // P-chain blockchain ID + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + // Number of outputs + 0x00, 0x00, 0x00, 0x02, + // Outputs[0] + // AVAX assetID + 0x21, 0xe6, 0x73, 0x17, 0xcb, 0xc4, 0xbe, 0x2a, + 0xeb, 0x00, 0x67, 0x7a, 0xd6, 0x46, 0x27, 0x78, + 0xa8, 0xf5, 0x22, 0x74, 0xb9, 0xd6, 0x05, 0xdf, + 0x25, 0x91, 0xb2, 0x30, 0x27, 0xa8, 0x7d, 0xff, + // Stakeable locked output type ID + 0x00, 0x00, 0x00, 0x16, + // Locktime + 0x00, 0x00, 0x00, 0x00, 0x05, 0x39, 0x7f, 0xb1, + // secp256k1fx transfer output type ID + 0x00, 0x00, 0x00, 0x07, + // amount + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01, + // secp256k1fx output locktime + 0x00, 0x00, 0x00, 0x00, 0x00, 0xbc, 0x61, 0x4e, + // threshold + 0x00, 0x00, 0x00, 0x00, + // number of addresses + 0x00, 0x00, 0x00, 0x00, + // Outputs[1] + // custom asset ID + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + // Stakeable locked output type ID + 0x00, 0x00, 0x00, 0x16, + // Locktime + 0x00, 0x00, 0x00, 0x00, 0x34, 0x3e, 0xfc, 0xea, + // secp256k1fx transfer output type ID + 0x00, 0x00, 0x00, 0x07, + // amount + 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, + // secp256k1fx output locktime + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + // threshold + 0x00, 0x00, 0x00, 0x01, + // number of addresses + 0x00, 0x00, 0x00, 0x01, + // address[0] + 0x44, 0x55, 0x66, 0x77, 0x88, 0x99, 0xaa, 0xbb, + 0x44, 0x55, 0x66, 0x77, 0x88, 0x99, 0xaa, 0xbb, + 0x44, 0x55, 0x66, 0x77, + // number of inputs + 0x00, 0x00, 0x00, 0x03, + // inputs[0] + // TxID + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + // Tx output index + 0x00, 0x00, 0x00, 0x01, + // AVAX assetID + 0x21, 0xe6, 0x73, 0x17, 0xcb, 0xc4, 0xbe, 0x2a, + 0xeb, 0x00, 0x67, 0x7a, 0xd6, 0x46, 0x27, 0x78, + 0xa8, 0xf5, 0x22, 0x74, 0xb9, 0xd6, 0x05, 0xdf, + 0x25, 0x91, 0xb2, 0x30, 0x27, 0xa8, 0x7d, 0xff, + // secp256k1fx transfer input type ID + 0x00, 0x00, 0x00, 0x05, + // input amount = 1 Avax + 0x00, 0x00, 0x00, 0x00, 0x3b, 0x9a, 0xca, 0x00, + // number of signatures needed in input + 0x00, 0x00, 0x00, 0x02, + // index of first signer + 0x00, 0x00, 0x00, 0x02, + // index of second signer + 0x00, 0x00, 0x00, 0x05, + // inputs[1] + // TxID + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + // Tx output index + 0x00, 0x00, 0x00, 0x02, + // Custom asset ID + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + // Stakeable locked input type ID + 0x00, 0x00, 0x00, 0x15, + // Locktime + 0x00, 0x00, 0x00, 0x00, 0x34, 0x3e, 0xfc, 0xea, + // secp256k1fx transfer input type ID + 0x00, 0x00, 0x00, 0x05, + // input amount + 0xef, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, + // number of signatures needed in input + 0x00, 0x00, 0x00, 0x01, + // index of signer + 0x00, 0x00, 0x00, 0x00, + // inputs[2] + // TxID + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + 0xff, 0xee, 0xdd, 0xcc, 0xbb, 0xaa, 0x99, 0x88, + // Tx output index + 0x00, 0x00, 0x00, 0x03, + // custom asset ID + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + 0x99, 0x77, 0x55, 0x77, 0x11, 0x33, 0x55, 0x31, + // secp256k1fx transfer input type ID + 0x00, 0x00, 0x00, 0x05, + // input amount + 0x10, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + // number of signatures needed in input + 0x00, 0x00, 0x00, 0x00, + // length of memo + 0x00, 0x00, 0x00, 0x14, + // memo + 0xf0, 0x9f, 0x98, 0x85, 0x0a, 0x77, 0x65, 0x6c, + 0x6c, 0x20, 0x74, 0x68, 0x61, 0x74, 0x27, 0x73, + 0x01, 0x23, 0x45, 0x21, + // balance + 0x00, 0x00, 0x00, 0x00, 0x3b, 0x9a, 0xca, 0x00, + // proof of possession + 0x8c, 0xfd, 0x79, 0x09, 0xd1, 0x53, 0xb9, 0x60, + 0x4b, 0x62, 0xb1, 0x43, 0xba, 0x36, 0x20, 0x7b, + 0xb7, 0xe6, 0x48, 0x67, 0x42, 0x44, 0x80, 0x20, + 0x2a, 0x67, 0xdc, 0x68, 0x76, 0x83, 0x46, 0xd9, + 0x5c, 0x90, 0x98, 0x3c, 0x2d, 0x27, 0x9c, 0x64, + 0xc4, 0x3c, 0x51, 0x13, 0x6b, 0x2a, 0x05, 0xe0, + 0x16, 0x02, 0xd5, 0x2a, 0xa6, 0x37, 0x6f, 0xda, + 0x17, 0xfa, 0x6e, 0x2a, 0x18, 0xa0, 0x83, 0xe4, + 0x9d, 0x9c, 0x45, 0x0e, 0xab, 0x7b, 0x89, 0xb1, + 0xd5, 0x55, 0x5d, 0xa5, 0xc4, 0x89, 0x87, 0x2e, + 0x02, 0xb7, 0xe5, 0x22, 0x7b, 0x77, 0x55, 0x0a, + 0xf1, 0x33, 0x0e, 0x5a, 0x71, 0xf8, 0xc3, 0x68, + // length of message + 0x00, 0x00, 0x00, 0x07, + // message + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + } + require.Equal(expectedBytes, txBytes) + + ctx := snowtest.Context(t, constants.PlatformChainID) + unsignedTx.InitCtx(ctx) + + txJSON, err := json.MarshalIndent(unsignedTx, "", "\t") + require.NoError(err) + require.Equal( + // Normalize newlines for Windows + strings.ReplaceAll(string(registerSubnetValidatorTxJSON), "\r\n", "\n"), + string(txJSON), + ) +} + +func TestRegisterSubnetValidatorTxSyntacticVerify(t *testing.T) { + ctx := snowtest.Context(t, ids.GenerateTestID()) + tests := []struct { + name string + tx *RegisterSubnetValidatorTx + expectedErr error + }{ + { + name: "nil tx", + tx: nil, + expectedErr: ErrNilTx, + }, + { + name: "already verified", + // The tx includes invalid data to verify that a cached result is + // returned. + tx: &RegisterSubnetValidatorTx{ + BaseTx: BaseTx{ + SyntacticallyVerified: true, + }, + }, + expectedErr: nil, + }, + { + name: "invalid BaseTx", + tx: &RegisterSubnetValidatorTx{ + BaseTx: BaseTx{}, + }, + expectedErr: avax.ErrWrongNetworkID, + }, + { + name: "passes verification", + tx: &RegisterSubnetValidatorTx{ + BaseTx: BaseTx{ + BaseTx: avax.BaseTx{ + NetworkID: ctx.NetworkID, + BlockchainID: ctx.ChainID, + }, + }, + }, + expectedErr: nil, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + require := require.New(t) + + err := test.tx.SyntacticVerify(ctx) + require.ErrorIs(err, test.expectedErr) + if test.expectedErr != nil { + return + } + require.True(test.tx.SyntacticallyVerified) + }) + } +} diff --git a/vms/platformvm/txs/register_subnet_validator_tx_test.json b/vms/platformvm/txs/register_subnet_validator_tx_test.json new file mode 100644 index 000000000000..f741e2fdbb37 --- /dev/null +++ b/vms/platformvm/txs/register_subnet_validator_tx_test.json @@ -0,0 +1,175 @@ +{ + "networkID": 10, + "blockchainID": "11111111111111111111111111111111LpoYY", + "outputs": [ + { + "assetID": "FvwEAhmxKfeiG8SnEvq42hc6whRyY3EFYAvebMqDNDGCgxN5Z", + "fxID": "spdxUxVJQbX85MGxMHbKw1sHxMnSqJ3QBzDyDYEP3h6TLuxqQ", + "output": { + "locktime": 87654321, + "output": { + "addresses": [], + "amount": 1, + "locktime": 12345678, + "threshold": 0 + } + } + }, + { + "assetID": "2Ab62uWwJw1T6VvmKD36ufsiuGZuX1pGykXAvPX1LtjTRHxwcc", + "fxID": "spdxUxVJQbX85MGxMHbKw1sHxMnSqJ3QBzDyDYEP3h6TLuxqQ", + "output": { + "locktime": 876543210, + "output": { + "addresses": [ + "P-testing1g32kvaugnx4tk3z4vemc3xd2hdz92enhgrdu9n" + ], + "amount": 18446744073709551615, + "locktime": 0, + "threshold": 1 + } + } + } + ], + "inputs": [ + { + "txID": "2wiU5PnFTjTmoAXGZutHAsPF36qGGyLHYHj9G1Aucfmb3JFFGN", + "outputIndex": 1, + "assetID": "FvwEAhmxKfeiG8SnEvq42hc6whRyY3EFYAvebMqDNDGCgxN5Z", + "fxID": "spdxUxVJQbX85MGxMHbKw1sHxMnSqJ3QBzDyDYEP3h6TLuxqQ", + "input": { + "amount": 1000000000, + "signatureIndices": [ + 2, + 5 + ] + } + }, + { + "txID": "2wiU5PnFTjTmoAXGZutHAsPF36qGGyLHYHj9G1Aucfmb3JFFGN", + "outputIndex": 2, + "assetID": "2Ab62uWwJw1T6VvmKD36ufsiuGZuX1pGykXAvPX1LtjTRHxwcc", + "fxID": "spdxUxVJQbX85MGxMHbKw1sHxMnSqJ3QBzDyDYEP3h6TLuxqQ", + "input": { + "locktime": 876543210, + "input": { + "amount": 17293822569102704639, + "signatureIndices": [ + 0 + ] + } + } + }, + { + "txID": "2wiU5PnFTjTmoAXGZutHAsPF36qGGyLHYHj9G1Aucfmb3JFFGN", + "outputIndex": 3, + "assetID": "2Ab62uWwJw1T6VvmKD36ufsiuGZuX1pGykXAvPX1LtjTRHxwcc", + "fxID": "spdxUxVJQbX85MGxMHbKw1sHxMnSqJ3QBzDyDYEP3h6TLuxqQ", + "input": { + "amount": 1152921504606846976, + "signatureIndices": [] + } + } + ], + "memo": "0xf09f98850a77656c6c2074686174277301234521", + "balance": 1000000000, + "proofOfPossession": [ + 140, + 253, + 121, + 9, + 209, + 83, + 185, + 96, + 75, + 98, + 177, + 67, + 186, + 54, + 32, + 123, + 183, + 230, + 72, + 103, + 66, + 68, + 128, + 32, + 42, + 103, + 220, + 104, + 118, + 131, + 70, + 217, + 92, + 144, + 152, + 60, + 45, + 39, + 156, + 100, + 196, + 60, + 81, + 19, + 107, + 42, + 5, + 224, + 22, + 2, + 213, + 42, + 166, + 55, + 111, + 218, + 23, + 250, + 110, + 42, + 24, + 160, + 131, + 228, + 157, + 156, + 69, + 14, + 171, + 123, + 137, + 177, + 213, + 85, + 93, + 165, + 196, + 137, + 135, + 46, + 2, + 183, + 229, + 34, + 123, + 119, + 85, + 10, + 241, + 51, + 14, + 90, + 113, + 248, + 195, + 104 + ], + "message": "0x6d657373616765" +} \ No newline at end of file diff --git a/vms/platformvm/txs/visitor.go b/vms/platformvm/txs/visitor.go index 21c46476fa5f..02627c198f16 100644 --- a/vms/platformvm/txs/visitor.go +++ b/vms/platformvm/txs/visitor.go @@ -28,4 +28,5 @@ type Visitor interface { // Etna Transactions: ConvertSubnetTx(*ConvertSubnetTx) error + RegisterSubnetValidatorTx(*RegisterSubnetValidatorTx) error } diff --git a/vms/platformvm/validators/manager.go b/vms/platformvm/validators/manager.go index bd6fecce96ef..be37e6ee73e2 100644 --- a/vms/platformvm/validators/manager.go +++ b/vms/platformvm/validators/manager.go @@ -26,10 +26,18 @@ import ( ) const ( - validatorSetsCacheSize = 64 - maxRecentlyAcceptedWindowSize = 64 - minRecentlyAcceptedWindowSize = 0 - recentlyAcceptedWindowTTL = 30 * time.Second + // MaxRecentlyAcceptedWindowSize is the maximum number of blocks that the + // recommended minimum height will lag behind the last accepted block. + MaxRecentlyAcceptedWindowSize = 64 + // MinRecentlyAcceptedWindowSize is the minimum number of blocks that the + // recommended minimum height will lag behind the last accepted block. + MinRecentlyAcceptedWindowSize = 0 + // RecentlyAcceptedWindowTTL is the amount of time after a block is accepted + // to avoid recommending it as the minimum height. The size constraints take + // precedence over this time constraint. + RecentlyAcceptedWindowTTL = 30 * time.Second + + validatorSetsCacheSize = 64 ) var ( @@ -111,9 +119,9 @@ func NewManager( recentlyAccepted: window.New[ids.ID]( window.Config{ Clock: clk, - MaxSize: maxRecentlyAcceptedWindowSize, - MinSize: minRecentlyAcceptedWindowSize, - TTL: recentlyAcceptedWindowTTL, + MaxSize: MaxRecentlyAcceptedWindowSize, + MinSize: MinRecentlyAcceptedWindowSize, + TTL: RecentlyAcceptedWindowTTL, }, ), } diff --git a/vms/platformvm/warp/payload/addressed_call.go b/vms/platformvm/warp/payload/addressed_call.go index b3617ce487da..9950be42ec0d 100644 --- a/vms/platformvm/warp/payload/addressed_call.go +++ b/vms/platformvm/warp/payload/addressed_call.go @@ -37,7 +37,7 @@ func ParseAddressedCall(b []byte) (*AddressedCall, error) { } payload, ok := payloadIntf.(*AddressedCall) if !ok { - return nil, fmt.Errorf("%w: %T", errWrongType, payloadIntf) + return nil, fmt.Errorf("%w: %T", ErrWrongType, payloadIntf) } return payload, nil } diff --git a/vms/platformvm/warp/payload/hash.go b/vms/platformvm/warp/payload/hash.go index 330f74fd869d..73804d169bdf 100644 --- a/vms/platformvm/warp/payload/hash.go +++ b/vms/platformvm/warp/payload/hash.go @@ -33,7 +33,7 @@ func ParseHash(b []byte) (*Hash, error) { } payload, ok := payloadIntf.(*Hash) if !ok { - return nil, fmt.Errorf("%w: %T", errWrongType, payloadIntf) + return nil, fmt.Errorf("%w: %T", ErrWrongType, payloadIntf) } return payload, nil } diff --git a/vms/platformvm/warp/payload/payload.go b/vms/platformvm/warp/payload/payload.go index c5c09464803e..0f7831c6b343 100644 --- a/vms/platformvm/warp/payload/payload.go +++ b/vms/platformvm/warp/payload/payload.go @@ -8,7 +8,7 @@ import ( "fmt" ) -var errWrongType = errors.New("wrong payload type") +var ErrWrongType = errors.New("wrong payload type") // Payload provides a common interface for all payloads implemented by this // package. diff --git a/vms/platformvm/warp/payload/payload_test.go b/vms/platformvm/warp/payload/payload_test.go index 86b584ae33db..6ae1fcc09c3c 100644 --- a/vms/platformvm/warp/payload/payload_test.go +++ b/vms/platformvm/warp/payload/payload_test.go @@ -33,10 +33,10 @@ func TestParseWrongPayloadType(t *testing.T) { require.NoError(err) _, err = ParseAddressedCall(hashPayload.Bytes()) - require.ErrorIs(err, errWrongType) + require.ErrorIs(err, ErrWrongType) _, err = ParseHash(addressedPayload.Bytes()) - require.ErrorIs(err, errWrongType) + require.ErrorIs(err, ErrWrongType) } func TestParse(t *testing.T) { diff --git a/wallet/chain/p/builder/builder.go b/wallet/chain/p/builder/builder.go index e8762d7b2a68..8a429fbe24c3 100644 --- a/wallet/chain/p/builder/builder.go +++ b/wallet/chain/p/builder/builder.go @@ -12,6 +12,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/constants" + "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/components/avax" @@ -164,6 +165,20 @@ type Builder interface { options ...common.Option, ) (*txs.ConvertSubnetTx, error) + // RegisterSubnetValidatorTx adds a validator to an L1. + // + // - [balance] that the validator should allocate to continuous fees + // - [proofOfPossession] is the BLS PoP for the key included in the Warp + // message + // - [message] is the Warp message that authorizes this validator to be + // added + NewRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, + ) (*txs.RegisterSubnetValidatorTx, error) + // NewImportTx creates an import transaction that attempts to consume all // the available UTXOs and import the funds to [to]. // @@ -863,6 +878,63 @@ func (b *builder) NewConvertSubnetTx( return tx, b.initCtx(tx) } +func (b *builder) NewRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, +) (*txs.RegisterSubnetValidatorTx, error) { + var ( + toBurn = map[ids.ID]uint64{ + b.context.AVAXAssetID: balance, + } + toStake = map[ids.ID]uint64{} + + ops = common.NewOptions(options) + memo = ops.Memo() + memoComplexity = gas.Dimensions{ + gas.Bandwidth: uint64(len(memo)), + } + ) + warpComplexity, err := fee.WarpComplexity(message) + if err != nil { + return nil, err + } + complexity, err := fee.IntrinsicRegisterSubnetValidatorTxComplexities.Add( + &memoComplexity, + &warpComplexity, + ) + if err != nil { + return nil, err + } + + inputs, outputs, _, err := b.spend( + toBurn, + toStake, + 0, + complexity, + nil, + ops, + ) + if err != nil { + return nil, err + } + + tx := &txs.RegisterSubnetValidatorTx{ + BaseTx: txs.BaseTx{BaseTx: avax.BaseTx{ + NetworkID: b.context.NetworkID, + BlockchainID: constants.PlatformChainID, + Ins: inputs, + Outs: outputs, + Memo: memo, + }}, + Balance: balance, + ProofOfPossession: proofOfPossession, + Message: message, + } + return tx, b.initCtx(tx) +} + func (b *builder) NewImportTx( sourceChainID ids.ID, to *secp256k1fx.OutputOwners, diff --git a/wallet/chain/p/builder/with_options.go b/wallet/chain/p/builder/with_options.go index e83683b89687..89fb35e9bd26 100644 --- a/wallet/chain/p/builder/with_options.go +++ b/wallet/chain/p/builder/with_options.go @@ -7,6 +7,7 @@ import ( "time" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/platformvm/signer" "github.com/ava-labs/avalanchego/vms/platformvm/txs" @@ -170,6 +171,20 @@ func (w *withOptions) NewConvertSubnetTx( ) } +func (w *withOptions) NewRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, +) (*txs.RegisterSubnetValidatorTx, error) { + return w.builder.NewRegisterSubnetValidatorTx( + balance, + proofOfPossession, + message, + common.UnionOptions(w.options, options)..., + ) +} + func (w *withOptions) NewImportTx( sourceChainID ids.ID, to *secp256k1fx.OutputOwners, diff --git a/wallet/chain/p/builder_test.go b/wallet/chain/p/builder_test.go index f56a0eb0896b..bc03f30814e9 100644 --- a/wallet/chain/p/builder_test.go +++ b/wallet/chain/p/builder_test.go @@ -26,7 +26,9 @@ import ( "github.com/ava-labs/avalanchego/vms/platformvm/stakeable" "github.com/ava-labs/avalanchego/vms/platformvm/txs" "github.com/ava-labs/avalanchego/vms/platformvm/txs/fee" + "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/vms/platformvm/warp/message" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" "github.com/ava-labs/avalanchego/vms/secp256k1fx" "github.com/ava-labs/avalanchego/vms/types" "github.com/ava-labs/avalanchego/wallet/chain/p/builder" @@ -747,6 +749,106 @@ func TestConvertSubnetTx(t *testing.T) { } } +func TestRegisterSubnetValidatorTx(t *testing.T) { + const ( + expiry = 1731005097 + weight = 7905001371 + + balance = units.Avax + ) + + sk, err := bls.NewSecretKey() + require.NoError(t, err) + pop := signer.NewProofOfPossession(sk) + + addressedCallPayload, err := message.NewRegisterSubnetValidator( + subnetID, + nodeID, + pop.PublicKey, + expiry, + message.PChainOwner{ + Threshold: 1, + Addresses: []ids.ShortID{ + ids.GenerateTestShortID(), + }, + }, + message.PChainOwner{ + Threshold: 1, + Addresses: []ids.ShortID{ + ids.GenerateTestShortID(), + }, + }, + weight, + ) + require.NoError(t, err) + + addressedCall, err := payload.NewAddressedCall( + utils.RandomBytes(20), + addressedCallPayload.Bytes(), + ) + require.NoError(t, err) + + unsignedWarp, err := warp.NewUnsignedMessage( + constants.UnitTestID, + ids.GenerateTestID(), + addressedCall.Bytes(), + ) + require.NoError(t, err) + + signers := set.NewBits(0) + + unsignedBytes := unsignedWarp.Bytes() + sig := bls.Sign(sk, unsignedBytes) + sigBytes := [bls.SignatureLen]byte{} + copy(sigBytes[:], bls.SignatureToBytes(sig)) + + warp, err := warp.NewMessage( + unsignedWarp, + &warp.BitSetSignature{ + Signers: signers.Bytes(), + Signature: sigBytes, + }, + ) + require.NoError(t, err) + warpMessageBytes := warp.Bytes() + + for _, e := range testEnvironmentPostEtna { + t.Run(e.name, func(t *testing.T) { + var ( + require = require.New(t) + chainUTXOs = utxotest.NewDeterministicChainUTXOs(t, map[ids.ID][]*avax.UTXO{ + constants.PlatformChainID: utxos, + }) + backend = wallet.NewBackend(e.context, chainUTXOs, nil) + builder = builder.New(set.Of(utxoAddr), e.context, backend) + ) + + utx, err := builder.NewRegisterSubnetValidatorTx( + balance, + pop.ProofOfPossession, + warpMessageBytes, + common.WithMemo(e.memo), + ) + require.NoError(err) + require.Equal(balance, utx.Balance) + require.Equal(pop.ProofOfPossession, utx.ProofOfPossession) + require.Equal(types.JSONByteSlice(warpMessageBytes), utx.Message) + require.Equal(types.JSONByteSlice(e.memo), utx.Memo) + requireFeeIsCorrect( + require, + e.feeCalculator, + utx, + &utx.BaseTx.BaseTx, + nil, + nil, + map[ids.ID]uint64{ + e.context.AVAXAssetID: balance, // Balance of the validator + }, + ) + }) + } +} + func makeTestUTXOs(utxosKey *secp256k1.PrivateKey) []*avax.UTXO { // Note: we avoid ids.GenerateTestNodeID here to make sure that UTXO IDs // won't change run by run. This simplifies checking what utxos are included diff --git a/wallet/chain/p/signer/visitor.go b/wallet/chain/p/signer/visitor.go index b358e1f5d5ea..bb4f6d37f87f 100644 --- a/wallet/chain/p/signer/visitor.go +++ b/wallet/chain/p/signer/visitor.go @@ -198,6 +198,14 @@ func (s *visitor) ConvertSubnetTx(tx *txs.ConvertSubnetTx) error { return sign(s.tx, true, txSigners) } +func (s *visitor) RegisterSubnetValidatorTx(tx *txs.RegisterSubnetValidatorTx) error { + txSigners, err := s.getSigners(constants.PlatformChainID, tx.Ins) + if err != nil { + return err + } + return sign(s.tx, true, txSigners) +} + func (s *visitor) getSigners(sourceChainID ids.ID, ins []*avax.TransferableInput) ([][]keychain.Signer, error) { txSigners := make([][]keychain.Signer, len(ins)) for credIndex, transferInput := range ins { diff --git a/wallet/chain/p/wallet/backend_visitor.go b/wallet/chain/p/wallet/backend_visitor.go index f2f9e646edf8..733eaa505259 100644 --- a/wallet/chain/p/wallet/backend_visitor.go +++ b/wallet/chain/p/wallet/backend_visitor.go @@ -123,6 +123,10 @@ func (b *backendVisitor) ConvertSubnetTx(tx *txs.ConvertSubnetTx) error { return b.baseTx(&tx.BaseTx) } +func (b *backendVisitor) RegisterSubnetValidatorTx(tx *txs.RegisterSubnetValidatorTx) error { + return b.baseTx(&tx.BaseTx) +} + func (b *backendVisitor) baseTx(tx *txs.BaseTx) error { return b.b.removeUTXOs( b.ctx, diff --git a/wallet/chain/p/wallet/wallet.go b/wallet/chain/p/wallet/wallet.go index a36522cc6b3a..fd6fd9a782e0 100644 --- a/wallet/chain/p/wallet/wallet.go +++ b/wallet/chain/p/wallet/wallet.go @@ -7,6 +7,7 @@ import ( "time" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/platformvm/txs" "github.com/ava-labs/avalanchego/vms/secp256k1fx" @@ -150,6 +151,21 @@ type Wallet interface { options ...common.Option, ) (*txs.Tx, error) + // IssueRegisterSubnetValidatorTx creates, signs, and issues a transaction + // that adds a validator to an L1. + // + // - [balance] that the validator should allocate to continuous fees + // - [proofOfPossession] is the BLS PoP for the key included in the Warp + // message + // - [message] is the Warp message that authorizes this validator to be + // added + IssueRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, + ) (*txs.Tx, error) + // IssueImportTx creates, signs, and issues an import transaction that // attempts to consume all the available UTXOs and import the funds to [to]. // @@ -404,6 +420,19 @@ func (w *wallet) IssueConvertSubnetTx( return w.IssueUnsignedTx(utx, options...) } +func (w *wallet) IssueRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, +) (*txs.Tx, error) { + utx, err := w.builder.NewRegisterSubnetValidatorTx(balance, proofOfPossession, message, options...) + if err != nil { + return nil, err + } + return w.IssueUnsignedTx(utx, options...) +} + func (w *wallet) IssueImportTx( sourceChainID ids.ID, to *secp256k1fx.OutputOwners, diff --git a/wallet/chain/p/wallet/with_options.go b/wallet/chain/p/wallet/with_options.go index e73d5a3215ee..c3e9527e9478 100644 --- a/wallet/chain/p/wallet/with_options.go +++ b/wallet/chain/p/wallet/with_options.go @@ -7,6 +7,7 @@ import ( "time" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/crypto/bls" "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/platformvm/txs" "github.com/ava-labs/avalanchego/vms/secp256k1fx" @@ -159,6 +160,20 @@ func (w *withOptions) IssueConvertSubnetTx( ) } +func (w *withOptions) IssueRegisterSubnetValidatorTx( + balance uint64, + proofOfPossession [bls.SignatureLen]byte, + message []byte, + options ...common.Option, +) (*txs.Tx, error) { + return w.wallet.IssueRegisterSubnetValidatorTx( + balance, + proofOfPossession, + message, + common.UnionOptions(w.options, options)..., + ) +} + func (w *withOptions) IssueImportTx( sourceChainID ids.ID, to *secp256k1fx.OutputOwners, diff --git a/wallet/subnet/primary/examples/register-subnet-validator/main.go b/wallet/subnet/primary/examples/register-subnet-validator/main.go new file mode 100644 index 000000000000..68248a7e18eb --- /dev/null +++ b/wallet/subnet/primary/examples/register-subnet-validator/main.go @@ -0,0 +1,141 @@ +// Copyright (C) 2019-2024, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package main + +import ( + "context" + "encoding/hex" + "encoding/json" + "log" + "time" + + "github.com/ava-labs/avalanchego/api/info" + "github.com/ava-labs/avalanchego/genesis" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/crypto/bls" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/ava-labs/avalanchego/vms/platformvm/warp" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/message" + "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + "github.com/ava-labs/avalanchego/wallet/subnet/primary" +) + +func main() { + key := genesis.EWOQKey + uri := primary.LocalAPIURI + kc := secp256k1fx.NewKeychain(key) + subnetID := ids.FromStringOrPanic("2DeHa7Qb6sufPkmQcFWG2uCd4pBPv9WB6dkzroiMQhd1NSRtof") + chainID := ids.FromStringOrPanic("2BMFrJ9xeh5JdwZEx6uuFcjfZC2SV2hdbMT8ee5HrvjtfJb5br") + address := []byte{} + weight := uint64(1) + blsSKHex := "3f783929b295f16cd1172396acb23b20eed057b9afb1caa419e9915f92860b35" + + blsSKBytes, err := hex.DecodeString(blsSKHex) + if err != nil { + log.Fatalf("failed to decode secret key: %s\n", err) + } + + sk, err := bls.SecretKeyFromBytes(blsSKBytes) + if err != nil { + log.Fatalf("failed to parse secret key: %s\n", err) + } + + ctx := context.Background() + infoClient := info.NewClient(uri) + + nodeInfoStartTime := time.Now() + nodeID, nodePoP, err := infoClient.GetNodeID(ctx) + if err != nil { + log.Fatalf("failed to fetch node IDs: %s\n", err) + } + log.Printf("fetched node ID %s in %s\n", nodeID, time.Since(nodeInfoStartTime)) + + // MakeWallet fetches the available UTXOs owned by [kc] on the network that + // [uri] is hosting. + walletSyncStartTime := time.Now() + wallet, err := primary.MakeWallet(ctx, &primary.WalletConfig{ + URI: uri, + AVAXKeychain: kc, + EthKeychain: kc, + }) + if err != nil { + log.Fatalf("failed to initialize wallet: %s\n", err) + } + log.Printf("synced wallet in %s\n", time.Since(walletSyncStartTime)) + + // Get the P-chain wallet + pWallet := wallet.P() + context := pWallet.Builder().Context() + + expiry := uint64(time.Now().Add(5 * time.Minute).Unix()) // This message will expire in 5 minutes + addressedCallPayload, err := message.NewRegisterSubnetValidator( + subnetID, + nodeID, + nodePoP.PublicKey, + expiry, + message.PChainOwner{}, + message.PChainOwner{}, + weight, + ) + if err != nil { + log.Fatalf("failed to create RegisterSubnetValidator message: %s\n", err) + } + addressedCallPayloadJSON, err := json.MarshalIndent(addressedCallPayload, "", "\t") + if err != nil { + log.Fatalf("failed to marshal RegisterSubnetValidator message: %s\n", err) + } + log.Println(string(addressedCallPayloadJSON)) + + addressedCall, err := payload.NewAddressedCall( + address, + addressedCallPayload.Bytes(), + ) + if err != nil { + log.Fatalf("failed to create AddressedCall message: %s\n", err) + } + + unsignedWarp, err := warp.NewUnsignedMessage( + context.NetworkID, + chainID, + addressedCall.Bytes(), + ) + if err != nil { + log.Fatalf("failed to create unsigned Warp message: %s\n", err) + } + + // This example assumes that the hard-coded BLS key is for the first + // validator in the signature bit-set. + signers := set.NewBits(0) + + unsignedBytes := unsignedWarp.Bytes() + sig := bls.Sign(sk, unsignedBytes) + sigBytes := [bls.SignatureLen]byte{} + copy(sigBytes[:], bls.SignatureToBytes(sig)) + + warp, err := warp.NewMessage( + unsignedWarp, + &warp.BitSetSignature{ + Signers: signers.Bytes(), + Signature: sigBytes, + }, + ) + if err != nil { + log.Fatalf("failed to create Warp message: %s\n", err) + } + + registerSubnetValidatorStartTime := time.Now() + registerSubnetValidatorTx, err := pWallet.IssueRegisterSubnetValidatorTx( + units.Avax, + nodePoP.ProofOfPossession, + warp.Bytes(), + ) + if err != nil { + log.Fatalf("failed to issue register subnet validator transaction: %s\n", err) + } + + validationID := addressedCallPayload.ValidationID() + log.Printf("registered new subnet validator %s to subnet %s with txID %s as validationID %s in %s\n", nodeID, subnetID, registerSubnetValidatorTx.ID(), validationID, time.Since(registerSubnetValidatorStartTime)) +}