diff --git a/pkg/morph/deploy/alphabet.go b/pkg/morph/deploy/alphabet.go new file mode 100644 index 00000000000..6203669093a --- /dev/null +++ b/pkg/morph/deploy/alphabet.go @@ -0,0 +1,100 @@ +package deploy + +import ( + "context" + "fmt" + + "github.com/nspcc-dev/neo-go/pkg/core/native/noderoles" + "github.com/nspcc-dev/neo-go/pkg/crypto/keys" + "github.com/nspcc-dev/neo-go/pkg/rpcclient/rolemgmt" + "github.com/nspcc-dev/neo-go/pkg/wallet" + "go.uber.org/zap" +) + +// initAlphabetPrm groups parameters of Alphabet members initialization. +type initAlphabetPrm struct { + logger *zap.Logger + + blockchain Blockchain + + // based on blockchain + monitor *blockchainMonitor + + committee keys.PublicKeys + localAcc *wallet.Account +} + +// initAlphabet designates NeoFS Alphabet role to all committee members on the +// given Blockchain. +func initAlphabet(ctx context.Context, prm initAlphabetPrm) error { + // wrap the parent context into the context of the current function so that + // transaction wait routines do not leak + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + committeeActor, err := newCommitteeNotaryActor(prm.blockchain, prm.localAcc, prm.committee) + if err != nil { + return fmt.Errorf("create Notary service client sending transactions to be signed by the committee: %w", err) + } + + roleContract := rolemgmt.New(committeeActor) + txMonitor := newTransactionGroupMonitor(committeeActor) + + for ; ; prm.monitor.waitForNextBlock(ctx) { + select { + case <-ctx.Done(): + return fmt.Errorf("wait for NeoFS Alphabet role to be designated for the committee: %w", ctx.Err()) + default: + } + + prm.logger.Info("checking NeoFS Alphabet role of the committee members...") + + accsWithAlphabetRole, err := roleContract.GetDesignatedByRole(noderoles.NeoFSAlphabet, prm.monitor.currentHeight()) + if err != nil { + prm.logger.Error("failed to check role of the committee, will try again later", zap.Error(err)) + continue + } + + someoneWithoutRole := len(accsWithAlphabetRole) < len(prm.committee) + if !someoneWithoutRole { + for i := range prm.committee { + if !accsWithAlphabetRole.Contains(prm.committee[i]) { + someoneWithoutRole = true + break + } + } + } + if !someoneWithoutRole { + prm.logger.Info("all committee members have a NeoFS Alphabet role") + return nil + } + + prm.logger.Info("not all members of the committee have a NeoFS Alphabet role, designation is needed") + + if txMonitor.isPending() { + prm.logger.Info("previously sent Notary request designating NeoFS Alphabet role to the committee is still pending, will wait for the outcome") + continue + } + + tx, err := roleContract.DesignateAsRoleTransaction(noderoles.NeoFSAlphabet, prm.committee) + if err != nil { + prm.logger.Error("failed to make transaction designating NeoFS Alphabet role to the committee, will try again later", zap.Error(err)) + continue + } + + mainTxID, fallbackTxID, vub, err := committeeActor.Notarize(tx, nil) + if err != nil { + if isErrNotEnoughGAS(err) { + prm.logger.Info("insufficient Notary balance to send new Notary request designating NeoFS Alphabet role to the committee, skip") + } else { + prm.logger.Error("failed to send new Notary request designating NeoFS Alphabet role to the committee, skip", zap.Error(err)) + } + continue + } + + prm.logger.Info("Notary request designating NeoFS Alphabet role to the committee has been successfully sent, will wait for the outcome", + zap.Stringer("main tx", mainTxID), zap.Stringer("fallback tx", fallbackTxID), zap.Uint32("vub", vub)) + + txMonitor.trackPendingTransactionsAsync(ctx, vub, mainTxID, fallbackTxID) + } +} diff --git a/pkg/morph/deploy/contracts.go b/pkg/morph/deploy/contracts.go index dcce0d988e2..f6cf8812331 100644 --- a/pkg/morph/deploy/contracts.go +++ b/pkg/morph/deploy/contracts.go @@ -1,7 +1,401 @@ package deploy +import ( + "context" + "encoding/json" + "errors" + "fmt" + + "github.com/nspcc-dev/neo-go/pkg/core/state" + "github.com/nspcc-dev/neo-go/pkg/core/transaction" + "github.com/nspcc-dev/neo-go/pkg/crypto/keys" + "github.com/nspcc-dev/neo-go/pkg/rpcclient/actor" + "github.com/nspcc-dev/neo-go/pkg/rpcclient/management" + "github.com/nspcc-dev/neo-go/pkg/rpcclient/nns" + "github.com/nspcc-dev/neo-go/pkg/smartcontract/manifest" + "github.com/nspcc-dev/neo-go/pkg/smartcontract/nef" + "github.com/nspcc-dev/neo-go/pkg/util" + "github.com/nspcc-dev/neo-go/pkg/wallet" + "go.uber.org/zap" +) + // various common methods of the NeoFS contracts. const ( methodUpdate = "update" methodVersion = "version" ) + +// syncNeoFSContractPrm groups parameters of syncNeoFSContract. +type syncNeoFSContractPrm struct { + logger *zap.Logger + + blockchain Blockchain + + // based on blockchain + monitor *blockchainMonitor + + localAcc *wallet.Account + + // address of the NeoFS NNS contract deployed in the blockchain + nnsContract util.Uint160 + systemEmail string + + committee keys.PublicKeys + committeeGroupKey *keys.PrivateKey + + localNEF nef.File + localManifest manifest.Manifest + + // L2 domain name in domainContractAddresses TLD in the NNS + domainName string + + // if set, syncNeoFSContract attempts to deploy the contract when it's + // missing on the chain + tryDeploy bool + // is contract must be deployed by the committee + committeeDeployRequired bool + + // optional constructor of extra arguments to be passed into method deploying + // the contract. If returns both nil, no data is passed (noExtraDeployArgs can + // be used). + // + // Ignored if tryDeploy is unset. + buildExtraDeployArgs func() ([]interface{}, error) + + // constructor of extra arguments to be passed into method updating the + // contract. If returns both nil, no data is passed. + buildVersionedExtraUpdateArgs func(versionOnChain contractVersion) ([]interface{}, error) +} + +// syncNeoFSContract behaves similar to updateNNSContract but also attempts to +// deploy the contract if it is missing on the chain and tryDeploy flag is set. +// If committeeDeployRequired is set, the contract is deployed on behalf of the +// committee. +// +// Returns address of the on-chain contract synchronized with the record of the +// NNS domain with parameterized name. +func syncNeoFSContract(ctx context.Context, prm syncNeoFSContractPrm) (util.Uint160, error) { + bLocalNEF, err := prm.localNEF.Bytes() + if err != nil { + // not really expected + return util.Uint160{}, fmt.Errorf("encode local NEF of the contract into binary: %w", err) + } + + jLocalManifest, err := json.Marshal(prm.localManifest) + if err != nil { + // not really expected + return util.Uint160{}, fmt.Errorf("encode local manifest of the contract into JSON: %w", err) + } + + localActor, err := actor.NewSimple(prm.blockchain, prm.localAcc) + if err != nil { + return util.Uint160{}, fmt.Errorf("init transaction sender from local account: %w", err) + } + + committeeActor, err := newCommitteeNotaryActor(prm.blockchain, prm.localAcc, prm.committee) + if err != nil { + return util.Uint160{}, fmt.Errorf("create Notary service client sending transactions to be signed by the committee: %w", err) + } + + localVersion, err := readContractLocalVersion(prm.blockchain, prm.localNEF, prm.localManifest) + if err != nil { + return util.Uint160{}, fmt.Errorf("read version of the local contract: %w", err) + } + + // wrap the parent context into the context of the current function so that + // transaction wait routines do not leak + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + setGroupInManifest(&prm.localManifest, prm.localNEF, prm.committeeGroupKey, prm.localAcc.ScriptHash()) + + var managementContract *management.Contract + if prm.committeeDeployRequired { + deployCommitteeActor, err := newCommitteeNotaryActorWithScope(prm.blockchain, prm.localAcc, prm.committee, transaction.Global) + if err != nil { + return util.Uint160{}, fmt.Errorf("create Notary service client sending deploy transactions to be signed by the committee: %w", err) + } + + managementContract = management.New(deployCommitteeActor) + } else { + managementContract = management.New(localActor) + } + + var alreadyUpdated bool + domainNameForAddress := prm.domainName + "." + domainContractAddresses + l := prm.logger.With(zap.String("contract", prm.localManifest.Name), zap.String("domain", domainNameForAddress)) + deployTxMonitor := newTransactionGroupMonitor(localActor) + updateTxMonitor := newTransactionGroupMonitor(localActor) + registerDomainTxMonitor := newTransactionGroupMonitor(localActor) + registerTLDTxMonitor := newTransactionGroupMonitor(localActor) + setDomainRecordTxMonitor := newTransactionGroupMonitor(localActor) + + for ; ; prm.monitor.waitForNextBlock(ctx) { + select { + case <-ctx.Done(): + return util.Uint160{}, fmt.Errorf("wait for the contract synchronization: %w", ctx.Err()) + default: + } + + l.Info("reading on-chain state of the contract by NNS domain name...") + + var missingDomainName, missingDomainRecord bool + + onChainState, err := readContractOnChainStateByDomainName(prm.blockchain, prm.nnsContract, domainNameForAddress) + if err != nil { + if errors.Is(err, errContractNotFound) { + l.Error("contract is recorded in the NNS but not found on the chain, will wait for a background fix") + continue + } + + missingDomainName = errors.Is(err, errMissingDomain) + if !missingDomainName { + missingDomainRecord = errors.Is(err, errMissingDomainRecord) + if !missingDomainRecord { + if errors.Is(err, errInvalidContractDomainRecord) { + l.Error("contract's domain record is invalid/unsupported, will wait for a background fix", zap.Error(err)) + } else { + l.Error("failed to read on-chain state of the contract record by NNS domain name, will try again later", zap.Error(err)) + } + continue + } + } + + l.Error("could not read on-chain state of the contract by NNS domain name, trying by pre-calculated address...") + + preCalculatedAddr := state.CreateContractHash(prm.localAcc.ScriptHash(), prm.localNEF.Checksum, prm.localManifest.Name) + + onChainState, err = prm.blockchain.GetContractStateByHash(preCalculatedAddr) + if err != nil && !isErrContractNotFound(err) { + l.Error("failed to read on-chain state of the contract by pre-calculated address, will try again later", + zap.Stringer("address", preCalculatedAddr), zap.Error(err)) + } + } + + if onChainState == nil { + // according to instructions above, we get here when contract is missing on the chain + if !prm.tryDeploy { + l.Info("contract is missing on the chain but attempts to deploy are disabled, will wait for background deployment") + continue + } + + l.Info("contract is missing on the chain, deployment needed") + + if deployTxMonitor.isPending() { + l.Info("previously sent transaction deploying the contract is still pending, will wait for the outcome") + continue + } + + extraDeployArgs, err := prm.buildExtraDeployArgs() + if err != nil { + l.Info("failed to prepare extra deployment arguments, will try again later", zap.Error(err)) + continue + } + + // just to definitely avoid mutation + nefCp := prm.localNEF + manifestCp := prm.localManifest + + if prm.committeeDeployRequired { + l.Info("contract requires committee witness for deployment, sending Notary request...") + + mainTxID, fallbackTxID, vub, err := committeeActor.Notarize(managementContract.DeployTransaction(&nefCp, &manifestCp, extraDeployArgs)) + if err != nil { + if isErrNotEnoughGAS(err) { + l.Info("insufficient Notary balance to deploy the contract, will try again later") + } else { + l.Error("failed to send Notary request deploying the contract, will try again later", zap.Error(err)) + } + continue + } + + l.Info("Notary request deploying the contract has been successfully sent, will wait for the outcome", + zap.Stringer("main tx", mainTxID), zap.Stringer("fallback tx", fallbackTxID), zap.Uint32("vub", vub)) + + deployTxMonitor.trackPendingTransactionsAsync(ctx, vub, mainTxID, fallbackTxID) + + continue + } + + l.Info("contract does not require committee witness for deployment, sending simple transaction...") + + txID, vub, err := managementContract.Deploy(&nefCp, &manifestCp, extraDeployArgs) + if err != nil { + if isErrNotEnoughGAS(err) { + l.Info("not enough GAS to deploy the contract, will try again later") + } else { + l.Error("failed to send transaction deploying the contract, will try again later", zap.Error(err)) + } + continue + } + + l.Info("transaction deploying the contract has been successfully sent, will wait for the outcome", + zap.Stringer("tx", txID), zap.Uint32("vub", vub), + ) + + deployTxMonitor.trackPendingTransactionsAsync(ctx, vub, txID) + + continue + } + + if onChainState.NEF.Checksum == prm.localNEF.Checksum { + // manifests may differ, but currently we should bump internal contract version + // (i.e. change NEF) to make such updates. Right now they are not supported due + // to dubious practical need + // Track https://github.com/nspcc-dev/neofs-contract/issues/340 + l.Info("same local and on-chain checksums of the the contract NEF, update is not needed") + + if !missingDomainName && !missingDomainRecord { + return onChainState.Hash, nil + } + } else if !alreadyUpdated { + // see comments same code block in updateNNSContract + l.Info("NEF checksums of the on-chain and local contract states differ, need an update") + + versionOnChain, err := readContractOnChainVersion(prm.blockchain, onChainState.Hash) + if err != nil { + l.Error("failed to read on-chain version of the contract, will try again later", zap.Error(err)) + continue + } + + if v := localVersion.cmp(versionOnChain); v == -1 { + prm.logger.Info("local contract version is < than the on-chain one, update is not needed", + zap.Stringer("local", localVersion), zap.Stringer("on-chain", versionOnChain)) + return onChainState.Hash, nil + } else if v == 0 { + return util.Uint160{}, fmt.Errorf("local and on-chain contracts have different NEF checksums but same version '%s'", versionOnChain) + } + + extraUpdateArgs, err := prm.buildVersionedExtraUpdateArgs(versionOnChain) + if err != nil { + l.Error("failed to prepare build extra arguments for the contract update, will try again later", + zap.Stringer("on-chain version", versionOnChain), zap.Error(err)) + continue + } + + tx, err := committeeActor.MakeCall(onChainState.Hash, methodUpdate, + bLocalNEF, jLocalManifest, extraUpdateArgs) + if err != nil { + if isErrContractAlreadyUpdated(err) { + // note that we can come here only if local version is > than the on-chain one + // (compared above) + l.Info("the contract has already been updated") + if !missingDomainName && !missingDomainRecord { + return onChainState.Hash, nil + } + alreadyUpdated = true + } else { + l.Error("failed to make transaction updating the contract, will try again later", zap.Error(err)) + } + continue + } + + if updateTxMonitor.isPending() { + l.Info("previously sent Notary request updating the contract is still pending, will wait for the outcome") + continue + } + + l.Info("sending new Notary request updating the contract...") + + mainTxID, fallbackTxID, vub, err := committeeActor.Notarize(tx, nil) + if err != nil { + if isErrNotEnoughGAS(err) { + l.Info("insufficient Notary balance to update the contract, will try again later") + } else { + l.Error("failed to send Notary request updating the contract, will try again later", zap.Error(err)) + } + continue + } + + l.Info("Notary request updating the contract has been successfully sent, will wait for the outcome", + zap.Stringer("main tx", mainTxID), zap.Stringer("fallback tx", fallbackTxID), zap.Uint32("vub", vub)) + + updateTxMonitor.trackPendingTransactionsAsync(ctx, vub, mainTxID, fallbackTxID) + + continue + } + + if missingDomainName { + l.Info("NNS domain is missing, registration is needed") + + if registerDomainTxMonitor.isPending() { + l.Info("previously sent transaction registering domain in the NNS is still pending, will wait for the outcome") + continue + } + + l.Info("sending new transaction registering domain in the NNS...") + + txID, vub, err := localActor.SendCall(prm.nnsContract, methodNNSRegister, + domainNameForAddress, localActor.Sender(), prm.systemEmail, nnsRefresh, nnsRetry, nnsExpire, nnsMinimum) + if err != nil { + switch { + default: + l.Error("failed to send transaction registering domain in the NNS, will try again later", zap.Error(err)) + case isErrNotEnoughGAS(err): + l.Info("not enough GAS to register domain in the NNS, will try again later") + case isErrTLDNotFound(err): + l.Info("missing TLD, need registration") + + if registerTLDTxMonitor.isPending() { + l.Info("previously sent Notary request registering TLD in the NNS is still pending, will wait for the outcome") + continue + } + + l.Info("sending new Notary registering TLD in the NNS...") + + mainTxID, fallbackTxID, vub, err := committeeActor.Notarize(committeeActor.MakeCall(prm.nnsContract, methodNNSRegisterTLD, + domainContractAddresses, prm.systemEmail, nnsRefresh, nnsRetry, nnsExpire, nnsMinimum)) + if err != nil { + if isErrNotEnoughGAS(err) { + l.Info("insufficient Notary balance to register TLD in the NNS, will try again later") + } else { + l.Error("failed to send Notary request registering TLD in the NNS, will try again later", zap.Error(err)) + } + continue + } + + l.Info("Notary request registering TLD in the NNS has been successfully sent, will wait for the outcome", + zap.Stringer("main tx", mainTxID), zap.Stringer("fallback tx", fallbackTxID), zap.Uint32("vub", vub)) + + registerTLDTxMonitor.trackPendingTransactionsAsync(ctx, vub, mainTxID, fallbackTxID) + } + continue + } + + l.Info("transaction registering domain in the NNS has been successfully sent, will wait for the outcome", + zap.Stringer("tx", txID), zap.Uint32("vub", vub), + ) + + registerDomainTxMonitor.trackPendingTransactionsAsync(ctx, vub, txID) + + continue + } + + // we come here only when missingDomainRecord is true + l.Info("missing domain record in the NNS, needed to be set") + + if setDomainRecordTxMonitor.isPending() { + l.Info("previously sent transaction setting domain record in the NNS is still pending, will wait for the outcome") + continue + } + + l.Info("sending new transaction setting domain record in the NNS...") + + txID, vub, err := localActor.SendCall(prm.nnsContract, methodNNSAddRecord, + domainNameForAddress, int64(nns.TXT), onChainState.Hash.StringLE()) + if err != nil { + if isErrNotEnoughGAS(err) { + l.Info("not enough GAS to set domain record in the NNS, will try again later") + } else { + l.Error("failed to send transaction setting domain record in the NNS, will try again later", zap.Error(err)) + } + continue + } + + l.Info("transaction setting domain record in the NNS has been successfully sent, will wait for the outcome", + zap.Stringer("tx", txID), zap.Uint32("vub", vub), + ) + + setDomainRecordTxMonitor.trackPendingTransactionsAsync(ctx, vub, txID) + } +} diff --git a/pkg/morph/deploy/deploy.go b/pkg/morph/deploy/deploy.go index 09328a5811e..7be96f8ed2a 100644 --- a/pkg/morph/deploy/deploy.go +++ b/pkg/morph/deploy/deploy.go @@ -5,7 +5,9 @@ import ( "context" "errors" "fmt" + "math/big" "sort" + "strconv" "github.com/nspcc-dev/neo-go/pkg/core/block" "github.com/nspcc-dev/neo-go/pkg/core/state" @@ -15,7 +17,10 @@ import ( "github.com/nspcc-dev/neo-go/pkg/rpcclient/notary" "github.com/nspcc-dev/neo-go/pkg/smartcontract/manifest" "github.com/nspcc-dev/neo-go/pkg/smartcontract/nef" + "github.com/nspcc-dev/neo-go/pkg/util" + "github.com/nspcc-dev/neo-go/pkg/vm/stackitem" "github.com/nspcc-dev/neo-go/pkg/wallet" + "github.com/nspcc-dev/neofs-node/pkg/morph/client/netmap" "go.uber.org/zap" ) @@ -35,6 +40,9 @@ type Blockchain interface { // requested contract is missing. GetContractStateByID(id int32) (*state.Contract, error) + // GetContractStateByHash is similar to GetContractStateByID but accepts address. + GetContractStateByHash(util.Uint160) (*state.Contract, error) + // ReceiveBlocks starts background process that forwards new blocks of the // blockchain to the provided channel. The process handles all new blocks when // ReceiveBlocks is called with nil filter. Returns unique identifier to be used @@ -72,6 +80,47 @@ type NNSPrm struct { SystemEmail string } +// AlphabetContractPrm groups deployment parameters of the NeoFS Alphabet contract. +type AlphabetContractPrm struct { + Common CommonDeployPrm +} + +// AuditContractPrm groups deployment parameters of the NeoFS Audit contract. +type AuditContractPrm struct { + Common CommonDeployPrm +} + +// BalanceContractPrm groups deployment parameters of the NeoFS Balance contract. +type BalanceContractPrm struct { + Common CommonDeployPrm +} + +// ContainerContractPrm groups deployment parameters of the Container contract. +type ContainerContractPrm struct { + Common CommonDeployPrm +} + +// NeoFSIDContractPrm groups deployment parameters of the NeoFS ID contract. +type NeoFSIDContractPrm struct { + Common CommonDeployPrm +} + +// NetmapContractPrm groups deployment parameters of the Netmap contract. +type NetmapContractPrm struct { + Common CommonDeployPrm + Config netmap.NetworkConfiguration +} + +// ProxyContractPrm groups deployment parameters of the NeoFS Proxy contract. +type ProxyContractPrm struct { + Common CommonDeployPrm +} + +// ReputationContractPrm groups deployment parameters of the NeoFS Reputation contract. +type ReputationContractPrm struct { + Common CommonDeployPrm +} + // Prm groups all parameters of the NeoFS Sidechain deployment procedure. type Prm struct { // Writes progress into the log. @@ -87,6 +136,15 @@ type Prm struct { KeyStorage KeyStorage NNS NNSPrm + + AlphabetContract AlphabetContractPrm + AuditContract AuditContractPrm + BalanceContract BalanceContractPrm + ContainerContract ContainerContractPrm + NeoFSIDContract NeoFSIDContractPrm + NetmapContract NetmapContractPrm + ProxyContract ProxyContractPrm + ReputationContract ReputationContractPrm } // Deploy initializes Neo network represented by given Prm.Blockchain as NeoFS @@ -101,8 +159,9 @@ type Prm struct { // 1. NNS contract deployment // 2. launch of a notary service for the committee // 3. committee group initialization -// 4. deployment/update of the NeoFS system contracts (currently only NNS) -// 5. deployment of custom contracts +// 4. Alphabet initialization +// 5. deployment/update of the NeoFS system contracts +// 6. deployment of custom contracts (currently not supported) // // See project documentation for details. func Deploy(ctx context.Context, prm Prm) error { @@ -236,6 +295,21 @@ func Deploy(ctx context.Context, prm Prm) error { prm.Logger.Info("committee group successfully initialized", zap.Stringer("public key", committeeGroupKey.PublicKey())) + prm.Logger.Info("initializing NeoFS Alphabet...") + + err = initAlphabet(ctx, initAlphabetPrm{ + logger: prm.Logger, + blockchain: prm.Blockchain, + monitor: monitor, + committee: committee, + localAcc: prm.LocalAccount, + }) + if err != nil { + return fmt.Errorf("init NeoFS Alphabet: %w", err) + } + + prm.Logger.Info("NeoFS Alphabet successfully initialized") + prm.Logger.Info("updating on-chain NNS contract...") err = updateNNSContract(ctx, updateNNSContractPrm{ @@ -256,9 +330,313 @@ func Deploy(ctx context.Context, prm Prm) error { prm.Logger.Info("on-chain NNS contract successfully updated") - // TODO: deploy/update other contracts + syncPrm := syncNeoFSContractPrm{ + logger: prm.Logger, + blockchain: prm.Blockchain, + monitor: monitor, + localAcc: prm.LocalAccount, + nnsContract: nnsOnChainAddress, + systemEmail: prm.NNS.SystemEmail, + committee: committee, + committeeGroupKey: committeeGroupKey, + } + + localAccLeads := localAccCommitteeIndex == 0 + + // we attempt to deploy contracts by single committee member (1st for + // simplicity) to reduce the likelihood of contract duplication in the chain and + // better predictability of the final address (the address is a function from + // the sender of the deploying transaction). While this approach is centralized, + // we still expect any node incl. 1st one to be "healthy". Updates are done + // concurrently. + syncPrm.tryDeploy = localAccLeads + + var notaryDisabledExtraUpdateArg bool + + // then go dependent contracts. Things may become better/easier after + // https://github.com/nspcc-dev/neofs-contract/issues/325 + resolveContractAddressDynamically := func(commonPrm CommonDeployPrm, contractDomain string) (util.Uint160, error) { + domain := calculateContractAddressDomain(contractDomain) + onChainState, err := readContractOnChainStateByDomainName(prm.Blockchain, nnsOnChainAddress, domain) + if err != nil { + // contract may be deployed but not registered in the NNS yet + if localAccLeads && (errors.Is(err, errMissingDomain) || errors.Is(err, errMissingDomainRecord)) { + return state.CreateContractHash(prm.LocalAccount.ScriptHash(), commonPrm.NEF.Checksum, commonPrm.Manifest.Name), nil + } + return util.Uint160{}, fmt.Errorf("failed to read on-chain state of the contract by NNS domain '%s': %w", domain, err) + } + return onChainState.Hash, nil + } + + // Alphabet + syncPrm.localNEF = prm.AlphabetContract.Common.NEF + syncPrm.localManifest = prm.AlphabetContract.Common.Manifest + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + for ind := 0; ind < len(committee) && ind < glagolitsaSize; ind++ { + syncPrm.tryDeploy = ind == localAccCommitteeIndex // each member deploys its own Alphabet contract + syncPrm.domainName = calculateAlphabetContractAddressDomain(ind) + syncPrm.buildExtraDeployArgs = func() ([]interface{}, error) { + netmapContractAddress, err := resolveContractAddressDynamically(prm.NetmapContract.Common, domainNetmap) + if err != nil { + return nil, fmt.Errorf("resolve address of the Netmap contract: %w", err) + } + proxyContractAddress, err := resolveContractAddressDynamically(prm.ProxyContract.Common, domainProxy) + if err != nil { + return nil, fmt.Errorf("resolve address of the Proxy contract: %w", err) + } + return []interface{}{ + notaryDisabledExtraUpdateArg, + netmapContractAddress, + proxyContractAddress, + glagoliticLetterByIndex(ind), + ind, + len(committee), + }, nil + } + + prm.Logger.Info("synchronizing Alphabet contract with the chain...", zap.Int("index", ind)) + + alphabetContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Alphabet contract #%d with the chain: %w", ind, err) + } + + prm.Logger.Info("Alphabet contract successfully synchronized", + zap.Int("index", ind), zap.Stringer("address", alphabetContractAddress)) + } + + // Audit + syncPrm.localNEF = prm.AuditContract.Common.NEF + syncPrm.localManifest = prm.AuditContract.Common.Manifest + syncPrm.domainName = domainAudit + syncPrm.buildExtraDeployArgs = noExtraDeployArgs + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing Audit contract with the chain...") + + auditContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Audit contract with the chain: %w", err) + } + + prm.Logger.Info("Audit contract successfully synchronized", zap.Stringer("address", auditContractAddress)) + + // Balance + syncPrm.localNEF = prm.BalanceContract.Common.NEF + syncPrm.localManifest = prm.BalanceContract.Common.Manifest + syncPrm.domainName = domainBalance + syncPrm.buildExtraDeployArgs = noExtraDeployArgs + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing Balance contract with the chain...") + + balanceContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Balance contract with the chain: %w", err) + } + + prm.Logger.Info("Balance contract successfully synchronized", zap.Stringer("address", balanceContractAddress)) + + // Container + syncPrm.localNEF = prm.ContainerContract.Common.NEF + syncPrm.localManifest = prm.ContainerContract.Common.Manifest + syncPrm.domainName = domainContainer + syncPrm.committeeDeployRequired = true + syncPrm.buildExtraDeployArgs = func() ([]interface{}, error) { + netmapContractAddress, err := resolveContractAddressDynamically(prm.NetmapContract.Common, domainNetmap) + if err != nil { + return nil, fmt.Errorf("resolve address of the Netmap contract: %w", err) + } + balanceContractAddress, err := resolveContractAddressDynamically(prm.BalanceContract.Common, domainBalance) + if err != nil { + return nil, fmt.Errorf("resolve address of the Balance contract: %w", err) + } + neoFSIDContractAddress, err := resolveContractAddressDynamically(prm.NeoFSIDContract.Common, domainNeoFSID) + if err != nil { + return nil, fmt.Errorf("resolve address of the NeoFSID contract: %w", err) + } + return []interface{}{ + notaryDisabledExtraUpdateArg, + netmapContractAddress, + balanceContractAddress, + neoFSIDContractAddress, + nnsOnChainAddress, + domainContainers, + }, nil + } + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing Container contract with the chain...") + + containerContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Container contract with the chain: %w", err) + } + + prm.Logger.Info("Container contract successfully synchronized", zap.Stringer("address", containerContractAddress)) + + syncPrm.committeeDeployRequired = false + + // NeoFSID + syncPrm.localNEF = prm.NeoFSIDContract.Common.NEF + syncPrm.localManifest = prm.NeoFSIDContract.Common.Manifest + syncPrm.domainName = domainNeoFSID + syncPrm.buildExtraDeployArgs = func() ([]interface{}, error) { + netmapContractAddress, err := resolveContractAddressDynamically(prm.NetmapContract.Common, domainNetmap) + if err != nil { + return nil, fmt.Errorf("resolve address of the Netmap contract: %w", err) + } + return []interface{}{ + notaryDisabledExtraUpdateArg, + netmapContractAddress, + }, nil + } + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing NeoFSID contract with the chain...") + + neoFSIDContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync NeoFSID contract with the chain: %w", err) + } + + prm.Logger.Info("NeoFSID contract successfully synchronized", zap.Stringer("address", neoFSIDContractAddress)) + + // Netmap + netConfig := []interface{}{ + []byte(netmap.MaxObjectSizeConfig), encodeUintConfig(prm.NetmapContract.Config.MaxObjectSize), + []byte(netmap.BasicIncomeRateConfig), encodeUintConfig(prm.NetmapContract.Config.StoragePrice), + []byte(netmap.AuditFeeConfig), encodeUintConfig(prm.NetmapContract.Config.AuditFee), + []byte(netmap.EpochDurationConfig), encodeUintConfig(prm.NetmapContract.Config.EpochDuration), + []byte(netmap.ContainerFeeConfig), encodeUintConfig(prm.NetmapContract.Config.ContainerFee), + []byte(netmap.ContainerAliasFeeConfig), encodeUintConfig(prm.NetmapContract.Config.ContainerAliasFee), + []byte(netmap.EtIterationsConfig), encodeUintConfig(prm.NetmapContract.Config.EigenTrustIterations), + []byte(netmap.EtAlphaConfig), encodeFloatConfig(prm.NetmapContract.Config.EigenTrustAlpha), + []byte(netmap.IrCandidateFeeConfig), encodeUintConfig(prm.NetmapContract.Config.IRCandidateFee), + []byte(netmap.WithdrawFeeConfig), encodeUintConfig(prm.NetmapContract.Config.WithdrawalFee), + []byte(netmap.HomomorphicHashingDisabledKey), encodeBoolConfig(prm.NetmapContract.Config.HomomorphicHashingDisabled), + []byte(netmap.MaintenanceModeAllowedConfig), encodeBoolConfig(prm.NetmapContract.Config.MaintenanceModeAllowed), + } + + for i := range prm.NetmapContract.Config.Raw { + netConfig = append(netConfig, []byte(prm.NetmapContract.Config.Raw[i].Name), prm.NetmapContract.Config.Raw[i].Value) + } + + syncPrm.localNEF = prm.NetmapContract.Common.NEF + syncPrm.localManifest = prm.NetmapContract.Common.Manifest + syncPrm.domainName = domainNetmap + syncPrm.buildExtraDeployArgs = func() ([]interface{}, error) { + balanceContractAddress, err := resolveContractAddressDynamically(prm.BalanceContract.Common, domainBalance) + if err != nil { + return nil, fmt.Errorf("resolve address of the Balance contract: %w", err) + } + containerContractAddress, err := resolveContractAddressDynamically(prm.ContainerContract.Common, domainContainer) + if err != nil { + return nil, fmt.Errorf("resolve address of the Container contract: %w", err) + } + return []interface{}{ + notaryDisabledExtraUpdateArg, + balanceContractAddress, + containerContractAddress, + netConfig, + []interface{}(nil), // keys, currently unused + domainContractAddresses, + }, nil + } + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing Netmap contract with the chain...") + + netmapContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Netmap contract with the chain: %w", err) + } + + prm.Logger.Info("Netmap contract successfully synchronized", zap.Stringer("address", netmapContractAddress)) + + // Proxy + syncPrm.localNEF = prm.ProxyContract.Common.NEF + syncPrm.localManifest = prm.ProxyContract.Common.Manifest + syncPrm.domainName = domainProxy + syncPrm.buildExtraDeployArgs = noExtraDeployArgs + syncPrm.buildVersionedExtraUpdateArgs = noExtraUpdateArgs + + prm.Logger.Info("synchronizing Proxy contract with the chain...") + + proxyContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Proxy contract with the chain: %w", err) + } + + prm.Logger.Info("Proxy contract successfully synchronized", zap.Stringer("address", proxyContractAddress)) + + // Reputation + syncPrm.localNEF = prm.ReputationContract.Common.NEF + syncPrm.localManifest = prm.ReputationContract.Common.Manifest + syncPrm.domainName = domainReputation + syncPrm.buildExtraDeployArgs = noExtraDeployArgs + syncPrm.buildVersionedExtraUpdateArgs = func(versionOnChain contractVersion) ([]interface{}, error) { + if versionOnChain.equals(0, 17, 0) { + return []interface{}{notaryDisabledExtraUpdateArg}, nil + } + return nil, nil + } + + prm.Logger.Info("synchronizing Reputation contract with the chain...") + + reputationContractAddress, err := syncNeoFSContract(ctx, syncPrm) + if err != nil { + return fmt.Errorf("sync Reputation contract with the chain: %w", err) + } + + prm.Logger.Info("Reputation contract successfully synchronized", zap.Stringer("address", reputationContractAddress)) return nil } func noExtraUpdateArgs(contractVersion) ([]interface{}, error) { return nil, nil } + +func noExtraDeployArgs() ([]interface{}, error) { return nil, nil } + +func encodeUintConfig(v uint64) []byte { + return stackitem.NewBigInteger(new(big.Int).SetUint64(v)).Bytes() +} + +func encodeFloatConfig(v float64) []byte { + return []byte(strconv.FormatFloat(v, 'f', -1, 64)) +} + +func encodeBoolConfig(v bool) []byte { + return stackitem.NewBool(v).Bytes() +} diff --git a/pkg/morph/deploy/nns.go b/pkg/morph/deploy/nns.go index 66f9bec0025..10486910acb 100644 --- a/pkg/morph/deploy/nns.go +++ b/pkg/morph/deploy/nns.go @@ -28,8 +28,26 @@ const ( domainDesignateNotaryPrefix = "designate-committee-notary-" domainDesignateNotaryTx = domainDesignateNotaryPrefix + "tx." + domainBootstrap domainContractAddresses = "neofs" + domainContainers = "container" + + domainAlphabetFmt = "alphabet%d" + domainAudit = "audit" + domainBalance = "balance" + domainContainer = "container" + domainNeoFSID = "neofsid" + domainNetmap = "netmap" + domainProxy = "proxy" + domainReputation = "reputation" ) +func calculateAlphabetContractAddressDomain(index int) string { + return fmt.Sprintf(domainAlphabetFmt, index) +} + +func calculateContractAddressDomain(contractDomain string) string { + return contractDomain + "." + domainContractAddresses +} + func designateNotarySignatureDomainForMember(memberIndex int) string { return fmt.Sprintf("%s%d.%s", domainDesignateNotaryPrefix, memberIndex, domainBootstrap) } @@ -40,10 +58,11 @@ func committeeGroupDomainForMember(memberIndex int) string { // various methods of the NeoFS NNS contract. const ( - methodNNSRegister = "register" - methodNNSResolve = "resolve" - methodNNSAddRecord = "addRecord" - methodNNSSetRecord = "setRecord" + methodNNSRegister = "register" + methodNNSRegisterTLD = "registerTLD" + methodNNSResolve = "resolve" + methodNNSAddRecord = "addRecord" + methodNNSSetRecord = "setRecord" ) // default NNS domain settings. See DNS specification and also diff --git a/pkg/morph/deploy/notary.go b/pkg/morph/deploy/notary.go index 64c4e7cb766..e5391bf1956 100644 --- a/pkg/morph/deploy/notary.go +++ b/pkg/morph/deploy/notary.go @@ -842,10 +842,17 @@ func makeUnsignedDesignateCommitteeNotaryTx(roleContract *rolemgmt.Contract, com return tx, nil } -// newCommitteeNotaryActor returns notary.Actor that builds and sends Notary -// service requests witnessed by the specified committee members to the provided -// Blockchain. Given local account pays for transactions. +// newCommitteeNotaryActor calls newCommitteeNotaryActorWithScope with transaction.CalledByEntry +// witness scope appropriate for most transactions. func newCommitteeNotaryActor(b Blockchain, localAcc *wallet.Account, committee keys.PublicKeys) (*notary.Actor, error) { + return newCommitteeNotaryActorWithScope(b, localAcc, committee, transaction.Global) +} + +// newCommitteeNotaryActorWithScope returns notary.Actor builds and sends Notary +// service requests witnessed by the specified committee members to the provided +// Blockchain. Composed main transactions will have specified witness scope. +// Given local account pays for transactions. +func newCommitteeNotaryActorWithScope(b Blockchain, localAcc *wallet.Account, committee keys.PublicKeys, witnessScope transaction.WitnessScope) (*notary.Actor, error) { committeeMultiSigM := smartcontract.GetMajorityHonestNodeCount(len(committee)) committeeMultiSigAcc := wallet.NewAccountFromPrivateKey(localAcc.PrivateKey()) diff --git a/pkg/morph/deploy/util.go b/pkg/morph/deploy/util.go index 30d37d4e865..e31236a64b3 100644 --- a/pkg/morph/deploy/util.go +++ b/pkg/morph/deploy/util.go @@ -12,6 +12,7 @@ import ( "github.com/nspcc-dev/neo-go/pkg/core/interop/interopnames" "github.com/nspcc-dev/neo-go/pkg/core/state" "github.com/nspcc-dev/neo-go/pkg/crypto/keys" + "github.com/nspcc-dev/neo-go/pkg/encoding/address" "github.com/nspcc-dev/neo-go/pkg/io" "github.com/nspcc-dev/neo-go/pkg/neorpc" "github.com/nspcc-dev/neo-go/pkg/neorpc/result" @@ -48,6 +49,10 @@ func isErrContractAlreadyUpdated(err error) bool { return strings.Contains(err.Error(), common.ErrAlreadyUpdated) } +func isErrTLDNotFound(err error) bool { + return strings.Contains(err.Error(), "TLD not found") +} + func setGroupInManifest(manif *manifest.Manifest, nefFile nef.File, groupPrivKey *keys.PrivateKey, deployerAcc util.Uint160) { contractAddress := state.CreateContractHash(deployerAcc, nefFile.Checksum, manif.Name) sig := groupPrivKey.Sign(contractAddress.BytesBE()) @@ -325,3 +330,56 @@ func (x *transactionGroupMonitor) trackPendingTransactionsAsync(ctx context.Cont cancel() }() } + +var errInvalidContractDomainRecord = errors.New("invalid contract domain record") + +var errContractNotFound = errors.New("contract not found") + +// readContractOnChainStateByDomainName reads address state of contract deployed +// in the given Blockchain and recorded in the NNS with the specified domain +// name. Returns errMissingDomain if domain doesn't exist. Returns +// errMissingDomainRecord if domain has no records. Returns +// errInvalidContractDomainRecord if domain record has invalid/unsupported +// format. Returns errContractNotFound if contract is recorded in the NNS but +// missing in the Blockchain. +func readContractOnChainStateByDomainName(b Blockchain, nnsContract util.Uint160, domainName string) (*state.Contract, error) { + rec, err := lookupNNSDomainRecord(invoker.New(b, nil), nnsContract, domainName) + if err != nil { + return nil, err + } + + // historically two formats may occur + addr, err := util.Uint160DecodeStringLE(rec) + if err != nil { + addr, err = address.StringToUint160(rec) + if err != nil { + return nil, fmt.Errorf("%w: domain record '%s' neither NEO address nor little-endian hex-encoded script hash", errInvalidContractDomainRecord, rec) + } + } + + res, err := b.GetContractStateByHash(addr) + if err != nil { + if isErrContractNotFound(err) { + return nil, errContractNotFound + } + return nil, fmt.Errorf("get contract by address=%s: %w", addr, err) + } + + return res, nil +} + +const glagolitsaSize = 41 + +// returns string representation of Glagolitic letter compatible with NeoFS +// Alphabet contract by index. Index must be in range [0, glagolitsaSize). +// +// Track https://github.com/nspcc-dev/neofs-node/issues/2431 +func glagoliticLetterByIndex(ind int) string { + return []string{ + "az", "buky", "vedi", "glagoli", "dobro", "yest", "zhivete", "dzelo", + "zemlja", "izhe", "izhei", "gerv", "kako", "ljudi", "mislete", "nash", + "on", "pokoj", "rtsi", "slovo", "tverdo", "uk", "fert", "kher", "oht", + "shta", "tsi", "cherv", "sha", "yer", "yeri", "yerj", "yat", "jo", "yu", + "small.yus", "small.iotated.yus", "big.yus", "big.iotated.yus", "fita", "izhitsa", + }[ind] +}