From ba18cc1c241034a898520325dc7914611e89061c Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Wed, 18 Oct 2023 13:25:24 +0200
Subject: [PATCH 01/56] Remove validator component.
---
components/validator/component.go | 93 ----------
components/validator/issuer.go | 160 ------------------
components/validator/params.go | 34 ----
pkg/blockhandler/account.go | 70 --------
pkg/blockhandler/block_params.go | 122 -------------
.../{blockissuer.go => blockhandler.go} | 0
6 files changed, 479 deletions(-)
delete mode 100644 components/validator/component.go
delete mode 100644 components/validator/issuer.go
delete mode 100644 components/validator/params.go
delete mode 100644 pkg/blockhandler/account.go
delete mode 100644 pkg/blockhandler/block_params.go
rename pkg/blockhandler/{blockissuer.go => blockhandler.go} (100%)
diff --git a/components/validator/component.go b/components/validator/component.go
deleted file mode 100644
index c7a79bb96..000000000
--- a/components/validator/component.go
+++ /dev/null
@@ -1,93 +0,0 @@
-package validator
-
-import (
- "context"
- "sync/atomic"
- "time"
-
- "go.uber.org/dig"
-
- "github.com/iotaledger/hive.go/app"
- "github.com/iotaledger/hive.go/runtime/event"
- "github.com/iotaledger/hive.go/runtime/timed"
- "github.com/iotaledger/iota-core/pkg/blockhandler"
- "github.com/iotaledger/iota-core/pkg/daemon"
- "github.com/iotaledger/iota-core/pkg/protocol"
- "github.com/iotaledger/iota-core/pkg/protocol/engine/notarization"
- iotago "github.com/iotaledger/iota.go/v4"
-)
-
-func init() {
- Component = &app.Component{
- Name: "Validator",
- DepsFunc: func(cDeps dependencies) { deps = cDeps },
- Params: params,
- Run: run,
- IsEnabled: func(_ *dig.Container) bool {
- return ParamsValidator.Enabled
- },
- }
-}
-
-var (
- Component *app.Component
- deps dependencies
-
- isValidator atomic.Bool
- executor *timed.TaskExecutor[iotago.AccountID]
- validatorAccount blockhandler.Account
-)
-
-type dependencies struct {
- dig.In
-
- Protocol *protocol.Protocol
- BlockHandler *blockhandler.BlockHandler
-}
-
-func run() error {
- validatorAccount = blockhandler.AccountFromParams(ParamsValidator.Account, ParamsValidator.PrivateKey)
-
- executor = timed.NewTaskExecutor[iotago.AccountID](1)
-
- return Component.Daemon().BackgroundWorker(Component.Name, func(ctx context.Context) {
- Component.LogInfof("Starting Validator with IssuerID: %s", validatorAccount.ID())
-
- checkValidatorStatus(ctx)
-
- deps.Protocol.Events.Engine.Notarization.SlotCommitted.Hook(func(details *notarization.SlotCommittedDetails) {
- checkValidatorStatus(ctx)
- }, event.WithWorkerPool(Component.WorkerPool))
-
- <-ctx.Done()
-
- executor.Shutdown()
-
- Component.LogInfo("Stopping Validator... done")
- }, daemon.PriorityActivity)
-}
-
-func checkValidatorStatus(ctx context.Context) {
- account, exists, err := deps.Protocol.MainEngineInstance().Ledger.Account(validatorAccount.ID(), deps.Protocol.MainEngineInstance().Storage.Settings().LatestCommitment().Slot())
- if err != nil {
- Component.LogErrorf("error when retrieving BlockIssuer account %s: %w", validatorAccount.ID(), err)
-
- return
- }
-
- if !exists || account.StakeEndEpoch <= deps.Protocol.CurrentAPI().TimeProvider().EpochFromSlot(deps.Protocol.CurrentAPI().TimeProvider().SlotFromTime(time.Now())) {
- if prevValue := isValidator.Swap(false); prevValue {
- // If the account stops being a validator, don't issue any blocks.
- Component.LogInfof("BlockIssuer account %s stopped being a validator", validatorAccount.ID())
- executor.Cancel(validatorAccount.ID())
- }
-
- return
- }
-
- if prevValue := isValidator.Swap(true); !prevValue {
- Component.LogInfof("BlockIssuer account %s became a validator", validatorAccount.ID())
- // If the account becomes a validator, start issue validator blocks.
- executor.ExecuteAfter(validatorAccount.ID(), func() { issueValidatorBlock(ctx) }, ParamsValidator.CommitteeBroadcastInterval)
- }
-}
diff --git a/components/validator/issuer.go b/components/validator/issuer.go
deleted file mode 100644
index 6c05d67a3..000000000
--- a/components/validator/issuer.go
+++ /dev/null
@@ -1,160 +0,0 @@
-package validator
-
-import (
- "context"
- "time"
-
- "github.com/iotaledger/hive.go/ierrors"
- "github.com/iotaledger/iota-core/pkg/model"
- iotago "github.com/iotaledger/iota.go/v4"
- "github.com/iotaledger/iota.go/v4/builder"
-)
-
-var ErrBlockTooRecent = ierrors.New("block is too recent compared to latest commitment")
-
-func issueValidatorBlock(ctx context.Context) {
- // Get the main engine instance in case it changes mid-execution.
- engineInstance := deps.Protocol.MainEngineInstance()
-
- blockIssuingTime := time.Now()
- nextBroadcast := blockIssuingTime.Add(ParamsValidator.CommitteeBroadcastInterval)
-
- // Use 'defer' because nextBroadcast is updated during function execution, and the value at the end needs to be used.
- defer func() {
- executor.ExecuteAt(validatorAccount.ID(), func() { issueValidatorBlock(ctx) }, nextBroadcast)
- }()
-
- if !ParamsValidator.IgnoreBootstrapped && !engineInstance.SyncManager.IsBootstrapped() {
- Component.LogDebug("Not issuing validator block because node is not bootstrapped yet.")
-
- return
- }
-
- protocolParametersHash, err := deps.Protocol.CurrentAPI().ProtocolParameters().Hash()
- if err != nil {
- Component.LogWarnf("failed to get protocol parameters hash: %s", err.Error())
-
- return
- }
-
- parents := engineInstance.TipSelection.SelectTips(iotago.BlockTypeValidationMaxParents)
-
- addressableCommitment, err := getAddressableCommitment(deps.Protocol.CurrentAPI().TimeProvider().SlotFromTime(blockIssuingTime))
- if err != nil && ierrors.Is(err, ErrBlockTooRecent) {
- commitment, parentID, reviveChainErr := reviveChain(blockIssuingTime)
- if reviveChainErr != nil {
- Component.LogError("error reviving chain: %s", reviveChainErr.Error())
- return
- }
-
- addressableCommitment = commitment
- parents = make(model.ParentReferences)
- parents[iotago.StrongParentType] = []iotago.BlockID{parentID}
- } else if err != nil {
- Component.LogWarnf("error getting commitment: %s", err.Error())
-
- return
- }
-
- // create the validation block here using the validation block builder from iota.go
- validationBlock, err := builder.NewValidationBlockBuilder(deps.Protocol.CurrentAPI()).
- IssuingTime(blockIssuingTime).
- ProtocolParametersHash(protocolParametersHash).
- SlotCommitmentID(addressableCommitment.MustID()).
- HighestSupportedVersion(deps.Protocol.LatestAPI().Version()).
- LatestFinalizedSlot(engineInstance.SyncManager.LatestFinalizedSlot()).
- StrongParents(parents[iotago.StrongParentType]).
- WeakParents(parents[iotago.WeakParentType]).
- ShallowLikeParents(parents[iotago.ShallowLikeParentType]).
- Sign(validatorAccount.ID(), validatorAccount.PrivateKey()).
- Build()
- if err != nil {
- Component.LogWarnf("error creating validation block: %s", err.Error())
-
- return
- }
-
- modelBlock, err := model.BlockFromBlock(validationBlock)
- if err != nil {
- Component.LogWarnf("error creating model block from validation block: %s", err.Error())
-
- return
- }
-
- if !engineInstance.SybilProtection.SeatManager().Committee(deps.Protocol.CurrentAPI().TimeProvider().SlotFromTime(blockIssuingTime)).HasAccount(validatorAccount.ID()) {
- // update nextBroadcast value here, so that this updated value is used in the `defer`
- // callback to schedule issuing of the next block at a different interval than for committee members
- nextBroadcast = blockIssuingTime.Add(ParamsValidator.CandidateBroadcastInterval)
- }
-
- if err = deps.BlockHandler.SubmitBlock(modelBlock); err != nil {
- Component.LogWarnf("error issuing validator block: %s", err.Error())
-
- return
- }
-
- Component.LogDebugf("Issued validator block: %s - commitment %s %d - latest finalized slot %d", modelBlock.ID(), modelBlock.ProtocolBlock().SlotCommitmentID, modelBlock.ProtocolBlock().SlotCommitmentID.Slot(), modelBlock.ProtocolBlock().LatestFinalizedSlot)
-}
-
-func reviveChain(issuingTime time.Time) (*iotago.Commitment, iotago.BlockID, error) {
- lastCommittedSlot := deps.Protocol.MainEngineInstance().Storage.Settings().LatestCommitment().Slot()
- apiForSlot := deps.Protocol.APIForSlot(lastCommittedSlot)
-
- // Get a rootblock as recent as possible for the parent.
- parentBlockID := iotago.EmptyBlockID
- for rootBlock := range deps.Protocol.MainEngineInstance().EvictionState.ActiveRootBlocks() {
- if rootBlock.Slot() > parentBlockID.Slot() {
- parentBlockID = rootBlock
- }
-
- // Exit the loop if we found a rootblock in the last committed slot (which is the highest we can get).
- if parentBlockID.Slot() == lastCommittedSlot {
- break
- }
- }
-
- issuingSlot := apiForSlot.TimeProvider().SlotFromTime(issuingTime)
-
- // Force commitments until minCommittableAge relative to the block's issuing time. We basically "pretend" that
- // this block was already accepted at the time of issuing so that we have a commitment to reference.
- if issuingSlot < apiForSlot.ProtocolParameters().MinCommittableAge() { // Should never happen as we're beyond maxCommittableAge which is > minCommittableAge.
- return nil, iotago.EmptyBlockID, ierrors.Errorf("issuing slot %d is smaller than min committable age %d", issuingSlot, apiForSlot.ProtocolParameters().MinCommittableAge())
- }
- commitUntilSlot := issuingSlot - apiForSlot.ProtocolParameters().MinCommittableAge()
-
- if err := deps.Protocol.MainEngineInstance().Notarization.ForceCommitUntil(commitUntilSlot); err != nil {
- return nil, iotago.EmptyBlockID, ierrors.Wrapf(err, "failed to force commit until slot %d", commitUntilSlot)
- }
-
- commitment, err := deps.Protocol.MainEngineInstance().Storage.Commitments().Load(commitUntilSlot)
- if err != nil {
- return nil, iotago.EmptyBlockID, ierrors.Wrapf(err, "failed to commit until slot %d to revive chain", commitUntilSlot)
- }
-
- return commitment.Commitment(), parentBlockID, nil
-}
-
-func getAddressableCommitment(blockSlot iotago.SlotIndex) (*iotago.Commitment, error) {
- protoParams := deps.Protocol.CurrentAPI().ProtocolParameters()
- commitment := deps.Protocol.MainEngineInstance().Storage.Settings().LatestCommitment().Commitment()
-
- if blockSlot > commitment.Slot+protoParams.MaxCommittableAge() {
- return nil, ierrors.Wrapf(ErrBlockTooRecent, "can't issue block: block slot %d is too far in the future, latest commitment is %d", blockSlot, commitment.Slot)
- }
-
- if blockSlot < commitment.Slot+protoParams.MinCommittableAge() {
- if blockSlot < protoParams.MinCommittableAge() || commitment.Slot < protoParams.MinCommittableAge() {
- return commitment, nil
- }
-
- commitmentSlot := commitment.Slot - protoParams.MinCommittableAge()
- loadedCommitment, err := deps.Protocol.MainEngineInstance().Storage.Commitments().Load(commitmentSlot)
- if err != nil {
- return nil, ierrors.Wrapf(err, "error loading valid commitment of slot %d according to minCommittableAge from storage", commitmentSlot)
- }
-
- return loadedCommitment.Commitment(), nil
- }
-
- return commitment, nil
-}
diff --git a/components/validator/params.go b/components/validator/params.go
deleted file mode 100644
index 037498ab2..000000000
--- a/components/validator/params.go
+++ /dev/null
@@ -1,34 +0,0 @@
-package validator
-
-import (
- "time"
-
- "github.com/iotaledger/hive.go/app"
-)
-
-// ParametersValidator contains the definition of the configuration parameters used by the Validator component.
-type ParametersValidator struct {
- // Enabled defines whether the Validator component is enabled.
- Enabled bool `default:"false" usage:"whether the Validator component is enabled"`
- // CommitteeBroadcastInterval the interval at which the node will broadcast its committee validator block.
- CommitteeBroadcastInterval time.Duration `default:"500ms" usage:"the interval at which the node will broadcast its committee validator block"`
- // CandidateBroadcastInterval the interval at which the node will broadcast its candidate validator block.
- CandidateBroadcastInterval time.Duration `default:"30m" usage:"the interval at which the node will broadcast its candidate validator block"`
- // ParentsCount is the number of parents that node will choose for its validator blocks.
- ParentsCount int `default:"8" usage:"the number of parents that node will choose for its validator blocks"`
- // IgnoreBootstrapped sets whether the Validator component should start issuing validator blocks before the main engine is bootstrapped.
- IgnoreBootstrapped bool `default:"false" usage:"whether the Validator component should start issuing validator blocks before the main engine is bootstrapped"`
- // Account the accountID of the account that will issue the blocks.
- Account string `default:"" usage:"the accountID of the validator account that will issue the blocks"`
- // PrivateKey the private key of the account that will issue the blocks.
- PrivateKey string `default:"" usage:"the private key of the validator account that will issue the blocks"`
-}
-
-// ParamsValidator contains the values of the configuration parameters used by the Activity component.
-var ParamsValidator = &ParametersValidator{}
-
-var params = &app.ComponentParams{
- Params: map[string]any{
- "validator": ParamsValidator,
- },
-}
diff --git a/pkg/blockhandler/account.go b/pkg/blockhandler/account.go
deleted file mode 100644
index 5d71cb86b..000000000
--- a/pkg/blockhandler/account.go
+++ /dev/null
@@ -1,70 +0,0 @@
-package blockhandler
-
-import (
- "crypto/ed25519"
- "fmt"
-
- "github.com/iotaledger/hive.go/crypto"
- iotago "github.com/iotaledger/iota.go/v4"
-)
-
-// Account represents an account.
-type Account interface {
- // ID returns the accountID.
- ID() iotago.AccountID
-
- // Address returns the account address.
- Address() iotago.Address
-
- // PrivateKey returns the account private key for signing.
- PrivateKey() ed25519.PrivateKey
-}
-
-var _ Account = &Ed25519Account{}
-
-// Ed25519Account is an account that uses an Ed25519 key pair.
-type Ed25519Account struct {
- accountID iotago.AccountID
- privateKey ed25519.PrivateKey
-}
-
-// NewEd25519Account creates a new Ed25519Account.
-func NewEd25519Account(accountID iotago.AccountID, privateKey ed25519.PrivateKey) *Ed25519Account {
- return &Ed25519Account{
- accountID: accountID,
- privateKey: privateKey,
- }
-}
-
-// ID returns the accountID.
-func (e *Ed25519Account) ID() iotago.AccountID {
- return e.accountID
-}
-
-// Address returns the account address.
-func (e *Ed25519Account) Address() iotago.Address {
- ed25519PubKey, ok := e.privateKey.Public().(ed25519.PublicKey)
- if !ok {
- panic("invalid public key type")
- }
-
- return iotago.Ed25519AddressFromPubKey(ed25519PubKey)
-}
-
-// PrivateKey returns the account private key for signing.
-func (e *Ed25519Account) PrivateKey() ed25519.PrivateKey {
- return e.privateKey
-}
-
-func AccountFromParams(accountHex, privateKey string) Account {
- accountID, err := iotago.AccountIDFromHexString(accountHex)
- if err != nil {
- panic(fmt.Sprintln("invalid accountID hex string", err))
- }
- privKey, err := crypto.ParseEd25519PrivateKeyFromString(privateKey)
- if err != nil {
- panic(fmt.Sprintln("invalid ed25519 private key string", err))
- }
-
- return NewEd25519Account(accountID, privKey)
-}
diff --git a/pkg/blockhandler/block_params.go b/pkg/blockhandler/block_params.go
deleted file mode 100644
index 951f285c8..000000000
--- a/pkg/blockhandler/block_params.go
+++ /dev/null
@@ -1,122 +0,0 @@
-package blockhandler
-
-import (
- "time"
-
- "github.com/iotaledger/hive.go/runtime/options"
- "github.com/iotaledger/iota-core/pkg/model"
- iotago "github.com/iotaledger/iota.go/v4"
-)
-
-type BlockHeaderParams struct {
- ParentsCount int
- References model.ParentReferences
- SlotCommitment *iotago.Commitment
- LatestFinalizedSlot *iotago.SlotIndex
- IssuingTime *time.Time
- ProtocolVersion *iotago.Version
- Issuer Account
-}
-type BasicBlockParams struct {
- BlockHeader *BlockHeaderParams
- Payload iotago.Payload
-}
-type ValidatorBlockParams struct {
- BlockHeader *BlockHeaderParams
- HighestSupportedVersion *iotago.Version
- ProtocolParametersHash *iotago.Identifier
-}
-
-func WithParentsCount(parentsCount int) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.ParentsCount = parentsCount
- }
-}
-
-func WithStrongParents(blockIDs ...iotago.BlockID) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- if builder.References == nil {
- builder.References = make(model.ParentReferences)
- }
-
- builder.References[iotago.StrongParentType] = blockIDs
- }
-}
-func WithWeakParents(blockIDs ...iotago.BlockID) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- if builder.References == nil {
- builder.References = make(model.ParentReferences)
- }
-
- builder.References[iotago.WeakParentType] = blockIDs
- }
-}
-
-func WithShallowLikeParents(blockIDs ...iotago.BlockID) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- if builder.References == nil {
- builder.References = make(model.ParentReferences)
- }
-
- builder.References[iotago.ShallowLikeParentType] = blockIDs
- }
-}
-
-func WithSlotCommitment(commitment *iotago.Commitment) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.SlotCommitment = commitment
- }
-}
-
-func WithLatestFinalizedSlot(commitmentIndex iotago.SlotIndex) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.LatestFinalizedSlot = &commitmentIndex
- }
-}
-
-func WithIssuingTime(issuingTime time.Time) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.IssuingTime = &issuingTime
- }
-}
-
-func WithProtocolVersion(version iotago.Version) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.ProtocolVersion = &version
- }
-}
-func WithIssuer(issuer Account) func(builder *BlockHeaderParams) {
- return func(builder *BlockHeaderParams) {
- builder.Issuer = issuer
- }
-}
-
-func WithValidationBlockHeaderOptions(opts ...options.Option[BlockHeaderParams]) func(builder *ValidatorBlockParams) {
- return func(builder *ValidatorBlockParams) {
- builder.BlockHeader = options.Apply(&BlockHeaderParams{}, opts)
- }
-}
-
-func WithBasicBlockHeader(opts ...options.Option[BlockHeaderParams]) func(builder *BasicBlockParams) {
- return func(builder *BasicBlockParams) {
- builder.BlockHeader = options.Apply(&BlockHeaderParams{}, opts)
- }
-}
-
-func WithPayload(payload iotago.Payload) func(builder *BasicBlockParams) {
- return func(builder *BasicBlockParams) {
- builder.Payload = payload
- }
-}
-
-func WithHighestSupportedVersion(highestSupportedVersion iotago.Version) func(builder *ValidatorBlockParams) {
- return func(builder *ValidatorBlockParams) {
- builder.HighestSupportedVersion = &highestSupportedVersion
- }
-}
-
-func WithProtocolParametersHash(protocolParametersHash iotago.Identifier) func(builder *ValidatorBlockParams) {
- return func(builder *ValidatorBlockParams) {
- builder.ProtocolParametersHash = &protocolParametersHash
- }
-}
diff --git a/pkg/blockhandler/blockissuer.go b/pkg/blockhandler/blockhandler.go
similarity index 100%
rename from pkg/blockhandler/blockissuer.go
rename to pkg/blockhandler/blockhandler.go
From 5a46738fea19909353586975fe0b43803872bc52 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 20 Oct 2023 12:17:27 +0200
Subject: [PATCH 02/56] Add INX server endpoints to support inx-validator
---
components/app/app.go | 2 -
components/dashboard/component.go | 2 +-
components/inx/server_accounts.go | 44 ++++++++++++
components/inx/server_node.go | 1 +
go.mod | 34 +++++-----
go.sum | 68 +++++++++----------
.../engine/syncmanager/syncmanager.go | 1 +
.../trivialsyncmanager/syncmanager.go | 1 +
tools/evil-spammer/go.mod | 30 ++++----
tools/evil-spammer/go.sum | 60 ++++++++--------
tools/gendoc/go.mod | 34 +++++-----
tools/gendoc/go.sum | 68 +++++++++----------
tools/genesis-snapshot/go.mod | 26 +++----
tools/genesis-snapshot/go.sum | 52 +++++++-------
14 files changed, 234 insertions(+), 189 deletions(-)
create mode 100644 components/inx/server_accounts.go
diff --git a/components/app/app.go b/components/app/app.go
index 4cb029500..0ef0cabb0 100644
--- a/components/app/app.go
+++ b/components/app/app.go
@@ -14,7 +14,6 @@ import (
"github.com/iotaledger/iota-core/components/protocol"
"github.com/iotaledger/iota-core/components/restapi"
coreapi "github.com/iotaledger/iota-core/components/restapi/core"
- "github.com/iotaledger/iota-core/components/validator"
)
var (
@@ -38,7 +37,6 @@ func App() *app.App {
debugapi.Component,
metricstracker.Component,
protocol.Component,
- validator.Component,
dashboardmetrics.Component,
dashboard.Component,
metrics.Component,
diff --git a/components/dashboard/component.go b/components/dashboard/component.go
index 62326f082..360702aaa 100644
--- a/components/dashboard/component.go
+++ b/components/dashboard/component.go
@@ -155,7 +155,7 @@ func currentNodeStatus() *nodestatus {
status.TangleTime = tangleTime{
Synced: syncStatus.NodeSynced,
- Bootstrapped: deps.Protocol.MainEngineInstance().SyncManager.IsBootstrapped(),
+ Bootstrapped: syncStatus.NodeBootstrapped,
AcceptedBlockSlot: int64(syncStatus.LastAcceptedBlockSlot),
ConfirmedBlockSlot: int64(syncStatus.LastConfirmedBlockSlot),
CommittedSlot: int64(syncStatus.LatestCommitment.Slot()),
diff --git a/components/inx/server_accounts.go b/components/inx/server_accounts.go
new file mode 100644
index 000000000..998120dd6
--- /dev/null
+++ b/components/inx/server_accounts.go
@@ -0,0 +1,44 @@
+package inx
+
+import (
+ "context"
+
+ "github.com/iotaledger/hive.go/ierrors"
+ inx "github.com/iotaledger/inx/go"
+ iotago "github.com/iotaledger/iota.go/v4"
+)
+
+func (s *Server) ReadIsAccountValidator(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
+ slot := iotago.SlotIndex(accountInfoRequest.GetAccountSlot())
+ accountID, _, err := iotago.AccountIDFromBytes(accountInfoRequest.AccountId)
+ if err != nil {
+ return nil, ierrors.Wrap(err, "error when parsing account id")
+ }
+
+ account, exists, err := deps.Protocol.MainEngineInstance().Ledger.Account(accountID, slot)
+ if err != nil {
+ return nil, ierrors.Wrapf(err, "error when retrieving account data for %s", accountID)
+ }
+
+ return inx.WrapBoolResponse(exists && account.StakeEndEpoch <= deps.Protocol.APIForSlot(slot).TimeProvider().EpochFromSlot(slot)), nil
+}
+
+func (s *Server) ReadIsCommitteeMember(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
+ slot := iotago.SlotIndex(accountInfoRequest.GetAccountSlot())
+ accountID, _, err := iotago.AccountIDFromBytes(accountInfoRequest.AccountId)
+ if err != nil {
+ return nil, ierrors.Wrap(err, "error when parsing account id")
+ }
+
+ return inx.WrapBoolResponse(deps.Protocol.MainEngineInstance().SybilProtection.SeatManager().Committee(slot).HasAccount(accountID)), nil
+}
+
+func (s *Server) ReadIsCandidate(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
+ slot := iotago.SlotIndex(accountInfoRequest.GetAccountSlot())
+ accountID, _, err := iotago.AccountIDFromBytes(accountInfoRequest.AccountId)
+ if err != nil {
+ return nil, ierrors.Wrap(err, "error when parsing account id")
+ }
+
+ return inx.WrapBoolResponse(deps.Protocol.MainEngineInstance().SybilProtection.IsCandidateActive(accountID, deps.Protocol.APIForSlot(slot).TimeProvider().EpochFromSlot(slot))), nil
+}
diff --git a/components/inx/server_node.go b/components/inx/server_node.go
index 9a2f8386e..0039cad2b 100644
--- a/components/inx/server_node.go
+++ b/components/inx/server_node.go
@@ -25,6 +25,7 @@ func inxNodeStatus(status *syncmanager.SyncStatus) *inx.NodeStatus {
return &inx.NodeStatus{
IsHealthy: status.NodeSynced,
+ IsBootstrapped: status.NodeBootstrapped,
LastAcceptedBlockSlot: uint32(status.LastAcceptedBlockSlot),
LastConfirmedBlockSlot: uint32(status.LastConfirmedBlockSlot),
LatestCommitment: inxCommitment(status.LatestCommitment),
diff --git a/go.mod b/go.mod
index 8f58c8568..2466177a3 100644
--- a/go.mod
+++ b/go.mod
@@ -10,22 +10,22 @@ require (
github.com/google/uuid v1.3.1
github.com/gorilla/websocket v1.5.0
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
- github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811
+ github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad
- github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
github.com/libp2p/go-libp2p v0.30.0
@@ -39,7 +39,7 @@ require (
github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98
github.com/zyedidia/generic v1.2.1
go.uber.org/atomic v1.11.0
- go.uber.org/dig v1.17.0
+ go.uber.org/dig v1.17.1
golang.org/x/crypto v0.14.0
google.golang.org/grpc v1.58.3
google.golang.org/protobuf v1.31.0
@@ -88,7 +88,7 @@ require (
github.com/huin/goupnp v1.3.0 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 // indirect
github.com/ipfs/boxo v0.10.0 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
diff --git a/go.sum b/go.sum
index 1de1123f2..41bc1c322 100644
--- a/go.sum
+++ b/go.sum
@@ -277,40 +277,40 @@ github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJ
github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 h1:/Td52GNRrMzh0lVKwQu/BbVciwrKn5LwtkWAzd41X4Q=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38 h1:t60nsq/lI168gM0WqIP44FDHk784uXy35P6uKt9WbMU=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 h1:xYh6+dUl+iocWofo0Z/0/OJGSdxYaPV1oYo1HlFj8+s=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 h1:3OUYK9wgyH4U6xfGnLdU1QdYQ5dpS/6WRju+eXP051E=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 h1:iXqM0/Y4poiiuNf+0tu3nUj9Dzp4slrWHJ8GXKgt2Zk=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 h1:xyaKXgNt8CQ4t14Raqu/4+aBRZQunGOlK+T+99egkSA=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38 h1:6I6PEdLuwiIpmiXFVEW5lPaNe+xFdaBR6NSHsNZ3o6c=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 h1:SCBXTAhrLFs0o9EJxO9ZBGPyfq1VhOdb5pP7+vFhdtk=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38 h1:qks472rk/MjwwFvDrlpckxWTf4c1yd+k7ve/IkeuPYk=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 h1:6HIBNKOWLXY5Gk1s+q93D6IsTcqwANigfMO3ePtV3HQ=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38 h1:lTp8+8GgFH63IMY0G9B90v2vmOlgo+c3XiZg4f0Te+I=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38 h1:lPiKbh8o/SN2FGCN9RF7iN1zE+KXCidp815GSOieU8s=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 h1:njGr4TrKdpa66XyioxhDU3slV9gZOZeEg7/M8bjXw7M=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 h1:jh0LHzzLqVO+SW9NkvNIgYo1g3CyTBnR8Qv3+PNH+dY=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811 h1:nR8uTm1Htn2RQOT53RV2zdCIflpuF8N00Tv2FWYimY0=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811/go.mod h1:rmclNpXw5sKJDHU0e51Ar/9zL00P7Uu9hkfaM7vAAiE=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 h1:7TnpRtJourFhAEz+/yq9c3zXKSF6cd1xNBHy8jI3zfs=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1 h1:ymIiuWzy7uaDn51WVEWJVD8fFAgEc0gsg4elPX3MxzA=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 h1:ZUd2zsuTG1tT2u3UUpz9b6IRvZRzJwDOWHeBIbRmdwU=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 h1:o2z+yaMMoLmiQi0IaDjPGWcRaMhC8UiUwbnmkQl4wOU=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 h1:dFwSvX/p0P2QLQPdXSqiJJQ8eYtd2YI+PFJN/Msh6Jk=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 h1:3bLEYHGjvctjFHKkH3l1pJEI3iqbcRv423LFahsCFEQ=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1 h1:ohOZfLq91DTWuf63i2mbpuHWBqFr7TvZ2Pn8k/U2iXo=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 h1:kbQsSCboaEVcrF50ENpFn568RVadvUWpsFe5a/bKUT0=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1 h1:bUxkrvfRnEKcwOuD9HJoL3pNCp0RdkpfMcX1vgV4KAY=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 h1:nP19sKaWMBSql9BTlASjWYWKpiz98zEotiqUGuFYxcg=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1 h1:lJ8AXHNWex6oJ0bL9G+jbm0nl+eeTNv095a7x5Kca8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 h1:Kq/R4w9hKuNMqffz1w/ucpAb18gCb7saz846Fhlxnjc=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 h1:Qu+gNYpDHsc36L69LjulfAwkZvF9mbXYRiSJi9E1SGA=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJKdCgmByzhCv4IAnjpwDCJH54SYK8iBzctKGOfw=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648 h1:Lhk/aEyFwDflE03kdI6Xeap/awDJbxVsBzKh08U9hTY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/boxo v0.10.0 h1:tdDAxq8jrsbRkYoF+5Rcqyeb91hgWe2hp7iLu7ORZLY=
github.com/ipfs/boxo v0.10.0/go.mod h1:Fg+BnfxZ0RPzR0nOodzdIq3A7KgoWAOWsEIImrIQdBM=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
@@ -688,8 +688,8 @@ go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE=
go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0=
-go.uber.org/dig v1.17.0 h1:5Chju+tUvcC+N7N6EV08BJz41UZuO3BmHcN4A287ZLI=
-go.uber.org/dig v1.17.0/go.mod h1:rTxpf7l5I0eBTlE6/9RL+lDybC7WFwY2QH55ZSjy1mU=
+go.uber.org/dig v1.17.1 h1:Tga8Lz8PcYNsWsyHMZ1Vm0OQOUaJNDyvPImgbAu9YSc=
+go.uber.org/dig v1.17.1/go.mod h1:Us0rSJiThwCv2GteUN0Q7OKvU7n5J4dxZ9JKUXozFdE=
go.uber.org/fx v1.20.0 h1:ZMC/pnRvhsthOZh9MZjMq5U8Or3mA9zBSPaLnzs3ihQ=
go.uber.org/fx v1.20.0/go.mod h1:qCUj0btiR3/JnanEr1TYEePfSw6o/4qYJscgvzQ5Ub0=
go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ=
diff --git a/pkg/protocol/engine/syncmanager/syncmanager.go b/pkg/protocol/engine/syncmanager/syncmanager.go
index d251dec88..6f0c0cb53 100644
--- a/pkg/protocol/engine/syncmanager/syncmanager.go
+++ b/pkg/protocol/engine/syncmanager/syncmanager.go
@@ -38,6 +38,7 @@ type SyncManager interface {
}
type SyncStatus struct {
+ NodeBootstrapped bool
NodeSynced bool
LastAcceptedBlockSlot iotago.SlotIndex
LastConfirmedBlockSlot iotago.SlotIndex
diff --git a/pkg/protocol/engine/syncmanager/trivialsyncmanager/syncmanager.go b/pkg/protocol/engine/syncmanager/trivialsyncmanager/syncmanager.go
index 8e93f69bd..38fd0bb00 100644
--- a/pkg/protocol/engine/syncmanager/trivialsyncmanager/syncmanager.go
+++ b/pkg/protocol/engine/syncmanager/trivialsyncmanager/syncmanager.go
@@ -139,6 +139,7 @@ func (s *SyncManager) SyncStatus() *syncmanager.SyncStatus {
return &syncmanager.SyncStatus{
NodeSynced: s.IsNodeSynced(),
+ NodeBootstrapped: s.IsBootstrapped(),
LastAcceptedBlockSlot: s.lastAcceptedBlockSlot,
LastConfirmedBlockSlot: s.lastConfirmedBlockSlot,
LatestCommitment: s.latestCommitment,
diff --git a/tools/evil-spammer/go.mod b/tools/evil-spammer/go.mod
index 89b8bdd05..589eb3997 100644
--- a/tools/evil-spammer/go.mod
+++ b/tools/evil-spammer/go.mod
@@ -10,16 +10,16 @@ require (
github.com/AlecAivazis/survey/v2 v2.3.7
github.com/ethereum/go-ethereum v1.13.4
github.com/google/martian v2.1.0+incompatible
- github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38
+ github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
github.com/iotaledger/iota-core/tools/genesis-snapshot v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd
github.com/mr-tron/base58 v1.2.0
go.uber.org/atomic v1.11.0
)
@@ -38,13 +38,13 @@ require (
github.com/holiman/uint256 v1.2.3 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 // indirect
+ github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 // indirect
github.com/klauspost/cpuid/v2 v2.2.5 // indirect
diff --git a/tools/evil-spammer/go.sum b/tools/evil-spammer/go.sum
index 66d6fc334..bad893b35 100644
--- a/tools/evil-spammer/go.sum
+++ b/tools/evil-spammer/go.sum
@@ -173,36 +173,36 @@ github.com/iancoleman/orderedmap v0.3.0 h1:5cbR2grmZR/DiVt+VJopEhtVs9YGInGIxAoMJ
github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJDkXXS7VoV7XGE=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 h1:/Td52GNRrMzh0lVKwQu/BbVciwrKn5LwtkWAzd41X4Q=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38 h1:t60nsq/lI168gM0WqIP44FDHk784uXy35P6uKt9WbMU=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 h1:xYh6+dUl+iocWofo0Z/0/OJGSdxYaPV1oYo1HlFj8+s=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 h1:3OUYK9wgyH4U6xfGnLdU1QdYQ5dpS/6WRju+eXP051E=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 h1:iXqM0/Y4poiiuNf+0tu3nUj9Dzp4slrWHJ8GXKgt2Zk=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 h1:xyaKXgNt8CQ4t14Raqu/4+aBRZQunGOlK+T+99egkSA=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38 h1:6I6PEdLuwiIpmiXFVEW5lPaNe+xFdaBR6NSHsNZ3o6c=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 h1:SCBXTAhrLFs0o9EJxO9ZBGPyfq1VhOdb5pP7+vFhdtk=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38 h1:qks472rk/MjwwFvDrlpckxWTf4c1yd+k7ve/IkeuPYk=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 h1:6HIBNKOWLXY5Gk1s+q93D6IsTcqwANigfMO3ePtV3HQ=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38 h1:lTp8+8GgFH63IMY0G9B90v2vmOlgo+c3XiZg4f0Te+I=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38 h1:lPiKbh8o/SN2FGCN9RF7iN1zE+KXCidp815GSOieU8s=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 h1:njGr4TrKdpa66XyioxhDU3slV9gZOZeEg7/M8bjXw7M=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 h1:jh0LHzzLqVO+SW9NkvNIgYo1g3CyTBnR8Qv3+PNH+dY=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648 h1:Lhk/aEyFwDflE03kdI6Xeap/awDJbxVsBzKh08U9hTY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 h1:7TnpRtJourFhAEz+/yq9c3zXKSF6cd1xNBHy8jI3zfs=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1 h1:ymIiuWzy7uaDn51WVEWJVD8fFAgEc0gsg4elPX3MxzA=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 h1:ZUd2zsuTG1tT2u3UUpz9b6IRvZRzJwDOWHeBIbRmdwU=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 h1:o2z+yaMMoLmiQi0IaDjPGWcRaMhC8UiUwbnmkQl4wOU=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 h1:dFwSvX/p0P2QLQPdXSqiJJQ8eYtd2YI+PFJN/Msh6Jk=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 h1:3bLEYHGjvctjFHKkH3l1pJEI3iqbcRv423LFahsCFEQ=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1 h1:ohOZfLq91DTWuf63i2mbpuHWBqFr7TvZ2Pn8k/U2iXo=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 h1:kbQsSCboaEVcrF50ENpFn568RVadvUWpsFe5a/bKUT0=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1 h1:bUxkrvfRnEKcwOuD9HJoL3pNCp0RdkpfMcX1vgV4KAY=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 h1:nP19sKaWMBSql9BTlASjWYWKpiz98zEotiqUGuFYxcg=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1 h1:lJ8AXHNWex6oJ0bL9G+jbm0nl+eeTNv095a7x5Kca8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 h1:Kq/R4w9hKuNMqffz1w/ucpAb18gCb7saz846Fhlxnjc=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 h1:Qu+gNYpDHsc36L69LjulfAwkZvF9mbXYRiSJi9E1SGA=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 9ffe27b94..b5eac266f 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -5,7 +5,7 @@ go 1.21
replace github.com/iotaledger/iota-core => ../../
require (
- github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38
+ github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
)
@@ -58,22 +58,22 @@ require (
github.com/huin/goupnp v1.3.0 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811 // indirect
+ github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad // indirect
- github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648 // indirect
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd // indirect
github.com/ipfs/boxo v0.10.0 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
@@ -162,7 +162,7 @@ require (
go.opentelemetry.io/otel/metric v1.16.0 // indirect
go.opentelemetry.io/otel/trace v1.16.0 // indirect
go.uber.org/atomic v1.11.0 // indirect
- go.uber.org/dig v1.17.0 // indirect
+ go.uber.org/dig v1.17.1 // indirect
go.uber.org/fx v1.20.0 // indirect
go.uber.org/multierr v1.11.0 // indirect
go.uber.org/zap v1.26.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 593c433c4..5991ebd59 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -279,42 +279,42 @@ github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJ
github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 h1:/Td52GNRrMzh0lVKwQu/BbVciwrKn5LwtkWAzd41X4Q=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38 h1:t60nsq/lI168gM0WqIP44FDHk784uXy35P6uKt9WbMU=
-github.com/iotaledger/hive.go/app v0.0.0-20231019113503-7986872a7a38/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 h1:7TnpRtJourFhAEz+/yq9c3zXKSF6cd1xNBHy8jI3zfs=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1 h1:ymIiuWzy7uaDn51WVEWJVD8fFAgEc0gsg4elPX3MxzA=
+github.com/iotaledger/hive.go/app v0.0.0-20231019175611-f85bce9102d1/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3 h1:4aVJTc0KS77uEw0Tny4r0n1ORwcbAQDECaCclgf/6lE=
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3/go.mod h1:TZeAqieDu+xDOZp2e9+S+8pZp1PrfgcwLUnxmd8IgLU=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 h1:xYh6+dUl+iocWofo0Z/0/OJGSdxYaPV1oYo1HlFj8+s=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 h1:3OUYK9wgyH4U6xfGnLdU1QdYQ5dpS/6WRju+eXP051E=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 h1:iXqM0/Y4poiiuNf+0tu3nUj9Dzp4slrWHJ8GXKgt2Zk=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 h1:xyaKXgNt8CQ4t14Raqu/4+aBRZQunGOlK+T+99egkSA=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38 h1:6I6PEdLuwiIpmiXFVEW5lPaNe+xFdaBR6NSHsNZ3o6c=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 h1:SCBXTAhrLFs0o9EJxO9ZBGPyfq1VhOdb5pP7+vFhdtk=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38 h1:qks472rk/MjwwFvDrlpckxWTf4c1yd+k7ve/IkeuPYk=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 h1:6HIBNKOWLXY5Gk1s+q93D6IsTcqwANigfMO3ePtV3HQ=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38 h1:lTp8+8GgFH63IMY0G9B90v2vmOlgo+c3XiZg4f0Te+I=
-github.com/iotaledger/hive.go/logger v0.0.0-20231019113503-7986872a7a38/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38 h1:lPiKbh8o/SN2FGCN9RF7iN1zE+KXCidp815GSOieU8s=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 h1:njGr4TrKdpa66XyioxhDU3slV9gZOZeEg7/M8bjXw7M=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 h1:jh0LHzzLqVO+SW9NkvNIgYo1g3CyTBnR8Qv3+PNH+dY=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811 h1:nR8uTm1Htn2RQOT53RV2zdCIflpuF8N00Tv2FWYimY0=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231011161248-cf0bd6e08811/go.mod h1:rmclNpXw5sKJDHU0e51Ar/9zL00P7Uu9hkfaM7vAAiE=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 h1:ZUd2zsuTG1tT2u3UUpz9b6IRvZRzJwDOWHeBIbRmdwU=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 h1:o2z+yaMMoLmiQi0IaDjPGWcRaMhC8UiUwbnmkQl4wOU=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 h1:dFwSvX/p0P2QLQPdXSqiJJQ8eYtd2YI+PFJN/Msh6Jk=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 h1:3bLEYHGjvctjFHKkH3l1pJEI3iqbcRv423LFahsCFEQ=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1 h1:ohOZfLq91DTWuf63i2mbpuHWBqFr7TvZ2Pn8k/U2iXo=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 h1:kbQsSCboaEVcrF50ENpFn568RVadvUWpsFe5a/bKUT0=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1 h1:bUxkrvfRnEKcwOuD9HJoL3pNCp0RdkpfMcX1vgV4KAY=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 h1:nP19sKaWMBSql9BTlASjWYWKpiz98zEotiqUGuFYxcg=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1 h1:lJ8AXHNWex6oJ0bL9G+jbm0nl+eeTNv095a7x5Kca8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231019175611-f85bce9102d1/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 h1:Kq/R4w9hKuNMqffz1w/ucpAb18gCb7saz846Fhlxnjc=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 h1:Qu+gNYpDHsc36L69LjulfAwkZvF9mbXYRiSJi9E1SGA=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJKdCgmByzhCv4IAnjpwDCJH54SYK8iBzctKGOfw=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648 h1:Lhk/aEyFwDflE03kdI6Xeap/awDJbxVsBzKh08U9hTY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/boxo v0.10.0 h1:tdDAxq8jrsbRkYoF+5Rcqyeb91hgWe2hp7iLu7ORZLY=
github.com/ipfs/boxo v0.10.0/go.mod h1:Fg+BnfxZ0RPzR0nOodzdIq3A7KgoWAOWsEIImrIQdBM=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
@@ -690,8 +690,8 @@ go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE=
go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0=
-go.uber.org/dig v1.17.0 h1:5Chju+tUvcC+N7N6EV08BJz41UZuO3BmHcN4A287ZLI=
-go.uber.org/dig v1.17.0/go.mod h1:rTxpf7l5I0eBTlE6/9RL+lDybC7WFwY2QH55ZSjy1mU=
+go.uber.org/dig v1.17.1 h1:Tga8Lz8PcYNsWsyHMZ1Vm0OQOUaJNDyvPImgbAu9YSc=
+go.uber.org/dig v1.17.1/go.mod h1:Us0rSJiThwCv2GteUN0Q7OKvU7n5J4dxZ9JKUXozFdE=
go.uber.org/fx v1.20.0 h1:ZMC/pnRvhsthOZh9MZjMq5U8Or3mA9zBSPaLnzs3ihQ=
go.uber.org/fx v1.20.0/go.mod h1:qCUj0btiR3/JnanEr1TYEePfSw6o/4qYJscgvzQ5Ub0=
go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index 15d39c92a..fcaa47021 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -5,12 +5,12 @@ go 1.21
replace github.com/iotaledger/iota-core => ../../
require (
- github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38
- github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38
+ github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1
+ github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd
github.com/mr-tron/base58 v1.2.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.14.0
@@ -26,14 +26,14 @@ require (
github.com/holiman/uint256 v1.2.3 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 // indirect
- github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 // indirect
+ github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 // indirect
+ github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/klauspost/cpuid/v2 v2.2.5 // indirect
github.com/kr/text v0.2.0 // indirect
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index 1fe1cefe1..f1651ce7e 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -28,32 +28,32 @@ github.com/iancoleman/orderedmap v0.3.0 h1:5cbR2grmZR/DiVt+VJopEhtVs9YGInGIxAoMJ
github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJDkXXS7VoV7XGE=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38 h1:/Td52GNRrMzh0lVKwQu/BbVciwrKn5LwtkWAzd41X4Q=
-github.com/iotaledger/hive.go/ads v0.0.0-20231019113503-7986872a7a38/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38 h1:xYh6+dUl+iocWofo0Z/0/OJGSdxYaPV1oYo1HlFj8+s=
-github.com/iotaledger/hive.go/constraints v0.0.0-20231019113503-7986872a7a38/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38 h1:3OUYK9wgyH4U6xfGnLdU1QdYQ5dpS/6WRju+eXP051E=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019113503-7986872a7a38/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38 h1:iXqM0/Y4poiiuNf+0tu3nUj9Dzp4slrWHJ8GXKgt2Zk=
-github.com/iotaledger/hive.go/crypto v0.0.0-20231019113503-7986872a7a38/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38 h1:xyaKXgNt8CQ4t14Raqu/4+aBRZQunGOlK+T+99egkSA=
-github.com/iotaledger/hive.go/ds v0.0.0-20231019113503-7986872a7a38/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38 h1:6I6PEdLuwiIpmiXFVEW5lPaNe+xFdaBR6NSHsNZ3o6c=
-github.com/iotaledger/hive.go/ierrors v0.0.0-20231019113503-7986872a7a38/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38 h1:SCBXTAhrLFs0o9EJxO9ZBGPyfq1VhOdb5pP7+vFhdtk=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231019113503-7986872a7a38/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38 h1:qks472rk/MjwwFvDrlpckxWTf4c1yd+k7ve/IkeuPYk=
-github.com/iotaledger/hive.go/lo v0.0.0-20231019113503-7986872a7a38/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38 h1:6HIBNKOWLXY5Gk1s+q93D6IsTcqwANigfMO3ePtV3HQ=
-github.com/iotaledger/hive.go/log v0.0.0-20231019113503-7986872a7a38/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38 h1:lPiKbh8o/SN2FGCN9RF7iN1zE+KXCidp815GSOieU8s=
-github.com/iotaledger/hive.go/runtime v0.0.0-20231019113503-7986872a7a38/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38 h1:njGr4TrKdpa66XyioxhDU3slV9gZOZeEg7/M8bjXw7M=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019113503-7986872a7a38/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38 h1:jh0LHzzLqVO+SW9NkvNIgYo1g3CyTBnR8Qv3+PNH+dY=
-github.com/iotaledger/hive.go/stringify v0.0.0-20231019113503-7986872a7a38/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648 h1:Lhk/aEyFwDflE03kdI6Xeap/awDJbxVsBzKh08U9hTY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019112751-e9872df31648/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1 h1:7TnpRtJourFhAEz+/yq9c3zXKSF6cd1xNBHy8jI3zfs=
+github.com/iotaledger/hive.go/ads v0.0.0-20231019175611-f85bce9102d1/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1 h1:ZUd2zsuTG1tT2u3UUpz9b6IRvZRzJwDOWHeBIbRmdwU=
+github.com/iotaledger/hive.go/constraints v0.0.0-20231019175611-f85bce9102d1/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1 h1:o2z+yaMMoLmiQi0IaDjPGWcRaMhC8UiUwbnmkQl4wOU=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231019175611-f85bce9102d1/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1 h1:dFwSvX/p0P2QLQPdXSqiJJQ8eYtd2YI+PFJN/Msh6Jk=
+github.com/iotaledger/hive.go/crypto v0.0.0-20231019175611-f85bce9102d1/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1 h1:3bLEYHGjvctjFHKkH3l1pJEI3iqbcRv423LFahsCFEQ=
+github.com/iotaledger/hive.go/ds v0.0.0-20231019175611-f85bce9102d1/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1 h1:ohOZfLq91DTWuf63i2mbpuHWBqFr7TvZ2Pn8k/U2iXo=
+github.com/iotaledger/hive.go/ierrors v0.0.0-20231019175611-f85bce9102d1/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1 h1:kbQsSCboaEVcrF50ENpFn568RVadvUWpsFe5a/bKUT0=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231019175611-f85bce9102d1/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1 h1:bUxkrvfRnEKcwOuD9HJoL3pNCp0RdkpfMcX1vgV4KAY=
+github.com/iotaledger/hive.go/lo v0.0.0-20231019175611-f85bce9102d1/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1 h1:nP19sKaWMBSql9BTlASjWYWKpiz98zEotiqUGuFYxcg=
+github.com/iotaledger/hive.go/log v0.0.0-20231019175611-f85bce9102d1/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 h1:Kq/R4w9hKuNMqffz1w/ucpAb18gCb7saz846Fhlxnjc=
+github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 h1:Qu+gNYpDHsc36L69LjulfAwkZvF9mbXYRiSJi9E1SGA=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
+github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
From 30b26cf93467990cbdb37ca6b869f180feeb99c6 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 20 Oct 2023 12:38:17 +0200
Subject: [PATCH 03/56] Update .gitignore entry for go.work
---
.gitignore | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/.gitignore b/.gitignore
index b404776d6..f8ded2667 100644
--- a/.gitignore
+++ b/.gitignore
@@ -15,8 +15,7 @@ iota-core
# IDE related files
.vscode/
.idea/
-go.work
-go.work.sum
+go.work*
# dist packages
dist/
From 9a991c4e2dffefdee0c1b8d2168403d8d9b800fc Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 20 Oct 2023 12:38:47 +0200
Subject: [PATCH 04/56] Bump inx and inx-app version
---
go.mod | 4 ++--
go.sum | 4 ++++
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 1 +
4 files changed, 8 insertions(+), 3 deletions(-)
diff --git a/go.mod b/go.mod
index 2466177a3..ca80b4063 100644
--- a/go.mod
+++ b/go.mod
@@ -23,8 +23,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
diff --git a/go.sum b/go.sum
index 41bc1c322..c72f3333f 100644
--- a/go.sum
+++ b/go.sum
@@ -307,8 +307,12 @@ github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:Ko
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJKdCgmByzhCv4IAnjpwDCJH54SYK8iBzctKGOfw=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 h1:P4g4eoNMW0Esz/SCKzkIby1W1TQ3P3j63RR/Zs9XsYU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91/go.mod h1:Pjdgj/uXuyqoUZUmobj8K9ATbdMIOkuNI7L8j4J6ORs=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 h1:EU1O4N5G601nteXW3VVAXF63EkG6TetI4CdQCZiODGU=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7/go.mod h1:UOcPh1kcNvvKmeOvhz8y19+In51g7ogefkh4MDTBphY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/boxo v0.10.0 h1:tdDAxq8jrsbRkYoF+5Rcqyeb91hgWe2hp7iLu7ORZLY=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index b5eac266f..d6b5d83b4 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -72,7 +72,7 @@ require (
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 // indirect
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd // indirect
github.com/ipfs/boxo v0.10.0 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 5991ebd59..5a6cc53be 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -313,6 +313,7 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJ
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7/go.mod h1:UOcPh1kcNvvKmeOvhz8y19+In51g7ogefkh4MDTBphY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
github.com/ipfs/boxo v0.10.0 h1:tdDAxq8jrsbRkYoF+5Rcqyeb91hgWe2hp7iLu7ORZLY=
From a8fc93f117d7484031af8c4e8a9bdb78724a1d22 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 20 Oct 2023 13:15:34 +0200
Subject: [PATCH 05/56] Run gendoc
---
config_defaults.json | 9 -----
.../docs/references/configuration.md | 34 ++-----------------
go.sum | 4 ---
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 7 ++--
5 files changed, 7 insertions(+), 49 deletions(-)
diff --git a/config_defaults.json b/config_defaults.json
index 2dc55de16..21ad064c4 100644
--- a/config_defaults.json
+++ b/config_defaults.json
@@ -116,15 +116,6 @@
"useMetricPrefix": false
}
},
- "validator": {
- "enabled": false,
- "committeeBroadcastInterval": "500ms",
- "candidateBroadcastInterval": "30m",
- "parentsCount": 8,
- "ignoreBootstrapped": false,
- "account": "",
- "privateKey": ""
- },
"dashboard": {
"enabled": true,
"bindAddress": "0.0.0.0:8081",
diff --git a/documentation/docs/references/configuration.md b/documentation/docs/references/configuration.md
index acf7bfa3f..2479ad48b 100644
--- a/documentation/docs/references/configuration.md
+++ b/documentation/docs/references/configuration.md
@@ -375,35 +375,7 @@ Example:
}
```
-## 10. Validator
-
-| Name | Description | Type | Default value |
-| -------------------------- | ------------------------------------------------------------------------------------------------------------ | ------- | ------------- |
-| enabled | Whether the Validator component is enabled | boolean | false |
-| committeeBroadcastInterval | The interval at which the node will broadcast its committee validator block | string | "500ms" |
-| candidateBroadcastInterval | The interval at which the node will broadcast its candidate validator block | string | "30m" |
-| parentsCount | The number of parents that node will choose for its validator blocks | int | 8 |
-| ignoreBootstrapped | Whether the Validator component should start issuing validator blocks before the main engine is bootstrapped | boolean | false |
-| account | The accountID of the validator account that will issue the blocks | string | "" |
-| privateKey | The private key of the validator account that will issue the blocks | string | "" |
-
-Example:
-
-```json
- {
- "validator": {
- "enabled": false,
- "committeeBroadcastInterval": "500ms",
- "candidateBroadcastInterval": "30m",
- "parentsCount": 8,
- "ignoreBootstrapped": false,
- "account": "",
- "privateKey": ""
- }
- }
-```
-
-## 11. Dashboard
+## 10. Dashboard
| Name | Description | Type | Default value |
| --------------------------------- | --------------------------------------- | ------- | -------------- |
@@ -445,7 +417,7 @@ Example:
}
```
-## 12. Metrics
+## 11. Metrics
| Name | Description | Type | Default value |
| --------------- | ---------------------------------------------------- | ------- | -------------- |
@@ -469,7 +441,7 @@ Example:
}
```
-## 13. Inx
+## 12. Inx
| Name | Description | Type | Default value |
| ----------- | ------------------------------------------------------ | ------- | ---------------- |
diff --git a/go.sum b/go.sum
index c72f3333f..588e5e3fb 100644
--- a/go.sum
+++ b/go.sum
@@ -305,12 +305,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJKdCgmByzhCv4IAnjpwDCJH54SYK8iBzctKGOfw=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 h1:P4g4eoNMW0Esz/SCKzkIby1W1TQ3P3j63RR/Zs9XsYU=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91/go.mod h1:Pjdgj/uXuyqoUZUmobj8K9ATbdMIOkuNI7L8j4J6ORs=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 h1:EU1O4N5G601nteXW3VVAXF63EkG6TetI4CdQCZiODGU=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7/go.mod h1:UOcPh1kcNvvKmeOvhz8y19+In51g7ogefkh4MDTBphY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index d6b5d83b4..8c3047857 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -71,7 +71,7 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231019175611-f85bce9102d1 // indirect
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1 // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd // indirect
github.com/ipfs/boxo v0.10.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 5a6cc53be..d635c0d48 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -309,10 +309,9 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231019175611-f85bce9102d1/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1 h1:KoN5DlV4m139RtoBMwKxVJiItttdwCn0CgWNzzCB3Tc=
github.com/iotaledger/hive.go/stringify v0.0.0-20231019175611-f85bce9102d1/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b h1:bJKLJKdCgmByzhCv4IAnjpwDCJH54SYK8iBzctKGOfw=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020100823-ac958da9ec1b/go.mod h1:jVBbv/VLDjxw3HthXXcCr1WFzOj48ODUJyxQQUmq6Ac=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad h1:TRM9EkAole9fYY1vHEVQ6zCEOGuvCWq/bczZ98Al5Ec=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231011154428-257141868dad/go.mod h1:plZ0+8yLdDWHedj3SfHUwQtIETD+lcS6M1iEAxcjzJ4=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 h1:P4g4eoNMW0Esz/SCKzkIby1W1TQ3P3j63RR/Zs9XsYU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91/go.mod h1:Pjdgj/uXuyqoUZUmobj8K9ATbdMIOkuNI7L8j4J6ORs=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 h1:EU1O4N5G601nteXW3VVAXF63EkG6TetI4CdQCZiODGU=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7/go.mod h1:UOcPh1kcNvvKmeOvhz8y19+In51g7ogefkh4MDTBphY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
From e0b61d96896bd6032032f9199e45710d64de9c08 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 20 Oct 2023 17:20:00 +0200
Subject: [PATCH 06/56] Add validators to docker-compose.yml
---
tools/docker-network/.env | 2 +-
tools/docker-network/docker-compose.yml | 169 +++++++++++++++---------
tools/docker-network/prometheus.yml | 6 +-
tools/docker-network/run.sh | 37 +++---
tools/evil-spammer/identity/config.go | 6 +-
5 files changed, 130 insertions(+), 90 deletions(-)
diff --git a/tools/docker-network/.env b/tools/docker-network/.env
index ce7903fb0..d953c5463 100644
--- a/tools/docker-network/.env
+++ b/tools/docker-network/.env
@@ -16,5 +16,5 @@ COMMON_CONFIG="
--protocol.snapshot.path=/app/data/snapshot.bin
"
MANUALPEERING_CONFIG="
---p2p.peers=/dns/validator-1/tcp/14666/p2p/12D3KooWRVt4Engu27jHnF2RjfX48EqiAqJbgLfFdHNt3Vn6BtJK\
+--p2p.peers=/dns/node-1/tcp/14666/p2p/12D3KooWRVt4Engu27jHnF2RjfX48EqiAqJbgLfFdHNt3Vn6BtJK\
"
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 92ae8e023..4405f4e7b 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -4,78 +4,68 @@ x-build-iota-core: &iota-core_build
context: ${DOCKER_BUILD_CONTEXT:-../../}
dockerfile: ${DOCKERFILE_PATH:-./Dockerfile}
services:
- validator-1:
+ node-1:
build: *iota-core_build
stop_grace_period: 1m
command: >
${COMMON_CONFIG}
${MANUALPEERING_CONFIG}
--p2p.identityPrivateKey=08735375679f3d8031353e94282ed1d65119e5c288fe56d6639d9184a3f978fee8febfedff11cc376daea0f59c395ae2e9a870a25ac4e36093000fbf4d0e8f18
- --validator.enabled=true
- --validator.ignoreBootstrapped=true
- --validator.account=0x907c02e9302e0f0571f10f885594e56d8c54ff0708ab7a39bc1b74d396b93b12
- --validator.privateKey=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b
--inx.enabled=true
--inx.bindAddress=0.0.0.0:9029
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
ports:
- - "8080:8080/tcp" # web API
- - "8081:8081/tcp" # dashboard
- - "6081:6061/tcp" # pprof
- - "9089:9029/tcp" # inx
+ - "8010:8080/tcp" # web API
+ - "8011:8081/tcp" # dashboard
+ - "6011:6061/tcp" # pprof
+ - "9019:9029/tcp" # inx
networks:
- iota-core
- validator-2:
- image: docker-network-validator-1:latest
+ node-2:
+ image: docker-network-node-1:latest
stop_grace_period: 1m
command: >
${COMMON_CONFIG}
${MANUALPEERING_CONFIG}
--p2p.identityPrivateKey=ba771419c52132a0dfb2521ed18667813f398da159010a55a0a482af939affb92d3338789ad4a07a7631b91791deb11f82ed5dc612822f24275e9f7a313b691f
- --validator.enabled=true
- --validator.account=0x375358f92cc94750669598b0aaa55a6ff73310b90710e1fad524c0f911be0fea
- --validator.privateKey=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064
--inx.enabled=true
--inx.bindAddress=0.0.0.0:9029
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
ports:
- - "8070:8080/tcp" # web API
- - "8071:8081/tcp" # dashboard
- - "6071:6061/tcp" # pprof
+ - "8020:8080/tcp" # web API
+ - "8021:8081/tcp" # dashboard
+ - "6021:6061/tcp" # pprof
- "9029:9029/tcp" # inx
networks:
- iota-core
- validator-3:
- image: docker-network-validator-1:latest
+ node-3:
+ image: docker-network-node-1:latest
stop_grace_period: 1m
command: >
${COMMON_CONFIG}
${MANUALPEERING_CONFIG}
--p2p.identityPrivateKey=a6261ac049755675ff1437654ca9f83b305055f01ff08c4f039209ef5a4a7d96d06fb61df77a8815209a8f4d204226dee593e50d0ec897ec440a2c1fbde77656
- --validator.enabled=true
- --validator.account=0x6aee704f25558e8aa7630fed0121da53074188abc423b3c5810f80be4936eb6e
- --validator.privateKey=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648
--inx.enabled=true
--inx.bindAddress=0.0.0.0:9029
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
ports:
- - "8090:8080/tcp" # web API
- - "8091:8081/tcp" # dashboard
- - "6091:6061/tcp" # pprof
- - "9099:9029/tcp" # inx
+ - "8030:8080/tcp" # web API
+ - "8031:8081/tcp" # dashboard
+ - "6031:6061/tcp" # pprof
+ - "9039:9029/tcp" # inx
networks:
- iota-core
- node-1:
- image: docker-network-validator-1:latest
+ node-4:
+ image: docker-network-node-1:latest
stop_grace_period: 1m
command: >
${COMMON_CONFIG}
@@ -94,8 +84,8 @@ services:
networks:
- iota-core
- node-2:
- image: docker-network-validator-1:latest
+ node-5:
+ image: docker-network-node-1:latest
stop_grace_period: 1m
command: >
${COMMON_CONFIG}
@@ -107,10 +97,10 @@ services:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
ports:
- - "8030:8080/tcp" # web API
- - "8031:8081/tcp" # dashboard
- - "6031:6061/tcp" # pprof
- - "9039:9029/tcp" # inx
+ - "8050:8080/tcp" # web API
+ - "8051:8081/tcp" # dashboard
+ - "6051:6061/tcp" # pprof
+ - "9059:9029/tcp" # inx
networks:
- iota-core
@@ -118,53 +108,104 @@ services:
image: iotaledger/inx-indexer:2.0-alpha
stop_grace_period: 1m
depends_on:
- validator-1:
+ node-1:
condition: service_started
command:
- - "--inx.address=validator-1:9029"
+ - "--inx.address=node-1:9029"
- "--restAPI.bindAddress=inx-indexer:9091"
networks:
- iota-core
- inx-blockissuer:
- image: iotaledger/inx-blockissuer:1.0-alpha
- stop_grace_period: 1m
- depends_on:
- validator-1:
- condition: service_started
- inx-indexer:
- condition: service_started
- restart: on-failure
- environment:
- - "BLOCKISSUER_PRV_KEY=432c624ca3260f910df35008d5c740593b222f1e196e6cdb8cd1ad080f0d4e33997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270"
- command:
- - "--inx.address=validator-1:9029"
- - "--restAPI.bindAddress=inx-blockissuer:9086"
- - "--blockIssuer.accountAddress=rms1prkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d5c6gkun"
- networks:
- - iota-core
+# inx-blockissuer:
+# image: iotaledger/inx-blockissuer:1.0-alpha
+# stop_grace_period: 1m
+# depends_on:
+# node-1:
+# condition: service_started
+# inx-indexer:
+# condition: service_started
+# restart: on-failure
+# environment:
+# - "BLOCKISSUER_PRV_KEY=432c624ca3260f910df35008d5c740593b222f1e196e6cdb8cd1ad080f0d4e33997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270"
+# command:
+# - "--inx.address=node-1:9029"
+# - "--restAPI.bindAddress=inx-blockissuer:9086"
+# - "--blockIssuer.accountAddress=rms1prkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d5c6gkun"
+# networks:
+# - iota-core
+#
+# inx-faucet:
+# image: iotaledger/inx-faucet:2.0-alpha
+# stop_grace_period: 1m
+# depends_on:
+# node-1:
+# condition: service_started
+# inx-indexer:
+# condition: service_started
+# inx-blockissuer:
+# condition: service_started
+# restart: on-failure
+# environment:
+# - "FAUCET_PRV_KEY=de52b9964dda96564e9fab362ab16c2669c715c6a2a853bece8a25fc58c599755b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737"
+# command:
+# - "--inx.address=node-1:9029"
+# - "--faucet.bindAddress=inx-faucet:8091"
+# ports:
+# - "8088:8091/tcp" # faucet frontend
+# networks:
+# - iota-core
- inx-faucet:
- image: iotaledger/inx-faucet:2.0-alpha
+ inx-validator-1:
+ image: iotaledger/inx-validator:latest
stop_grace_period: 1m
depends_on:
- validator-1:
+ node-1:
condition: service_started
inx-indexer:
condition: service_started
- inx-blockissuer:
- condition: service_started
restart: on-failure
environment:
- - "FAUCET_PRV_KEY=de52b9964dda96564e9fab362ab16c2669c715c6a2a853bece8a25fc58c599755b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737"
+ - "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
command:
- - "--inx.address=validator-1:9029"
- - "--faucet.bindAddress=inx-faucet:8091"
- ports:
- - "8088:8091/tcp" # faucet frontend
+ - "--inx.address=node-1:9029"
+ - "--validator.ignoreBootstrapped=true"
+ - "--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6"
networks:
- iota-core
+ inx-validator-2:
+ image: iotaledger/inx-validator:latest
+ stop_grace_period: 1m
+ depends_on:
+ node-1:
+ condition: service_started
+ inx-indexer:
+ condition: service_started
+ restart: on-failure
+ environment:
+ - "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
+ command:
+ - "--inx.address=node-2:9029"
+ - "--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl"
+ networks:
+ - iota-core
+
+ inx-validator-3:
+ image: iotaledger/inx-validator:latest
+ stop_grace_period: 1m
+ depends_on:
+ node-1:
+ condition: service_started
+ inx-indexer:
+ condition: service_started
+ restart: on-failure
+ environment:
+ - "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
+ command:
+ - "--inx.address=node-3:9029"
+ - "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
+ networks:
+ - iota-core
prometheus:
image: prom/prometheus:latest
profiles:
@@ -178,7 +219,7 @@ services:
networks:
- iota-core
depends_on:
- - validator-1
+ - node-1
grafana:
image: grafana/grafana:9.5.6
diff --git a/tools/docker-network/prometheus.yml b/tools/docker-network/prometheus.yml
index 28817e795..3b64a1c47 100644
--- a/tools/docker-network/prometheus.yml
+++ b/tools/docker-network/prometheus.yml
@@ -3,11 +3,11 @@ scrape_configs:
scrape_interval: 5s
static_configs:
- targets:
- - validator-1:9311
- - validator-2:9311
- - validator-3:9311
- node-1:9311
- node-2:9311
+ - node-3:9311
+ - node-4:9311
+ - node-5:9311
dns_sd_configs:
- names:
- 'peer_replica'
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 46a728c16..0ee2fbfe7 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -1,12 +1,16 @@
#!/bin/bash
# Create a function to join an array of strings by a given character
-function join { local IFS="$1"; shift; echo "$*"; }
+function join {
+ local IFS="$1"
+ shift
+ echo "$*"
+}
# All parameters can be optional now, just make sure we don't have too many
-if [[ $# -gt 4 ]] ; then
- echo 'Call with ./run [replicas=1|2|3|...] [grafana=0|1] [feature=0|1]'
- exit 0
+if [[ $# -gt 4 ]]; then
+ echo 'Call with ./run [replicas=1|2|3|...] [grafana=0|1] [feature=0|1]'
+ exit 0
fi
REPLICAS=${1:-1}
@@ -14,12 +18,10 @@ GRAFANA=${2:-0}
FEATURE=${3:-0}
DOCKER_COMPOSE_FILE=docker-compose.yml
-if [ $FEATURE -ne 0 ]
-then
+if [ $FEATURE -ne 0 ]; then
DOCKER_COMPOSE_FILE=docker-compose-feature.yml
fi
-
export DOCKER_BUILDKIT=1
export COMPOSE_DOCKER_CLI_BUILD=1
echo "Build iota-core"
@@ -28,8 +30,7 @@ echo "Build iota-core"
export DOCKER_BUILD_CONTEXT="../../"
export DOCKERFILE_PATH="./Dockerfile.dev"
-if [[ "$WITH_GO_WORK" -eq 1 ]]
-then
+if [[ "$WITH_GO_WORK" -eq 1 ]]; then
export DOCKER_BUILD_CONTEXT="../../../"
export DOCKERFILE_PATH="./iota-core/Dockerfile.dev"
fi
@@ -38,36 +39,34 @@ fi
echo $DOCKER_BUILD_CONTEXT $DOCKERFILE_PATH
docker compose -f $DOCKER_COMPOSE_FILE build --build-arg WITH_GO_WORK=${WITH_GO_WORK:-0} --build-arg DOCKER_BUILD_CONTEXT=${DOCKER_BUILD_CONTEXT} --build-arg DOCKERFILE_PATH=${DOCKERFILE_PATH}
-docker compose pull inx-indexer inx-blockissuer inx-faucet
+docker compose pull inx-indexer # inx-blockissuer inx-faucet
# check exit code of builder
-if [ $? -ne 0 ]
-then
+if [ $? -ne 0 ]; then
echo "Building failed. Please fix and try again!"
exit 1
fi
# create snapshot file
echo "Create snapshot"
-if [ $FEATURE -ne 0 ]
-then
- pushd ../genesis-snapshot; go run -tags=rocksdb . --config feature
+if [ $FEATURE -ne 0 ]; then
+ pushd ../genesis-snapshot
+ go run -tags=rocksdb . --config feature
else
- pushd ../genesis-snapshot; go run -tags=rocksdb . --config docker --seed 7R1itJx5hVuo9w9hjg5cwKFmek4HMSoBDgJZN8hKGxih
+ pushd ../genesis-snapshot
+ go run -tags=rocksdb . --config docker --seed 7R1itJx5hVuo9w9hjg5cwKFmek4HMSoBDgJZN8hKGxih
fi
popd
mv ../genesis-snapshot/*.snapshot .
chmod o+r *.snapshot
-
echo "Run iota-core network with ${DOCKER_COMPOSE_FILE}"
# IOTA_CORE_PEER_REPLICAS is used in docker-compose.yml to determine how many replicas to create
export IOTA_CORE_PEER_REPLICAS=$REPLICAS
# Profiles is created to set which docker profiles to run
# https://docs.docker.com/compose/profiles/
PROFILES=()
-if [ $GRAFANA -ne 0 ]
-then
+if [ $GRAFANA -ne 0 ]; then
PROFILES+=("grafana")
fi
diff --git a/tools/evil-spammer/identity/config.go b/tools/evil-spammer/identity/config.go
index 4529b5055..ebd279482 100644
--- a/tools/evil-spammer/identity/config.go
+++ b/tools/evil-spammer/identity/config.go
@@ -14,15 +14,15 @@ var Config = Params{}
var identityConfigJSON = `{
"docker": {
- "validator-1": [
+ "node-1": [
"8q491c3YWjbPwLmF2WD95YmCgh61j2kenCKHfGfByoWi",
"http://localhost:8080"
],
- "validator-2": [
+ "node-2": [
"4ata8GcTRMJ5sSv2jQJWmSYYTHct748p3tXmCFYm7wjA",
"http://localhost:8070"
],
- "validator-3": [
+ "node-3": [
"3YX6e7AL28hHihZewKdq6CMkEYVsTJBLgRiprUNiNq5E",
"http://localhost:8090"
]
From 80f6841dcf7e293bc19c701bc57e18ce9302c434 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 24 Oct 2023 12:22:24 +0200
Subject: [PATCH 07/56] Update dependency versions.
---
go.mod | 4 ++--
go.sum | 8 ++++----
tools/evil-spammer/go.mod | 2 +-
tools/evil-spammer/go.sum | 4 ++--
tools/gendoc/go.mod | 4 ++--
tools/gendoc/go.sum | 8 ++++----
tools/genesis-snapshot/go.mod | 2 +-
tools/genesis-snapshot/go.sum | 4 ++--
8 files changed, 18 insertions(+), 18 deletions(-)
diff --git a/go.mod b/go.mod
index 5d4cf4870..5f6416868 100644
--- a/go.mod
+++ b/go.mod
@@ -24,8 +24,8 @@ require (
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7
- github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
github.com/libp2p/go-libp2p v0.31.0
diff --git a/go.sum b/go.sum
index 09a448315..1293fb9d3 100644
--- a/go.sum
+++ b/go.sum
@@ -307,10 +307,10 @@ github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MD
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 h1:P4g4eoNMW0Esz/SCKzkIby1W1TQ3P3j63RR/Zs9XsYU=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91/go.mod h1:Pjdgj/uXuyqoUZUmobj8K9ATbdMIOkuNI7L8j4J6ORs=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7 h1:EU1O4N5G601nteXW3VVAXF63EkG6TetI4CdQCZiODGU=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231020100621-739388b552f7/go.mod h1:UOcPh1kcNvvKmeOvhz8y19+In51g7ogefkh4MDTBphY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd h1:FKiEzy3gOyWDGXs6mTMS9qFHGXSxH97N5w+08ljbEtU=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231019174124-aa2290512bcd/go.mod h1:XFwcDfAF0hv6wB/trCsyDmQyPLjEZ60yTrllJ2bprN8=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 h1:B0Ht9EijMtHJhbkVNEC8P5hbG3Px+/ClWVD3AKXPjSQ=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/evil-spammer/go.mod b/tools/evil-spammer/go.mod
index 91ec4b3b6..152c841ad 100644
--- a/tools/evil-spammer/go.mod
+++ b/tools/evil-spammer/go.mod
@@ -19,7 +19,7 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
github.com/iotaledger/iota-core/tools/genesis-snapshot v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d
github.com/mr-tron/base58 v1.2.0
go.uber.org/atomic v1.11.0
)
diff --git a/tools/evil-spammer/go.sum b/tools/evil-spammer/go.sum
index f66d3f1bb..cf5492131 100644
--- a/tools/evil-spammer/go.sum
+++ b/tools/evil-spammer/go.sum
@@ -201,8 +201,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8 h1:81aWESXFC04iKI9I140eDrBb9zBWXfVoAUMp9berk0c=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 65a416839..66fec4544 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -72,8 +72,8 @@ require (
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b // indirect
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd // indirect
- github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8 // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 // indirect
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index c13517eea..33bc76954 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -311,10 +311,10 @@ github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MD
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 h1:/8pbFXhTSroJvjJMfJqfHjzoT9N8B4LUY3SbKruD5MM=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0/go.mod h1:My1SB4vZj42EgTDNJ/dgW8lUpLNmvtzu8f89J5y2kP0=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd h1:hh5mAnnaZHOYAi4CIqR9K/mv786ex9AQgpisbJ4ZMow=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8 h1:81aWESXFC04iKI9I140eDrBb9zBWXfVoAUMp9berk0c=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 h1:B0Ht9EijMtHJhbkVNEC8P5hbG3Px+/ClWVD3AKXPjSQ=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index 0743c2ff7..adc7cdb7f 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -10,7 +10,7 @@ require (
github.com/iotaledger/hive.go/lo v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d
github.com/mr-tron/base58 v1.2.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.14.0
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index f50a589ed..0154c5692 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -52,8 +52,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8 h1:81aWESXFC04iKI9I140eDrBb9zBWXfVoAUMp9berk0c=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023190719-1a9daaa83ca8/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
From 970fea1f0b5687d9f4ba7fc1fac86cfe15a8d7b9 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 24 Oct 2023 13:14:49 +0200
Subject: [PATCH 08/56] Update dependencies.
---
go.mod | 2 +-
go.sum | 4 ++--
.../sybilprotectionv1/performance/performance.go | 12 ++++++++++--
tools/docker-network/docker-compose.yml | 3 +++
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 4 ++--
6 files changed, 19 insertions(+), 8 deletions(-)
diff --git a/go.mod b/go.mod
index 5f6416868..01e7baae5 100644
--- a/go.mod
+++ b/go.mod
@@ -23,7 +23,7 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941
github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d
github.com/labstack/echo/v4 v4.11.2
diff --git a/go.sum b/go.sum
index 1293fb9d3..67ff3a02a 100644
--- a/go.sum
+++ b/go.sum
@@ -305,8 +305,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91 h1:P4g4eoNMW0Esz/SCKzkIby1W1TQ3P3j63RR/Zs9XsYU=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231020103531-8702a4719b91/go.mod h1:Pjdgj/uXuyqoUZUmobj8K9ATbdMIOkuNI7L8j4J6ORs=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff h1:cLRjTHMuWaTz1LpEF9KNbsSDaH2OTiIM/FDAcwL3ohU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff/go.mod h1:0/7o9/MkkuKNhcEMz6xhPQLupf/OskjGigbmvnJtDL0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 h1:B0Ht9EijMtHJhbkVNEC8P5hbG3Px+/ClWVD3AKXPjSQ=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
diff --git a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
index 0a2468466..ab0099044 100644
--- a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
+++ b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
@@ -7,7 +7,6 @@ import (
"github.com/iotaledger/hive.go/ds"
"github.com/iotaledger/hive.go/ierrors"
"github.com/iotaledger/hive.go/kvstore"
- "github.com/iotaledger/hive.go/lo"
"github.com/iotaledger/hive.go/runtime/syncutils"
"github.com/iotaledger/iota-core/pkg/core/account"
"github.com/iotaledger/iota-core/pkg/model"
@@ -85,7 +84,16 @@ func (t *Tracker) EligibleValidatorCandidates(epoch iotago.EpochIndex) ds.Set[io
eligible := ds.NewSet[iotago.AccountID]()
- lo.PanicOnErr(t.committeeStore.Load(epoch - 1)).ForEach(func(accountID iotago.AccountID, _ *account.Pool) bool {
+ // Avoid underflow error. This will be handled properly once TopStakers branch is merged.
+ if epoch == 0 {
+ epoch = 1
+ }
+
+ committee, exists := t.LoadCommitteeForEpoch(epoch - 1)
+ if !exists {
+ panic(ierrors.Errorf("committee for epoch %d does not exist", epoch-1))
+ }
+ committee.ForEach(func(accountID iotago.AccountID, _ *account.Pool) bool {
eligible.Add(accountID)
return true
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 4405f4e7b..9cd180601 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -167,6 +167,7 @@ services:
environment:
- "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
command:
+ - "--logger.level=debug"
- "--inx.address=node-1:9029"
- "--validator.ignoreBootstrapped=true"
- "--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6"
@@ -185,6 +186,7 @@ services:
environment:
- "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
command:
+ - "--logger.level=debug"
- "--inx.address=node-2:9029"
- "--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl"
networks:
@@ -202,6 +204,7 @@ services:
environment:
- "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
command:
+ - "--logger.level=debug"
- "--inx.address=node-3:9029"
- "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
networks:
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 66fec4544..1a8d2fa96 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -71,7 +71,7 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d // indirect
github.com/ipfs/boxo v0.13.1 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 33bc76954..675a67e98 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -309,8 +309,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 h1:/8pbFXhTSroJvjJMfJqfHjzoT9N8B4LUY3SbKruD5MM=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0/go.mod h1:My1SB4vZj42EgTDNJ/dgW8lUpLNmvtzu8f89J5y2kP0=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff h1:cLRjTHMuWaTz1LpEF9KNbsSDaH2OTiIM/FDAcwL3ohU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff/go.mod h1:0/7o9/MkkuKNhcEMz6xhPQLupf/OskjGigbmvnJtDL0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 h1:B0Ht9EijMtHJhbkVNEC8P5hbG3Px+/ClWVD3AKXPjSQ=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
From 52732056fe2cb205298b54ce24ec499dc52f3860 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Wed, 25 Oct 2023 21:51:16 +0200
Subject: [PATCH 09/56] Post merge fixes
---
components/inx/server_accounts.go | 13 +++++++++++--
go.mod | 2 +-
go.sum | 4 ++--
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 4 ++--
tools/genesis-snapshot/go.mod | 2 +-
tools/genesis-snapshot/go.sum | 4 ++--
7 files changed, 20 insertions(+), 11 deletions(-)
diff --git a/components/inx/server_accounts.go b/components/inx/server_accounts.go
index 998120dd6..cb3fe68ad 100644
--- a/components/inx/server_accounts.go
+++ b/components/inx/server_accounts.go
@@ -29,8 +29,12 @@ func (s *Server) ReadIsCommitteeMember(_ context.Context, accountInfoRequest *in
if err != nil {
return nil, ierrors.Wrap(err, "error when parsing account id")
}
+ committee, exists := deps.Protocol.MainEngineInstance().SybilProtection.SeatManager().CommitteeInSlot(slot)
+ if !exists {
+ return nil, ierrors.Errorf("committee does not exist for slot %d", slot)
+ }
- return inx.WrapBoolResponse(deps.Protocol.MainEngineInstance().SybilProtection.SeatManager().Committee(slot).HasAccount(accountID)), nil
+ return inx.WrapBoolResponse(committee.HasAccount(accountID)), nil
}
func (s *Server) ReadIsCandidate(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
@@ -40,5 +44,10 @@ func (s *Server) ReadIsCandidate(_ context.Context, accountInfoRequest *inx.Acco
return nil, ierrors.Wrap(err, "error when parsing account id")
}
- return inx.WrapBoolResponse(deps.Protocol.MainEngineInstance().SybilProtection.IsCandidateActive(accountID, deps.Protocol.APIForSlot(slot).TimeProvider().EpochFromSlot(slot))), nil
+ isCandidateActive, err := deps.Protocol.MainEngineInstance().SybilProtection.IsCandidateActive(accountID, deps.Protocol.APIForSlot(slot).TimeProvider().EpochFromSlot(slot))
+ if err != nil {
+ return nil, ierrors.Wrap(err, "error when checking if candidate is active")
+ }
+
+ return inx.WrapBoolResponse(isCandidateActive), nil
}
diff --git a/go.mod b/go.mod
index 3895dd8e8..ae91ca026 100644
--- a/go.mod
+++ b/go.mod
@@ -25,7 +25,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd
- github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
github.com/libp2p/go-libp2p v0.31.0
diff --git a/go.sum b/go.sum
index 40540cd56..ea83849f2 100644
--- a/go.sum
+++ b/go.sum
@@ -309,8 +309,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 h1:/8pbF
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0/go.mod h1:My1SB4vZj42EgTDNJ/dgW8lUpLNmvtzu8f89J5y2kP0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd h1:hh5mAnnaZHOYAi4CIqR9K/mv786ex9AQgpisbJ4ZMow=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1 h1:7ChiFHg4KsdS4mKjYpFUUNe9t9nsQzNtJ/XnUV2w5ZM=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba h1:/OYVTnZTxOX87CrdZ86HwRjiVrxhkdy36/ePSq/EN24=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 1a8d2fa96..79aae10a9 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -73,7 +73,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b // indirect
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 // indirect
- github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d // indirect
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 675a67e98..d6d77e984 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -313,8 +313,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff h1:cLRjT
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231024101555-660b434164ff/go.mod h1:0/7o9/MkkuKNhcEMz6xhPQLupf/OskjGigbmvnJtDL0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941 h1:B0Ht9EijMtHJhbkVNEC8P5hbG3Px+/ClWVD3AKXPjSQ=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231024101228-5d9abdd08941/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d h1:vGfZmcCCloTzim6TysS3fXxc9xsTfXoVB6bsedZgp7A=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231023205010-58a0b5c7fe6d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba h1:/OYVTnZTxOX87CrdZ86HwRjiVrxhkdy36/ePSq/EN24=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index 080b961b4..13e737d52 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -10,7 +10,7 @@ require (
github.com/iotaledger/hive.go/lo v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba
github.com/mr-tron/base58 v1.2.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.14.0
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index fd8d7e3ec..637a77ed9 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -52,8 +52,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1 h1:7ChiFHg4KsdS4mKjYpFUUNe9t9nsQzNtJ/XnUV2w5ZM=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231025071930-7cc5b35d50b1/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba h1:/OYVTnZTxOX87CrdZ86HwRjiVrxhkdy36/ePSq/EN24=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
From 2b5a4ca9d3225685ce1db9d14d7fffb4e409b042 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 26 Oct 2023 09:44:03 +0200
Subject: [PATCH 10/56] Fix docker-compose after merge
---
tools/docker-network/docker-compose.yml | 75 +++++++++++--------------
1 file changed, 33 insertions(+), 42 deletions(-)
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 08e8d2290..68e85c0b2 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -8,7 +8,6 @@ services:
###################
# IOTA-CORE Nodes #
###################
-
node-1-validator:
build: *iota-core_build
stop_grace_period: 1m
@@ -194,7 +193,7 @@ services:
--inx.address=node-1-validator:9019
--restAPI.bindAddress=inx-indexer:9011
-inx-blockissuer:
+ inx-blockissuer:
image: iotaledger/inx-blockissuer:1.0-alpha
stop_grace_period: 1m
restart: unless-stopped
@@ -244,7 +243,6 @@ inx-blockissuer:
condition: service_started
inx-indexer:
condition: service_started
- restart: on-failure
environment:
- "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
command:
@@ -254,49 +252,42 @@ inx-blockissuer:
- "--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6"
networks:
- iota-core
- environment:
- - "BLOCKISSUER_PRV_KEY=432c624ca3260f910df35008d5c740593b222f1e196e6cdb8cd1ad080f0d4e33997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270"
- command: >
- --inx.address=node-1-validator:9029
- --restAPI.bindAddress=inx-blockissuer:9086
- --blockIssuer.accountAddress=rms1prkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d5c6gkun
- --blockIssuer.proofOfWork.targetTrailingZeros=5
inx-validator-2:
- image: iotaledger/inx-validator:latest
- stop_grace_period: 1m
- depends_on:
- node-1:
- condition: service_started
- inx-indexer:
- condition: service_started
- restart: on-failure
- environment:
- - "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
- command:
- - "--logger.level=debug"
- - "--inx.address=node-2:9029"
- - "--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl"
- networks:
- - iota-core
+ image: iotaledger/inx-validator:latest
+ stop_grace_period: 1m
+ restart: unless-stopped
+ depends_on:
+ node-2-validator:
+ condition: service_started
+ inx-indexer:
+ condition: service_started
+ environment:
+ - "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
+ command:
+ - "--logger.level=debug"
+ - "--inx.address=node-2-validator:9029"
+ - "--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl"
+ networks:
+ - iota-core
inx-validator-3:
- image: iotaledger/inx-validator:latest
- stop_grace_period: 1m
- depends_on:
- node-1:
- condition: service_started
- inx-indexer:
- condition: service_started
- restart: on-failure
- environment:
- - "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
- command:
- - "--logger.level=debug"
- - "--inx.address=node-3:9029"
- - "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
- networks:
- - iota-core
+ image: iotaledger/inx-validator:latest
+ stop_grace_period: 1m
+ restart: unless-stopped
+ depends_on:
+ node-3-validator:
+ condition: service_started
+ inx-indexer:
+ condition: service_started
+ environment:
+ - "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
+ command:
+ - "--logger.level=debug"
+ - "--inx.address=node-3:9029"
+ - "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
+ networks:
+ - iota-core
# Create our own network
networks:
From 8cb895a91f703003e51692afb395b8ff7c8fd7bd Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 26 Oct 2023 09:44:14 +0200
Subject: [PATCH 11/56] Add ForceCommitUntil and ActiveRootBlocks methods
---
components/inx/server_blocks.go | 6 ++++++
components/inx/server_commitments.go | 8 ++++++++
go.mod | 4 ++--
go.sum | 4 ++++
4 files changed, 20 insertions(+), 2 deletions(-)
diff --git a/components/inx/server_blocks.go b/components/inx/server_blocks.go
index 77a5e9b77..b73cd1941 100644
--- a/components/inx/server_blocks.go
+++ b/components/inx/server_blocks.go
@@ -16,6 +16,12 @@ import (
iotago "github.com/iotaledger/iota.go/v4"
)
+func (s *Server) ActiveRootBlocks(_ context.Context, _ *inx.NoParams) *inx.RootBlocksResponse {
+ activeRootBlocks := deps.Protocol.MainEngineInstance().EvictionState.ActiveRootBlocks()
+
+ return inx.WrapRootBlocks(activeRootBlocks)
+}
+
func (s *Server) ReadBlock(_ context.Context, blockID *inx.BlockId) (*inx.RawBlock, error) {
blkID := blockID.Unwrap()
block, exists := deps.Protocol.MainEngineInstance().Block(blkID) // block +1
diff --git a/components/inx/server_commitments.go b/components/inx/server_commitments.go
index 7dd44ba84..94926794f 100644
--- a/components/inx/server_commitments.go
+++ b/components/inx/server_commitments.go
@@ -22,6 +22,14 @@ func inxCommitment(commitment *model.Commitment) *inx.Commitment {
}
}
+func (s *Server) ForceCommitUntil(_ context.Context, slot *inx.SlotIndex) error {
+ err := deps.Protocol.MainEngineInstance().Notarization.ForceCommitUntil(slot.Unwrap())
+ if err != nil {
+ return ierrors.Wrapf(err, "error while performing force commit until %s", slot.Index)
+ }
+
+ return nil
+}
func (s *Server) ReadCommitment(_ context.Context, req *inx.CommitmentRequest) (*inx.Commitment, error) {
commitmentSlot := iotago.SlotIndex(req.GetCommitmentSlot())
diff --git a/go.mod b/go.mod
index ae91ca026..d0ca7efaa 100644
--- a/go.mod
+++ b/go.mod
@@ -23,8 +23,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777
github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
diff --git a/go.sum b/go.sum
index ea83849f2..ce318f8bd 100644
--- a/go.sum
+++ b/go.sum
@@ -307,8 +307,12 @@ github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MD
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0 h1:/8pbFXhTSroJvjJMfJqfHjzoT9N8B4LUY3SbKruD5MM=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231023191159-38919c4705e0/go.mod h1:My1SB4vZj42EgTDNJ/dgW8lUpLNmvtzu8f89J5y2kP0=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d h1:3DzcNXfH24h8As7yMoq9m4qL2nxZyXAYIosYv/ShZbs=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d/go.mod h1:XEWewbBaqL+APJJlckhQBsg6GVc4gRmNQ+ARQtSRrvc=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd h1:hh5mAnnaZHOYAi4CIqR9K/mv786ex9AQgpisbJ4ZMow=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231023190837-6e7b2cdfd4fd/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777 h1:e2C37LxDRxnoDVMK8rhktxSB/dnL0AcXFLNwkArdaoo=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba h1:/OYVTnZTxOX87CrdZ86HwRjiVrxhkdy36/ePSq/EN24=
github.com/iotaledger/iota.go/v4 v4.0.0-20231025085218-4121986877ba/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
From 17ed5141908e10b6f223903069b241deeb697ad1 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 26 Oct 2023 13:53:51 +0200
Subject: [PATCH 12/56] Fix some bugs
---
components/inx/server_blocks.go | 4 ++--
components/inx/server_commitments.go | 6 +++---
.../sybilprotectionv1/performance/performance.go | 11 +++++++----
tools/docker-network/docker-compose.yml | 4 ++--
tools/docker-network/run.sh | 2 +-
5 files changed, 15 insertions(+), 12 deletions(-)
diff --git a/components/inx/server_blocks.go b/components/inx/server_blocks.go
index b73cd1941..19cdfce31 100644
--- a/components/inx/server_blocks.go
+++ b/components/inx/server_blocks.go
@@ -16,10 +16,10 @@ import (
iotago "github.com/iotaledger/iota.go/v4"
)
-func (s *Server) ActiveRootBlocks(_ context.Context, _ *inx.NoParams) *inx.RootBlocksResponse {
+func (s *Server) ReadActiveRootBlocks(_ context.Context, _ *inx.NoParams) (*inx.RootBlocksResponse, error) {
activeRootBlocks := deps.Protocol.MainEngineInstance().EvictionState.ActiveRootBlocks()
- return inx.WrapRootBlocks(activeRootBlocks)
+ return inx.WrapRootBlocks(activeRootBlocks), nil
}
func (s *Server) ReadBlock(_ context.Context, blockID *inx.BlockId) (*inx.RawBlock, error) {
diff --git a/components/inx/server_commitments.go b/components/inx/server_commitments.go
index 94926794f..b16713f48 100644
--- a/components/inx/server_commitments.go
+++ b/components/inx/server_commitments.go
@@ -22,13 +22,13 @@ func inxCommitment(commitment *model.Commitment) *inx.Commitment {
}
}
-func (s *Server) ForceCommitUntil(_ context.Context, slot *inx.SlotIndex) error {
+func (s *Server) ForceCommitUntil(_ context.Context, slot *inx.SlotIndex) (*inx.NoParams, error) {
err := deps.Protocol.MainEngineInstance().Notarization.ForceCommitUntil(slot.Unwrap())
if err != nil {
- return ierrors.Wrapf(err, "error while performing force commit until %s", slot.Index)
+ return nil, ierrors.Wrapf(err, "error while performing force commit until %d", slot.Index)
}
- return nil
+ return &inx.NoParams{}, nil
}
func (s *Server) ReadCommitment(_ context.Context, req *inx.CommitmentRequest) (*inx.Commitment, error) {
commitmentSlot := iotago.SlotIndex(req.GetCommitmentSlot())
diff --git a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
index a09cdadd9..d679fedd4 100644
--- a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
+++ b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/performance.go
@@ -141,17 +141,20 @@ func (t *Tracker) ValidatorCandidates(epoch iotago.EpochIndex) (ds.Set[iotago.Ac
}
func (t *Tracker) getValidatorCandidates(epoch iotago.EpochIndex) (ds.Set[iotago.AccountID], error) {
+ candidates := ds.NewSet[iotago.AccountID]()
+
+ // Epoch 0 has no candidates as it's the genesis committee.
+ if epoch == 0 {
+ return candidates, nil
+ }
+
// we store candidates in the store for the epoch of their activity, but the passed argument points to the target epoch,
// so it's necessary to subtract one epoch from the passed value
- // TODO: handle query for epoch 0
-
candidateStore, err := t.committeeCandidatesInEpochFunc(epoch - 1)
if err != nil {
return nil, ierrors.Wrapf(err, "error while retrieving candidates for epoch %d", epoch)
}
- candidates := ds.NewSet[iotago.AccountID]()
-
var innerErr error
err = candidateStore.IterateKeys(kvstore.EmptyPrefix, func(key kvstore.Key) bool {
accountID, _, err := iotago.AccountIDFromBytes(key)
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 68e85c0b2..5b2f3ce5c 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -247,7 +247,7 @@ services:
- "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
command:
- "--logger.level=debug"
- - "--inx.address=node-1:9029"
+ - "--inx.address=node-1-validator:9029"
- "--validator.ignoreBootstrapped=true"
- "--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6"
networks:
@@ -284,7 +284,7 @@ services:
- "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
command:
- "--logger.level=debug"
- - "--inx.address=node-3:9029"
+ - "--inx.address=node-3-validator:9029"
- "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
networks:
- iota-core
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index d13453261..40744cb18 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -36,7 +36,7 @@ fi
echo $DOCKER_BUILD_CONTEXT $DOCKERFILE_PATH
docker compose -f $DOCKER_COMPOSE_FILE build --build-arg WITH_GO_WORK=${WITH_GO_WORK:-0} --build-arg DOCKER_BUILD_CONTEXT=${DOCKER_BUILD_CONTEXT} --build-arg DOCKERFILE_PATH=${DOCKERFILE_PATH}
-docker compose pull inx-indexer # inx-blockissuer inx-faucet
+docker compose pull inx-indexer inx-blockissuer inx-faucet
# check exit code of builder
if [ $? -ne 0 ]; then
From 0a2142c7233ee3b27fb398032de23d89ccfcc6f9 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 27 Oct 2023 09:42:51 +0200
Subject: [PATCH 13/56] Update inx and inx-app
---
go.mod | 4 ++--
go.sum | 8 ++++----
tools/gendoc/go.mod | 4 ++--
tools/gendoc/go.sum | 8 ++++----
4 files changed, 12 insertions(+), 12 deletions(-)
diff --git a/go.mod b/go.mod
index 69e6bdd7f..263347d68 100644
--- a/go.mod
+++ b/go.mod
@@ -23,8 +23,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
diff --git a/go.sum b/go.sum
index 923d237fd..e1ccbd64b 100644
--- a/go.sum
+++ b/go.sum
@@ -305,10 +305,10 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d h1:3DzcNXfH24h8As7yMoq9m4qL2nxZyXAYIosYv/ShZbs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d/go.mod h1:XEWewbBaqL+APJJlckhQBsg6GVc4gRmNQ+ARQtSRrvc=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777 h1:e2C37LxDRxnoDVMK8rhktxSB/dnL0AcXFLNwkArdaoo=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4 h1:0H27ANY7q1PM2FZpu2ns720dAx/KlHqQ3qMfZOmwRdU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4/go.mod h1:yx3gYzpPYH3qQgXzpyigx+DwQcmZWGaL8XaorJQWX8o=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472 h1:/yFKxodL9oi5YBpmGBXlxgs9VZDlfz6tQ7k0Gce7l7Y=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472/go.mod h1:iXG/tO+GQZQzgIUyITnQDigb6Ny1wSueHFIYne4HBkc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d h1:gcJz0J3xFELIPT7y4xqW+q25oOcK6QMlxNnrfFu8srA=
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 448246bea..a6d66cce7 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -71,8 +71,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777 // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4 // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472 // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 1b4de1068..e1a105cd1 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -309,10 +309,10 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b h1:MDZhTZTVDiydXcW5j4TA7HixVCyAdToIMPhHfJee7cE=
github.com/iotaledger/hive.go/stringify v0.0.0-20231020115340-13da292c580b/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d h1:3DzcNXfH24h8As7yMoq9m4qL2nxZyXAYIosYv/ShZbs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231026074243-aa6acabc771d/go.mod h1:XEWewbBaqL+APJJlckhQBsg6GVc4gRmNQ+ARQtSRrvc=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777 h1:e2C37LxDRxnoDVMK8rhktxSB/dnL0AcXFLNwkArdaoo=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231026074031-5d93e02f7777/go.mod h1:MK0SHfNicBmcaZb3qS3tA8NEJIWKNbcNtNNKuSDKqXY=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4 h1:0H27ANY7q1PM2FZpu2ns720dAx/KlHqQ3qMfZOmwRdU=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231027074131-1b65b7ff82a4/go.mod h1:yx3gYzpPYH3qQgXzpyigx+DwQcmZWGaL8XaorJQWX8o=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472 h1:/yFKxodL9oi5YBpmGBXlxgs9VZDlfz6tQ7k0Gce7l7Y=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231027074002-08b776089472/go.mod h1:iXG/tO+GQZQzgIUyITnQDigb6Ny1wSueHFIYne4HBkc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d h1:gcJz0J3xFELIPT7y4xqW+q25oOcK6QMlxNnrfFu8srA=
github.com/iotaledger/iota.go/v4 v4.0.0-20231026154111-efd63ff4f03d/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
From 320a9a9229f827ceffd65c0e050577faeaf4b72a Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Fri, 27 Oct 2023 10:17:20 +0200
Subject: [PATCH 14/56] Trigger OnlineCommitteeSeatRemoved only when the seat
was actually removed.
---
.../activitytracker/activitytrackerv1/activitytracker.go | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
diff --git a/pkg/protocol/sybilprotection/activitytracker/activitytrackerv1/activitytracker.go b/pkg/protocol/sybilprotection/activitytracker/activitytrackerv1/activitytracker.go
index e167e2ff5..9a58b6680 100644
--- a/pkg/protocol/sybilprotection/activitytracker/activitytrackerv1/activitytracker.go
+++ b/pkg/protocol/sybilprotection/activitytracker/activitytrackerv1/activitytracker.go
@@ -77,7 +77,9 @@ func (a *ActivityTracker) MarkSeatActive(seat account.SeatIndex, id iotago.Accou
func (a *ActivityTracker) markSeatInactive(seat account.SeatIndex) {
a.lastActivities.Delete(seat)
- a.onlineCommittee.Delete(seat)
- a.Events.OnlineCommitteeSeatRemoved.Trigger(seat)
+ // Only trigger the event if online committee member is removed.
+ if a.onlineCommittee.Delete(seat) {
+ a.Events.OnlineCommitteeSeatRemoved.Trigger(seat)
+ }
}
From bfe211fb54aa2d9cc83079d5dfbe5278fcbf90ea Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 12:24:47 +0100
Subject: [PATCH 15/56] Update go.mod versions
---
go.mod | 4 ++--
go.sum | 4 ++++
2 files changed, 6 insertions(+), 2 deletions(-)
diff --git a/go.mod b/go.mod
index ec5d2d7d5..54970ad19 100644
--- a/go.mod
+++ b/go.mod
@@ -23,8 +23,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
diff --git a/go.sum b/go.sum
index 4bf13faa4..9d4df161f 100644
--- a/go.sum
+++ b/go.sum
@@ -307,8 +307,12 @@ github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:Ol
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7IzdxrogQWGF1HfUUfR3KW8WhfxE4hUbfwZCDXFY=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 h1:/X+ooE7JtCPO8dU3Z7NCQi6bRdbQsmJ5CSMIFphb7Io=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1/go.mod h1:N/COrSaVwR8pSe/3JaTjofplvcpQSE5aItHPgv38UZw=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 h1:Ziqm6j7+uXxw3suhinqmXxEocXL04tAf3rcGSnPJiaU=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
From 15bdd0d37da8af41fef13801dc91a1c586befcec Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 12:25:39 +0100
Subject: [PATCH 16/56] Update go.mod versions
---
go.sum | 4 ----
tools/gendoc/go.mod | 4 ++--
tools/gendoc/go.sum | 8 ++++----
3 files changed, 6 insertions(+), 10 deletions(-)
diff --git a/go.sum b/go.sum
index 9d4df161f..af6743539 100644
--- a/go.sum
+++ b/go.sum
@@ -305,12 +305,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7IzdxrogQWGF1HfUUfR3KW8WhfxE4hUbfwZCDXFY=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 h1:/X+ooE7JtCPO8dU3Z7NCQi6bRdbQsmJ5CSMIFphb7Io=
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1/go.mod h1:N/COrSaVwR8pSe/3JaTjofplvcpQSE5aItHPgv38UZw=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 h1:Ziqm6j7+uXxw3suhinqmXxEocXL04tAf3rcGSnPJiaU=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 0fa7e6386..d536b2883 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -71,8 +71,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 955217c98..eff404424 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -309,10 +309,10 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7IzdxrogQWGF1HfUUfR3KW8WhfxE4hUbfwZCDXFY=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 h1:/X+ooE7JtCPO8dU3Z7NCQi6bRdbQsmJ5CSMIFphb7Io=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1/go.mod h1:N/COrSaVwR8pSe/3JaTjofplvcpQSE5aItHPgv38UZw=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 h1:Ziqm6j7+uXxw3suhinqmXxEocXL04tAf3rcGSnPJiaU=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
From d5481eb422b534db6ce5844af8cbf9a6805a94f5 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 12:52:58 +0100
Subject: [PATCH 17/56] Apply review suggestions.
---
components/inx/server_accounts.go | 2 +-
tools/docker-network/docker-compose.yml | 40 +++++++++++++------------
2 files changed, 22 insertions(+), 20 deletions(-)
diff --git a/components/inx/server_accounts.go b/components/inx/server_accounts.go
index cb3fe68ad..6e8585be6 100644
--- a/components/inx/server_accounts.go
+++ b/components/inx/server_accounts.go
@@ -8,7 +8,7 @@ import (
iotago "github.com/iotaledger/iota.go/v4"
)
-func (s *Server) ReadIsAccountValidator(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
+func (s *Server) ReadIsValidatorAccount(_ context.Context, accountInfoRequest *inx.AccountInfoRequest) (*inx.BoolResponse, error) {
slot := iotago.SlotIndex(accountInfoRequest.GetAccountSlot())
accountID, _, err := iotago.AccountIDFromBytes(accountInfoRequest.AccountId)
if err != nil {
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 941303cd1..98bf63fde 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -244,15 +244,16 @@ services:
condition: service_started
inx-indexer:
condition: service_started
- environment:
- - "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
- command:
- - "--logger.level=debug"
- - "--inx.address=node-1-validator:9029"
- - "--validator.ignoreBootstrapped=true"
- - "--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6"
networks:
- iota-core
+ environment:
+ - "VALIDATOR_PRV_KEY=443a988ea61797651217de1f4662d4d6da11fd78e67f94511453bf6576045a05293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"
+ command: >
+ --logger.level=debug
+ --inx.address=node-1-validator:9029
+ --validator.ignoreBootstrapped=true
+ --validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6
+
inx-validator-2:
image: iotaledger/inx-validator:latest
@@ -263,14 +264,14 @@ services:
condition: service_started
inx-indexer:
condition: service_started
- environment:
- - "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
- command:
- - "--logger.level=debug"
- - "--inx.address=node-2-validator:9029"
- - "--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl"
networks:
- iota-core
+ environment:
+ - "VALIDATOR_PRV_KEY=3a5d39f8b60367a17fd54dac2a32c172c8e1fd6cf74ce65f1e13edba565f281705c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"
+ command: >
+ --logger.level=debug
+ --inx.address=node-2-validator:9029
+ --validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl
inx-validator-3:
image: iotaledger/inx-validator:latest
@@ -281,14 +282,15 @@ services:
condition: service_started
inx-indexer:
condition: service_started
- environment:
- - "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
- command:
- - "--logger.level=debug"
- - "--inx.address=node-3-validator:9029"
- - "--validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt"
networks:
- iota-core
+ environment:
+ - "VALIDATOR_PRV_KEY=db39d2fde6301d313b108dc9db1ee724d0f405f6fde966bd776365bc5f4a5fb31e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"
+ command: >
+ --logger.level=debug
+ --inx.address=node-3-validator:9029
+ --validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt
+
# Create our own network
networks:
From 5851ebd9b1fe272be6a0f86e92f0841f457bbfd5 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 15:58:43 +0100
Subject: [PATCH 18/56] Add inx-validator deployment to Ansible.
---
deploy/ansible/hosts/feature.yml | 10 +-
.../templates/docker-compose-iota-core.yml.j2 | 32 +++++--
tools/docker-network/docker-compose.yml | 11 ---
tools/docker-network/run.sh | 2 +-
tools/genesis-snapshot/presets/presets.go | 94 ++++++++++++++-----
5 files changed, 101 insertions(+), 48 deletions(-)
diff --git a/deploy/ansible/hosts/feature.yml b/deploy/ansible/hosts/feature.yml
index ac8e44cfc..6f0f8446d 100644
--- a/deploy/ansible/hosts/feature.yml
+++ b/deploy/ansible/hosts/feature.yml
@@ -7,20 +7,24 @@ cores:
internal_nodes:
hosts:
node-01.feature.shimmer.iota.cafe:
- validatorAccount: "{{ NODE_01_ACCOUNTID }}"
+ # validatorAccountAddress: "{{ NODE_01_ACCOUNTID }}"
+ validatorAccountAddress:"rms1pqlhggrg2ml9p0q5c4593r2yd3jwgxn20d65ulyw6z9r7xmm78apq2067mf"
validatorPrvKey: "{{ NODE_01_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_01_P2PIDENTITYPRIVATEKEY }}"
node-02.feature.shimmer.iota.cafe:
- validatorAccount: "{{ NODE_02_ACCOUNTID }}"
+ # validatorAccountAddress: "{{ NODE_02_ACCOUNTID }}"
+ validatorAccountAddress:"rms1pzjwamvhjuqtw3dkfwmmj2fgcetcdyt4uxrnjxel4caxfstzz903y7hhr3d"
validatorPrvKey: "{{ NODE_02_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_02_P2PIDENTITYPRIVATEKEY }}"
node-03.feature.shimmer.iota.cafe:
- validatorAccount: "{{ NODE_03_ACCOUNTID }}"
+ # validatorAccountAddress: "{{ NODE_03_ACCOUNTID }}"
+ validatorAccountAddress:"rms1pz6kedkxyw9md2cmp0wcdhvsxrn2e7gzuyly76ffymy4dhvtkm58qqazeuk"
validatorPrvKey: "{{ NODE_03_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_03_P2PIDENTITYPRIVATEKEY }}"
node-04.feature.shimmer.iota.cafe:
p2pIdentityPrvKey: "{{ NODE_04_P2PIDENTITYPRIVATEKEY }}"
blockissuerPrvKey: "{{ NODE_04_BLOCKISSUER_PRV_KEY }}"
+ blockissuerAccountAddress: "rms1pqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkddrrfc"
faucetPrvKey: "{{ NODE_04_FAUCET_PRV_KEY }}"
node-05.feature.shimmer.iota.cafe:
p2pIdentityPrvKey: "{{ NODE_05_P2PIDENTITYPRIVATEKEY }}"
diff --git a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2 b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
index d3bc9d82b..c9879df51 100644
--- a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
+++ b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
@@ -49,14 +49,6 @@ services:
--restAPI.bindAddress=0.0.0.0:14265
--database.path=/app/data/database
--protocol.snapshot.path=/app/data/snapshot.bin
- {% if 'node-01' in inventory_hostname or 'node-02' in inventory_hostname or 'node-03' in inventory_hostname %}
- --validator.enabled=true
- {% if 'node-01' in inventory_hostname %}
- --validator.ignoreBootstrapped=true
- {% endif %}
- --validator.account={{validatorAccount}}
- --validator.privateKey={{validatorPrvKey}}
- {% endif %}
--dashboard.bindAddress=0.0.0.0:8081
--metrics.bindAddress=iota-core:9311
--inx.enabled=true
@@ -101,7 +93,7 @@ services:
command: >
--inx.address=iota-core:9029
--restAPI.bindAddress=inx-blockissuer:9086
- --blockIssuer.accountAddress=rms1pqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkddrrfc
+ --blockIssuer.accountAddress={{blockissuerAccountAddress}}
--blockIssuer.proofOfWork.targetTrailingZeros=5
inx-faucet:
@@ -125,4 +117,26 @@ services:
command: >
--inx.address=iota-core:9029
--faucet.bindAddress=0.0.0.0:8091
+{% endif %}
+
+{% if 'node-01' in inventory_hostname or 'node-02' in inventory_hostname or 'node-03' in inventory_hostname %}
+ inx-validator:
+ container_name: inx-validator
+ image: iotaledger/inx-validator:1.0-alpha
+ stop_grace_period: 1m
+ restart: unless-stopped
+ depends_on:
+ iota-core:
+ condition: service_healthy
+ networks:
+ - iota-core
+ environment:
+ - "VALIDATOR_PRV_KEY={{validatorPrvKey}}"
+ command: >
+ --logger.level=debug
+ --inx.address=iota-core:9029
+ {% if 'node-01' in inventory_hostname %}
+ --validator.ignoreBootstrapped=true
+ {% endif %}
+ --validator.accountAddress={{validatorAccountAddress}}
{% endif %}
\ No newline at end of file
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 98bf63fde..7874bd3db 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -242,8 +242,6 @@ services:
depends_on:
node-1-validator:
condition: service_started
- inx-indexer:
- condition: service_started
networks:
- iota-core
environment:
@@ -252,8 +250,6 @@ services:
--logger.level=debug
--inx.address=node-1-validator:9029
--validator.ignoreBootstrapped=true
- --validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6
-
inx-validator-2:
image: iotaledger/inx-validator:latest
@@ -262,8 +258,6 @@ services:
depends_on:
node-2-validator:
condition: service_started
- inx-indexer:
- condition: service_started
networks:
- iota-core
environment:
@@ -271,7 +265,6 @@ services:
command: >
--logger.level=debug
--inx.address=node-2-validator:9029
- --validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl
inx-validator-3:
image: iotaledger/inx-validator:latest
@@ -280,8 +273,6 @@ services:
depends_on:
node-3-validator:
condition: service_started
- inx-indexer:
- condition: service_started
networks:
- iota-core
environment:
@@ -289,8 +280,6 @@ services:
command: >
--logger.level=debug
--inx.address=node-3-validator:9029
- --validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt
-
# Create our own network
networks:
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 40744cb18..7ac82d24a 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -36,7 +36,7 @@ fi
echo $DOCKER_BUILD_CONTEXT $DOCKERFILE_PATH
docker compose -f $DOCKER_COMPOSE_FILE build --build-arg WITH_GO_WORK=${WITH_GO_WORK:-0} --build-arg DOCKER_BUILD_CONTEXT=${DOCKER_BUILD_CONTEXT} --build-arg DOCKERFILE_PATH=${DOCKERFILE_PATH}
-docker compose pull inx-indexer inx-blockissuer inx-faucet
+docker compose pull inx-indexer inx-blockissuer inx-faucet inx-validator-1
# check exit code of builder
if [ $? -ne 0 ]; then
diff --git a/tools/genesis-snapshot/presets/presets.go b/tools/genesis-snapshot/presets/presets.go
index 8157c81e2..29babde9f 100644
--- a/tools/genesis-snapshot/presets/presets.go
+++ b/tools/genesis-snapshot/presets/presets.go
@@ -37,7 +37,15 @@ var Base = []options.Option[snapshotcreator.Options]{
var Docker = []options.Option[snapshotcreator.Options]{
snapshotcreator.WithFilePath("docker-network.snapshot"),
snapshotcreator.WithAccounts(
- snapshotcreator.AccountDetails{ // node-1-validator
+ snapshotcreator.AccountDetails{
+ /*
+ node-01-validator
+
+ Ed25519 Public Key: 293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b
+ Ed25519 Address: rms1qzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3ytgk0ny
+ Account Address: rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6
+ Restricted Address: rms1xqqfqlqzayczurc9w8cslzz4jnjkmrz5lurs32m68x7pkaxnj6unkyspqg8mulpm, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x293dc170d9a59474e6d81cfba7f7d924c09b25d7166bcfba606e53114d0a758b"))),
Amount: mock.MinValidatorAccountAmount,
@@ -49,7 +57,15 @@ var Docker = []options.Option[snapshotcreator.Options]{
StakedAmount: mock.MinValidatorAccountAmount,
Mana: iotago.Mana(mock.MinValidatorAccountAmount),
},
- snapshotcreator.AccountDetails{ // node-2-validator
+ snapshotcreator.AccountDetails{
+ /*
+ node-02-validator
+
+ Ed25519 Public Key: 05c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064
+ Ed25519 Address: rms1qqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875flpz2p
+ Account Address: rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl
+ Restricted Address: rms1xqqrw56clykvj36sv62e3v9254dxlaenzzuswy8plt2jfs8ezxlql6spqgkulf7u, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x05c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x05c1de274451db8de8182d64c6ee0dca3ae0c9077e0b4330c976976171d79064"))),
Amount: mock.MinValidatorAccountAmount,
@@ -61,7 +77,15 @@ var Docker = []options.Option[snapshotcreator.Options]{
StakedAmount: mock.MinValidatorAccountAmount,
Mana: iotago.Mana(mock.MinValidatorAccountAmount),
},
- snapshotcreator.AccountDetails{ // node-3-validator
+ snapshotcreator.AccountDetails{
+ /*
+ node-03-validator
+
+ Ed25519 Public Key: 1e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648
+ Ed25519 Address: rms1qp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kuvz0a44
+ Account Address: rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt
+ Restricted Address: rms1xqqx4mnsfuj4tr525a3slmgpy8d9xp6p3z4ugganckqslq97fymwkmspqgnzrkjq, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x1e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x1e4b21eb51dcddf65c20db1065e1f1514658b23a3ddbf48d30c0efc926a9a648"))),
Amount: mock.MinValidatorAccountAmount,
@@ -77,10 +101,11 @@ var Docker = []options.Option[snapshotcreator.Options]{
/*
inx-blockissuer
- ed25519 private key: 432c624ca3260f910df35008d5c740593b222f1e196e6cdb8cd1ad080f0d4e33997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270
- ed25519 public key: 997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270
- ed25519 address: edc1c3a42a60a04b69c2fbb6e886414c71c464afbc7d19fb63b36a8065334ada
- bech32 address: rms1prkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d5c6gkun
+ Ed25519 Private Key: 432c624ca3260f910df35008d5c740593b222f1e196e6cdb8cd1ad080f0d4e33997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270
+ Ed25519 Public Key: 997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270
+ Ed25519 Address: rms1qrkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d583cnpd
+ Account Address: rms1prkursay9fs2qjmfctamd6yxg9x8r3ry47786x0mvwek4qr9xd9d5c6gkun
+ Restricted Address: rms1xqqwmswr5s4xpgztd8p0hdhgseq5cuwyvjhmclgeld3mx65qv5e54kspqgda0nrn, Capabilities: mana
*/
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x997be92a22b1933f36e26fba5f721756f95811d6b4ae21564197c2bfa4f28270"))),
@@ -95,12 +120,10 @@ var Docker = []options.Option[snapshotcreator.Options]{
/*
inx-faucet
- ed25519 private key: de52b9964dda96564e9fab362ab16c2669c715c6a2a853bece8a25fc58c599755b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737
- ed25519 public key: 5b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737
- ed25519 address: 2f64f9d179991f50542b01e034fa043b195403875b8677efaf196b41c88803d0
- bech32 address: rms1qqhkf7w30xv375z59vq7qd86qsa3j4qrsadcval04uvkkswg3qpaqf4hga2
-
- => restricted address with mana enabled: rms1xqqz7e8e69uej86s2s4srcp5lgzrkx25qwr4hpnha7h3j66pezyq85qpqg55v3ur
+ Ed25519 Private Key: de52b9964dda96564e9fab362ab16c2669c715c6a2a853bece8a25fc58c599755b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737
+ Ed25519 Public Key: 5b938327ea463e0c323c0fd44f6fc1843ed94daecc6909c6043d06b7152e4737
+ Ed25519 Address: rms1qqhkf7w30xv375z59vq7qd86qsa3j4qrsadcval04uvkkswg3qpaqf4hga2
+ Restricted Address: rms1xqqz7e8e69uej86s2s4srcp5lgzrkx25qwr4hpnha7h3j66pezyq85qpqg55v3ur, Capabilities: mana
*/
snapshotcreator.BasicOutputDetails{
Address: lo.Return2(iotago.ParseBech32("rms1xqqz7e8e69uej86s2s4srcp5lgzrkx25qwr4hpnha7h3j66pezyq85qpqg55v3ur")),
@@ -125,7 +148,15 @@ var Docker = []options.Option[snapshotcreator.Options]{
var Feature = []options.Option[snapshotcreator.Options]{
snapshotcreator.WithFilePath("docker-network.snapshot"),
snapshotcreator.WithAccounts(
- snapshotcreator.AccountDetails{ // node-01
+ snapshotcreator.AccountDetails{
+ /*
+ node-01-validator
+
+ Ed25519 Public Key: 01fb6b9db5d96240aef00bc950d1c67a6494513f6d7cf784e57b4972b96ab2fe
+ Ed25519 Address: rms1qqlhggrg2ml9p0q5c4593r2yd3jwgxn20d65ulyw6z9r7xmm78apq4y2mxh
+ Account Address: rms1pqlhggrg2ml9p0q5c4593r2yd3jwgxn20d65ulyw6z9r7xmm78apq2067mf
+ Restricted Address: rms1xqqr7apqdpt0u59uznzkskydg3kxfeq6dfah2nnu3mgg50cm00cl5yqpqgrpy62q, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x01fb6b9db5d96240aef00bc950d1c67a6494513f6d7cf784e57b4972b96ab2fe"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x01fb6b9db5d96240aef00bc950d1c67a6494513f6d7cf784e57b4972b96ab2fe"))),
Amount: mock.MinValidatorAccountAmount,
@@ -137,7 +168,15 @@ var Feature = []options.Option[snapshotcreator.Options]{
StakedAmount: mock.MinValidatorAccountAmount,
Mana: iotago.Mana(mock.MinValidatorAccountAmount),
},
- snapshotcreator.AccountDetails{ // node-02
+ snapshotcreator.AccountDetails{
+ /*
+ node-02-validator
+
+ Ed25519 Public Key: 83e7f71a440afd48981a8b4684ddae24434b7182ce5c47cfb56ac528525fd4b6
+ Ed25519 Address: rms1qzjwamvhjuqtw3dkfwmmj2fgcetcdyt4uxrnjxel4caxfstzz903ypu8xvn
+ Account Address: rms1pzjwamvhjuqtw3dkfwmmj2fgcetcdyt4uxrnjxel4caxfstzz903y7hhr3d
+ Restricted Address: rms1xqq2fmhdj7tspd69ke9m0wff9rr90p53whscwwgm87hr5expvgg47yspqgm6whkx, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x83e7f71a440afd48981a8b4684ddae24434b7182ce5c47cfb56ac528525fd4b6"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x83e7f71a440afd48981a8b4684ddae24434b7182ce5c47cfb56ac528525fd4b6"))),
Amount: mock.MinValidatorAccountAmount,
@@ -149,7 +188,15 @@ var Feature = []options.Option[snapshotcreator.Options]{
StakedAmount: mock.MinValidatorAccountAmount,
Mana: iotago.Mana(mock.MinValidatorAccountAmount),
},
- snapshotcreator.AccountDetails{ // node-03
+ snapshotcreator.AccountDetails{
+ /*
+ node-03-validator
+
+ Ed25519 Public Key: ac628986b2ef52a1679f2289fcd7b4198476976dea4c30ae34ff04ae52e14805
+ Ed25519 Address: rms1qz6kedkxyw9md2cmp0wcdhvsxrn2e7gzuyly76ffymy4dhvtkm58qlkjupg
+ Account Address: rms1pz6kedkxyw9md2cmp0wcdhvsxrn2e7gzuyly76ffymy4dhvtkm58qqazeuk
+ Restricted Address: rms1xqqt2m9kcc3chd4trv9ampkajqcwdt8eqtsnunmf9ynvj4ka3wmwsuqpqgvp9zxl, Capabilities: mana
+ */
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0xac628986b2ef52a1679f2289fcd7b4198476976dea4c30ae34ff04ae52e14805"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0xac628986b2ef52a1679f2289fcd7b4198476976dea4c30ae34ff04ae52e14805"))),
Amount: mock.MinValidatorAccountAmount,
@@ -165,9 +212,10 @@ var Feature = []options.Option[snapshotcreator.Options]{
/*
inx-blockissuer
- ed25519 public key: 670a1a20ddb02a6cec53ec3196bc7d5bd26df2f5a6ca90b5fffd71364f104b25
- ed25519 address: 3b07e3e84c1276f0b9d35cf218b3763e0cbdadaa7ca38588de2170c31b38e9bb
- bech32 address: rms1pqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkddrrfc
+ Ed25519 Public Key: 670a1a20ddb02a6cec53ec3196bc7d5bd26df2f5a6ca90b5fffd71364f104b25
+ Ed25519 Address: rms1qqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkjxnx5x
+ Account Address: rms1pqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkddrrfc
+ Restricted Address: rms1xqqrkplrapxpyahsh8f4eusckdmrur9a4k48egu93r0zzuxrrvuwnwcpqgj0nu8t, Capabilities: mana
*/
AccountID: blake2b.Sum256(lo.PanicOnErr(hexutil.DecodeHex("0x670a1a20ddb02a6cec53ec3196bc7d5bd26df2f5a6ca90b5fffd71364f104b25"))),
Address: iotago.Ed25519AddressFromPubKey(lo.PanicOnErr(hexutil.DecodeHex("0x670a1a20ddb02a6cec53ec3196bc7d5bd26df2f5a6ca90b5fffd71364f104b25"))),
@@ -182,11 +230,9 @@ var Feature = []options.Option[snapshotcreator.Options]{
/*
inx-faucet
- ed25519 public key: dcd760a51cfafe901f4ca0745d399af7146028af643e8a339c7bb82fbb1be7f9
- ed25519 address: 48acd764f626523646d5ccf22f807e96d30b7ab0064f370b66fa811985985ec4
- bech32 address: rms1qpy2e4my7cn9ydjx6hx0ytuq06tdxzm6kqry7dctvmagzxv9np0vg9c55n4
-
- => restricted address with mana enabled: rms1xqqy3txhvnmzv53kgm2ueu30splfd5ct02cqvnehpdn04qgeskv9a3qpqgrhlhv3
+ Ed25519 Public Key: dcd760a51cfafe901f4ca0745d399af7146028af643e8a339c7bb82fbb1be7f9
+ Ed25519 Address: rms1qpy2e4my7cn9ydjx6hx0ytuq06tdxzm6kqry7dctvmagzxv9np0vg9c55n4
+ Restricted Address: rms1xqqy3txhvnmzv53kgm2ueu30splfd5ct02cqvnehpdn04qgeskv9a3qpqgrhlhv3, Capabilities: mana
*/
snapshotcreator.BasicOutputDetails{
Address: lo.Return2(iotago.ParseBech32("rms1xqqy3txhvnmzv53kgm2ueu30splfd5ct02cqvnehpdn04qgeskv9a3qpqgrhlhv3")),
From 9c446699d5fa1b890c62831bcc6dbd6fd59a9558 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 16:20:14 +0100
Subject: [PATCH 19/56] Add validator account address
---
tools/docker-network/docker-compose.yml | 3 +++
1 file changed, 3 insertions(+)
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index 7874bd3db..c0c4a2401 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -250,6 +250,7 @@ services:
--logger.level=debug
--inx.address=node-1-validator:9029
--validator.ignoreBootstrapped=true
+ --validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6
inx-validator-2:
image: iotaledger/inx-validator:latest
@@ -265,6 +266,7 @@ services:
command: >
--logger.level=debug
--inx.address=node-2-validator:9029
+ --validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl
inx-validator-3:
image: iotaledger/inx-validator:latest
@@ -280,6 +282,7 @@ services:
command: >
--logger.level=debug
--inx.address=node-3-validator:9029
+ --validator.accountAddress=rms1pp4wuuz0y42caz48vv876qfpmffswsvg40zz8v79sy8cp0jfxm4kunflcgt
# Create our own network
networks:
From 9efbf6c126d2ba04bc0515cf4340cb7cb3c79be5 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Mon, 30 Oct 2023 17:00:52 +0100
Subject: [PATCH 20/56] Update inx and inx-app
---
go.mod | 4 ++--
go.sum | 8 ++++----
tools/gendoc/go.mod | 4 ++--
tools/gendoc/go.sum | 8 ++++----
4 files changed, 12 insertions(+), 12 deletions(-)
diff --git a/go.mod b/go.mod
index 54970ad19..379f529fd 100644
--- a/go.mod
+++ b/go.mod
@@ -23,8 +23,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
diff --git a/go.sum b/go.sum
index af6743539..304469955 100644
--- a/go.sum
+++ b/go.sum
@@ -305,10 +305,10 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 h1:/X+ooE7JtCPO8dU3Z7NCQi6bRdbQsmJ5CSMIFphb7Io=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1/go.mod h1:N/COrSaVwR8pSe/3JaTjofplvcpQSE5aItHPgv38UZw=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 h1:Ziqm6j7+uXxw3suhinqmXxEocXL04tAf3rcGSnPJiaU=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2 h1:nVuTdRTrRXes843xq296LjIr+QGwD1GfJMezltsZ0DE=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2/go.mod h1:N9rYLyJbP/GIZwg5vcJe96whWswTvgeQ0hkzZSRlaRc=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed h1:LKWhsZUdrQzsnhEL4bvp/kiztWcWtxbKSz2n6fICuKM=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index d536b2883..5e28271c5 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -71,8 +71,8 @@ require (
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2 // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index eff404424..5f1412ad7 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -309,10 +309,10 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1 h1:/X+ooE7JtCPO8dU3Z7NCQi6bRdbQsmJ5CSMIFphb7Io=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030111108-3774fe9809f1/go.mod h1:N/COrSaVwR8pSe/3JaTjofplvcpQSE5aItHPgv38UZw=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968 h1:Ziqm6j7+uXxw3suhinqmXxEocXL04tAf3rcGSnPJiaU=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030111018-bf900d105968/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2 h1:nVuTdRTrRXes843xq296LjIr+QGwD1GfJMezltsZ0DE=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231030154537-9f9e608942b2/go.mod h1:N9rYLyJbP/GIZwg5vcJe96whWswTvgeQ0hkzZSRlaRc=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed h1:LKWhsZUdrQzsnhEL4bvp/kiztWcWtxbKSz2n6fICuKM=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231030154325-ff4a4c375aed/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
From a5adb00b19af9c12782b5337f74b2fa92aa122f4 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Mon, 30 Oct 2023 17:21:11 +0100
Subject: [PATCH 21/56] Rename variables for feature deploy
---
deploy/ansible/hosts/feature.yml | 12 +++--------
.../templates/docker-compose-iota-core.yml.j2 | 6 +++---
deploy/ansible/run.sh | 21 ++++++++++---------
3 files changed, 17 insertions(+), 22 deletions(-)
diff --git a/deploy/ansible/hosts/feature.yml b/deploy/ansible/hosts/feature.yml
index 6f0f8446d..7048e1507 100644
--- a/deploy/ansible/hosts/feature.yml
+++ b/deploy/ansible/hosts/feature.yml
@@ -7,25 +7,19 @@ cores:
internal_nodes:
hosts:
node-01.feature.shimmer.iota.cafe:
- # validatorAccountAddress: "{{ NODE_01_ACCOUNTID }}"
- validatorAccountAddress:"rms1pqlhggrg2ml9p0q5c4593r2yd3jwgxn20d65ulyw6z9r7xmm78apq2067mf"
+ validatorAccountAddress:"{{ NODE_01_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_01_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_01_P2PIDENTITYPRIVATEKEY }}"
node-02.feature.shimmer.iota.cafe:
- # validatorAccountAddress: "{{ NODE_02_ACCOUNTID }}"
- validatorAccountAddress:"rms1pzjwamvhjuqtw3dkfwmmj2fgcetcdyt4uxrnjxel4caxfstzz903y7hhr3d"
+ validatorAccountAddress:"{{ NODE_02_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_02_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_02_P2PIDENTITYPRIVATEKEY }}"
node-03.feature.shimmer.iota.cafe:
- # validatorAccountAddress: "{{ NODE_03_ACCOUNTID }}"
- validatorAccountAddress:"rms1pz6kedkxyw9md2cmp0wcdhvsxrn2e7gzuyly76ffymy4dhvtkm58qqazeuk"
+ validatorAccountAddress:"{{ NODE_03_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_03_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_03_P2PIDENTITYPRIVATEKEY }}"
node-04.feature.shimmer.iota.cafe:
p2pIdentityPrvKey: "{{ NODE_04_P2PIDENTITYPRIVATEKEY }}"
- blockissuerPrvKey: "{{ NODE_04_BLOCKISSUER_PRV_KEY }}"
- blockissuerAccountAddress: "rms1pqas0clgfsf8du9e6dw0yx9nwclqe0dd4f728pvgmcshpscm8r5mkddrrfc"
- faucetPrvKey: "{{ NODE_04_FAUCET_PRV_KEY }}"
node-05.feature.shimmer.iota.cafe:
p2pIdentityPrvKey: "{{ NODE_05_P2PIDENTITYPRIVATEKEY }}"
vars:
diff --git a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2 b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
index c9879df51..5fb4ee16f 100644
--- a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
+++ b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
@@ -89,11 +89,11 @@ services:
inx-indexer:
condition: service_started
environment:
- - "BLOCKISSUER_PRV_KEY={{blockissuerPrvKey}}"
+ - "BLOCKISSUER_PRV_KEY={{NODE_04_BLOCKISSUER_PRIVKEY}}"
command: >
--inx.address=iota-core:9029
--restAPI.bindAddress=inx-blockissuer:9086
- --blockIssuer.accountAddress={{blockissuerAccountAddress}}
+ --blockIssuer.accountAddress={{NODE_04_BLOCKISSUER_ACCOUNTADDRESS}}
--blockIssuer.proofOfWork.targetTrailingZeros=5
inx-faucet:
@@ -113,7 +113,7 @@ services:
ports:
- "8091:8091/tcp" # Faucet Frontend
environment:
- - "FAUCET_PRV_KEY={{faucetPrvKey}}"
+ - "FAUCET_PRV_KEY={{NODE_04_FAUCET_PRIVKEY}}"
command: >
--inx.address=iota-core:9029
--faucet.bindAddress=0.0.0.0:8091
diff --git a/deploy/ansible/run.sh b/deploy/ansible/run.sh
index 3d800611d..eb97ecfd7 100755
--- a/deploy/ansible/run.sh
+++ b/deploy/ansible/run.sh
@@ -17,22 +17,23 @@ elkElasticUser=$ELASTIC_USER
elkElasticPassword=$ELASTIC_PASSWORD
grafanaAdminPassword=$GRAFANA_ADMIN_PASSWORD
-NODE_01_ACCOUNTID=$NODE_01_ACCOUNTID
+NODE_01_VALIDATOR_ACCOUNTADDRESS=$NODE_01_VALIDATOR_ACCOUNTADDRESS
NODE_01_VALIDATOR_PRIVKEY=$NODE_01_VALIDATOR_PRIVKEY
-NODE_01_P2PIDENTITYPRIVATEKEY=$NODE_01_P2PIDENTITYPRIVATEKEY
+NODE_01_P2PIDENTITY_PRIVKEY=$NODE_01_P2PIDENTITY_PRIVKEY
-NODE_02_ACCOUNTID=$NODE_02_ACCOUNTID
+NODE_02_VALIDATOR_ACCOUNTADDRESS=$NODE_02_VALIDATOR_ACCOUNTADDRESS
NODE_02_VALIDATOR_PRIVKEY=$NODE_02_VALIDATOR_PRIVKEY
-NODE_02_P2PIDENTITYPRIVATEKEY=$NODE_02_P2PIDENTITYPRIVATEKEY
+NODE_02_P2PIDENTITY_PRIVKEY=$NODE_02_P2PIDENTITY_PRIVKEY
-NODE_03_ACCOUNTID=$NODE_03_ACCOUNTID
+NODE_03_VALIDATOR_ACCOUNTADDRESS=$NODE_03_VALIDATOR_ACCOUNTADDRESS
NODE_03_VALIDATOR_PRIVKEY=$NODE_03_VALIDATOR_PRIVKEY
-NODE_03_P2PIDENTITYPRIVATEKEY=$NODE_03_P2PIDENTITYPRIVATEKEY
+NODE_03_P2PIDENTITY_PRIVKEY=$NODE_03_P2PIDENTITY_PRIVKEY
-NODE_04_P2PIDENTITYPRIVATEKEY=$NODE_04_P2PIDENTITYPRIVATEKEY
-NODE_04_BLOCKISSUER_PRV_KEY=$NODE_04_BLOCKISSUER_PRIVKEY
-NODE_04_FAUCET_PRV_KEY=$NODE_04_FAUCET_PRIVKEY
+NODE_04_BLOCKISSUER_ACCOUNTADDRESS=$NODE_04_BLOCKISSUER_ACCOUNTADDRESS
+NODE_04_BLOCKISSUER_PRIVKEY=$NODE_04_BLOCKISSUER_PRIVKEY
+NODE_04_FAUCET_PRIVKEY=$NODE_04_FAUCET_PRIVKEY
+NODE_04_P2PIDENTITY_PRIVKEY=$NODE_04_P2PIDENTITY_PRIVKEY
-NODE_05_P2PIDENTITYPRIVATEKEY=$NODE_05_P2PIDENTITYPRIVATEKEY
+NODE_05_P2PIDENTITY_PRIVKEY=$NODE_05_P2PIDENTITY_PRIVKEY
${ARGS[@]:2} deploy/ansible/"${2:-deploy.yml}"
From 5092fceba41215cfd294004daeca088c63abb9ed Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 31 Oct 2023 08:26:40 +0100
Subject: [PATCH 22/56] Fix start condition for inx-validator.
---
.../iota-core-node/templates/docker-compose-iota-core.yml.j2 | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2 b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
index 5fb4ee16f..f8a9f58a4 100644
--- a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
+++ b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
@@ -127,7 +127,7 @@ services:
restart: unless-stopped
depends_on:
iota-core:
- condition: service_healthy
+ condition: service_started
networks:
- iota-core
environment:
From 840f30d0e7a18224c5186ac165685517091a5cc4 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 31 Oct 2023 12:17:14 +0100
Subject: [PATCH 23/56] Fix ports
---
tools/docker-network/docker-compose.yml | 46 ++++++++++++-------------
1 file changed, 23 insertions(+), 23 deletions(-)
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index c0c4a2401..a16297065 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -20,10 +20,10 @@ services:
networks:
- iota-core
ports:
- - "8010:14265/tcp" # REST-API
- - "8011:8081/tcp" # Dashboard
- - "6011:6061/tcp" # pprof
- - "9019:9029/tcp" # INX
+ - "8050:14265/tcp" # REST-API
+ - "8051:8081/tcp" # Dashboard
+ - "6051:6061/tcp" # pprof
+ - "9059:9029/tcp" # INX
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
@@ -45,10 +45,10 @@ services:
networks:
- iota-core
ports:
- - "8020:14265/tcp" # REST-API
- - "8021:8081/tcp" # Dashboard
- - "6021:6061/tcp" # pprof
- - "9029:9029/tcp" # INX
+ - "8060:14265/tcp" # REST-API
+ - "8061:8081/tcp" # Dashboard
+ - "6061:6061/tcp" # pprof
+ - "9069:9029/tcp" # INX
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
@@ -70,10 +70,10 @@ services:
networks:
- iota-core
ports:
- - "8030:14265/tcp" # REST-API
- - "8031:8081/tcp" # Dashboard
- - "6031:6061/tcp" # pprof
- - "9039:9029/tcp" # INX
+ - "8070:14265/tcp" # REST-API
+ - "8071:8081/tcp" # Dashboard
+ - "6071:6061/tcp" # pprof
+ - "9079:9029/tcp" # INX
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
@@ -95,10 +95,10 @@ services:
networks:
- iota-core
ports:
- - "8040:14265/tcp" # REST-API
- - "8041:8081/tcp" # Dashboard
- - "6041:6061/tcp" # pprof
- - "9049:9029/tcp" # INX
+ - "8080:14265/tcp" # REST-API
+ - "8081:8081/tcp" # Dashboard
+ - "6081:6061/tcp" # pprof
+ - "9089:9029/tcp" # INX
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
@@ -120,10 +120,10 @@ services:
networks:
- iota-core
ports:
- - "8050:14265/tcp" # REST-API
- - "8051:8081/tcp" # Dashboard
- - "6051:6061/tcp" # pprof
- - "9059:9029/tcp" # INX
+ - "8090:14265/tcp" # REST-API
+ - "8091:8081/tcp" # Dashboard
+ - "6091:6061/tcp" # pprof
+ - "9099:9029/tcp" # INX
volumes:
- ./docker-network.snapshot:/app/data/snapshot.bin
- ./config.json:/app/config.json:ro
@@ -236,7 +236,7 @@ services:
--faucet.rateLimit.enabled=false
inx-validator-1:
- image: iotaledger/inx-validator:latest
+ image: iotaledger/inx-validator:1.0-alpha
stop_grace_period: 1m
restart: unless-stopped
depends_on:
@@ -253,7 +253,7 @@ services:
--validator.accountAddress=rms1pzg8cqhfxqhq7pt37y8cs4v5u4kcc48lquy2k73ehsdhf5ukhya3y5rx2w6
inx-validator-2:
- image: iotaledger/inx-validator:latest
+ image: iotaledger/inx-validator:1.0-alpha
stop_grace_period: 1m
restart: unless-stopped
depends_on:
@@ -269,7 +269,7 @@ services:
--validator.accountAddress=rms1pqm4xk8e9ny5w5rxjkvtp249tfhlwvcshyr3pc0665jvp7g3hc875k538hl
inx-validator-3:
- image: iotaledger/inx-validator:latest
+ image: iotaledger/inx-validator:1.0-alpha
stop_grace_period: 1m
restart: unless-stopped
depends_on:
From 799032842a52e4f257fa026308aee39216952c8e Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Tue, 31 Oct 2023 12:39:09 +0100
Subject: [PATCH 24/56] run.sh fix
---
deploy/ansible/run.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/deploy/ansible/run.sh b/deploy/ansible/run.sh
index eb97ecfd7..537af5676 100755
--- a/deploy/ansible/run.sh
+++ b/deploy/ansible/run.sh
@@ -34,6 +34,6 @@ NODE_04_BLOCKISSUER_PRIVKEY=$NODE_04_BLOCKISSUER_PRIVKEY
NODE_04_FAUCET_PRIVKEY=$NODE_04_FAUCET_PRIVKEY
NODE_04_P2PIDENTITY_PRIVKEY=$NODE_04_P2PIDENTITY_PRIVKEY
-NODE_05_P2PIDENTITY_PRIVKEY=$NODE_05_P2PIDENTITY_PRIVKEY
+NODE_05_P2PIDENTITY_PRIVKEY=$NODE_05_P2PIDENTITY_PRIVKEY" \
${ARGS[@]:2} deploy/ansible/"${2:-deploy.yml}"
From 1e7ef7d2426d5a51243fd009c357c6738913cdaa Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Tue, 31 Oct 2023 13:02:04 +0100
Subject: [PATCH 25/56] More syntax issues
---
deploy/ansible/hosts/feature.yml | 6 +++---
deploy/ansible/run.sh | 3 ++-
2 files changed, 5 insertions(+), 4 deletions(-)
diff --git a/deploy/ansible/hosts/feature.yml b/deploy/ansible/hosts/feature.yml
index 7048e1507..0dd0ade47 100644
--- a/deploy/ansible/hosts/feature.yml
+++ b/deploy/ansible/hosts/feature.yml
@@ -7,15 +7,15 @@ cores:
internal_nodes:
hosts:
node-01.feature.shimmer.iota.cafe:
- validatorAccountAddress:"{{ NODE_01_VALIDATOR_ACCOUNTADDRESS }}"
+ validatorAccountAddress: "{{ NODE_01_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_01_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_01_P2PIDENTITYPRIVATEKEY }}"
node-02.feature.shimmer.iota.cafe:
- validatorAccountAddress:"{{ NODE_02_VALIDATOR_ACCOUNTADDRESS }}"
+ validatorAccountAddress: "{{ NODE_02_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_02_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_02_P2PIDENTITYPRIVATEKEY }}"
node-03.feature.shimmer.iota.cafe:
- validatorAccountAddress:"{{ NODE_03_VALIDATOR_ACCOUNTADDRESS }}"
+ validatorAccountAddress: "{{ NODE_03_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_03_VALIDATOR_PRIVKEY }}"
p2pIdentityPrvKey: "{{ NODE_03_P2PIDENTITYPRIVATEKEY }}"
node-04.feature.shimmer.iota.cafe:
diff --git a/deploy/ansible/run.sh b/deploy/ansible/run.sh
index 537af5676..f2bbf73b0 100755
--- a/deploy/ansible/run.sh
+++ b/deploy/ansible/run.sh
@@ -4,6 +4,8 @@ export ANSIBLE_STRATEGY=free
export ANSIBLE_PIPELINING=true
export ANSIBLE_PERSISTENT_CONTROL_PATH_DIR="/tmp/"
+set -x
+
ARGS=("$@")
ansible-playbook -u root -i deploy/ansible/hosts/"${1:-feature.yml}" \
--forks 20 --ssh-common-args "-o ControlMaster=auto -o ControlPersist=5m" \
@@ -35,5 +37,4 @@ NODE_04_FAUCET_PRIVKEY=$NODE_04_FAUCET_PRIVKEY
NODE_04_P2PIDENTITY_PRIVKEY=$NODE_04_P2PIDENTITY_PRIVKEY
NODE_05_P2PIDENTITY_PRIVKEY=$NODE_05_P2PIDENTITY_PRIVKEY" \
-
${ARGS[@]:2} deploy/ansible/"${2:-deploy.yml}"
From 7c3766a1bd20d0f25bb891a3fa659e61d97f1595 Mon Sep 17 00:00:00 2001
From: muXxer
Date: Tue, 31 Oct 2023 13:11:06 +0100
Subject: [PATCH 26/56] Add Anchor Output
---
go.mod | 2 +-
go.sum | 4 +-
.../engine/utxoledger/iteration_test.go | 8 +--
.../engine/utxoledger/manager_test.go | 10 ++--
pkg/protocol/engine/utxoledger/output_test.go | 50 +++++++++++++++++--
pkg/testsuite/mock/utils.go | 28 ++++-------
pkg/testsuite/mock/wallet_transactions.go | 8 +--
.../snapshotcreator/snapshotcreator.go | 3 +-
pkg/utils/rand.go | 38 +++++++++++---
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 4 +-
tools/genesis-snapshot/go.mod | 2 +-
tools/genesis-snapshot/go.sum | 4 +-
13 files changed, 113 insertions(+), 50 deletions(-)
diff --git a/go.mod b/go.mod
index ec5d2d7d5..a35086aea 100644
--- a/go.mod
+++ b/go.mod
@@ -25,7 +25,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d
- github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
github.com/libp2p/go-libp2p v0.31.0
diff --git a/go.sum b/go.sum
index 4bf13faa4..b52f0eaa4 100644
--- a/go.sum
+++ b/go.sum
@@ -309,8 +309,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7I
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/pkg/protocol/engine/utxoledger/iteration_test.go b/pkg/protocol/engine/utxoledger/iteration_test.go
index 41a8b7050..968ed62b4 100644
--- a/pkg/protocol/engine/utxoledger/iteration_test.go
+++ b/pkg/protocol/engine/utxoledger/iteration_test.go
@@ -21,6 +21,7 @@ func TestUTXOComputeBalance(t *testing.T) {
initialOutput := tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputBasic, utils.RandAddress(iotago.AddressEd25519), 2_134_656_365)
require.NoError(t, manager.AddGenesisUnspentOutput(initialOutput))
require.NoError(t, manager.AddGenesisUnspentOutput(tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputAccount, utils.RandAddress(iotago.AddressAccount), 56_549_524)))
+ require.NoError(t, manager.AddGenesisUnspentOutput(tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputAnchor, utils.RandAddress(iotago.AddressAccount), 56_549_524)))
require.NoError(t, manager.AddGenesisUnspentOutput(tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputFoundry, utils.RandAddress(iotago.AddressAccount), 25_548_858)))
require.NoError(t, manager.AddGenesisUnspentOutput(tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputNFT, utils.RandAddress(iotago.AddressEd25519), 545_699_656)))
require.NoError(t, manager.AddGenesisUnspentOutput(tpkg.RandLedgerStateOutputOnAddressWithAmount(iotago.OutputBasic, utils.RandAddress(iotago.AddressAccount), 626_659_696)))
@@ -43,12 +44,12 @@ func TestUTXOComputeBalance(t *testing.T) {
unspent, err := manager.UnspentOutputs()
require.NoError(t, err)
- require.Equal(t, 5, len(unspent))
+ require.Equal(t, 6, len(unspent))
balance, count, err := manager.ComputeLedgerBalance()
require.NoError(t, err)
- require.Equal(t, 5, count)
- require.Equal(t, iotago.BaseToken(2_134_656_365+56_549_524+25_548_858+545_699_656+626_659_696), balance)
+ require.Equal(t, 6, count)
+ require.Equal(t, iotago.BaseToken(2_134_656_365+56_549_524+56_549_524+25_548_858+545_699_656+626_659_696), balance)
}
func TestUTXOIteration(t *testing.T) {
@@ -67,6 +68,7 @@ func TestUTXOIteration(t *testing.T) {
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputNFT, utils.RandAddress(iotago.AddressNFT)),
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputNFT, utils.RandAddress(iotago.AddressAccount)),
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputAccount, utils.RandAddress(iotago.AddressEd25519)),
+ tpkg.RandLedgerStateOutputOnAddress(iotago.OutputAnchor, utils.RandAddress(iotago.AddressEd25519)),
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputFoundry, utils.RandAddress(iotago.AddressAccount)),
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputFoundry, utils.RandAddress(iotago.AddressAccount)),
tpkg.RandLedgerStateOutputOnAddress(iotago.OutputFoundry, utils.RandAddress(iotago.AddressAccount)),
diff --git a/pkg/protocol/engine/utxoledger/manager_test.go b/pkg/protocol/engine/utxoledger/manager_test.go
index e9f38fe39..fe14c840e 100644
--- a/pkg/protocol/engine/utxoledger/manager_test.go
+++ b/pkg/protocol/engine/utxoledger/manager_test.go
@@ -23,6 +23,7 @@ func TestConfirmationApplyAndRollbackToEmptyLedger(t *testing.T) {
tpkg.RandLedgerStateOutputWithType(iotago.OutputNFT), // spent
tpkg.RandLedgerStateOutputWithType(iotago.OutputBasic), // spent
tpkg.RandLedgerStateOutputWithType(iotago.OutputAccount),
+ tpkg.RandLedgerStateOutputWithType(iotago.OutputAnchor),
tpkg.RandLedgerStateOutputWithType(iotago.OutputNFT),
tpkg.RandLedgerStateOutputWithType(iotago.OutputFoundry),
}
@@ -45,7 +46,7 @@ func TestConfirmationApplyAndRollbackToEmptyLedger(t *testing.T) {
return true
}))
- require.Equal(t, 7, outputCount)
+ require.Equal(t, 8, outputCount)
var unspentCount int
require.NoError(t, manager.ForEachUnspentOutput(func(_ *utxoledger.Output) bool {
@@ -53,7 +54,7 @@ func TestConfirmationApplyAndRollbackToEmptyLedger(t *testing.T) {
return true
}))
- require.Equal(t, 5, unspentCount)
+ require.Equal(t, 6, unspentCount)
var spentCount int
require.NoError(t, manager.ForEachSpentOutput(func(_ *utxoledger.Spent) bool {
@@ -113,6 +114,7 @@ func TestConfirmationApplyAndRollbackToPreviousLedger(t *testing.T) {
tpkg.RandLedgerStateOutputWithType(iotago.OutputFoundry),
tpkg.RandLedgerStateOutputWithType(iotago.OutputBasic), // spent
tpkg.RandLedgerStateOutputWithType(iotago.OutputAccount),
+ tpkg.RandLedgerStateOutputWithType(iotago.OutputAnchor),
}
index := iotago.SlotIndex(49)
@@ -161,7 +163,7 @@ func TestConfirmationApplyAndRollbackToPreviousLedger(t *testing.T) {
return true
}))
require.Empty(t, outputByOutputID)
- require.Equal(t, 7, outputCount)
+ require.Equal(t, 8, outputCount)
var unspentCount int
require.NoError(t, manager.ForEachUnspentOutput(func(output *utxoledger.Output) bool {
@@ -172,7 +174,7 @@ func TestConfirmationApplyAndRollbackToPreviousLedger(t *testing.T) {
return true
}))
- require.Equal(t, 4, unspentCount)
+ require.Equal(t, 5, unspentCount)
require.Empty(t, unspentByOutputID)
var spentCount int
diff --git a/pkg/protocol/engine/utxoledger/output_test.go b/pkg/protocol/engine/utxoledger/output_test.go
index 7cfc6548e..66f882265 100644
--- a/pkg/protocol/engine/utxoledger/output_test.go
+++ b/pkg/protocol/engine/utxoledger/output_test.go
@@ -290,8 +290,7 @@ func TestAccountOutputSerialization(t *testing.T) {
txCreationSlot := utils.RandSlotIndex()
blockID := utils.RandBlockID()
aliasID := utils.RandAccountID()
- stateController := utils.RandAccountID()
- governor := utils.RandAddress(iotago.AddressEd25519).(*iotago.Ed25519Address)
+ address := utils.RandAccountID().ToAddress()
issuer := utils.RandNFTID()
sender := utils.RandAccountID()
amount := iotago_tpkg.RandBaseToken(iotago.MaxBaseToken)
@@ -301,6 +300,49 @@ func TestAccountOutputSerialization(t *testing.T) {
Amount: amount,
AccountID: aliasID,
Conditions: iotago.AccountOutputUnlockConditions{
+ &iotago.AddressUnlockCondition{
+ Address: address,
+ },
+ },
+ Features: iotago.AccountOutputFeatures{
+ &iotago.SenderFeature{
+ Address: sender.ToAddress(),
+ },
+ },
+ ImmutableFeatures: iotago.AccountOutputImmFeatures{
+ &iotago.IssuerFeature{
+ Address: issuer.ToAddress(),
+ },
+ },
+ }
+
+ outputProof, err := iotago.NewOutputIDProof(iotago_tpkg.TestAPI, txCommitment, txCreationSlot, iotago.TxEssenceOutputs{iotaOutput}, 0)
+ require.NoError(t, err)
+
+ output := CreateOutputAndAssertSerialization(t, blockID, index, iotaOutput, outputProof)
+ spent := CreateSpentAndAssertSerialization(t, output)
+ outputID := output.OutputID()
+
+ require.ElementsMatch(t, byteutils.ConcatBytes([]byte{utxoledger.StoreKeyPrefixOutputUnspent}, outputID[:]), output.UnspentLookupKey())
+ AssertOutputUnspentAndSpentTransitions(t, output, spent)
+}
+
+func TestAnchorOutputSerialization(t *testing.T) {
+ txCommitment := iotago_tpkg.Rand32ByteArray()
+ txCreationSlot := utils.RandSlotIndex()
+ blockID := utils.RandBlockID()
+ aliasID := utils.RandAnchorID()
+ stateController := utils.RandAnchorID()
+ governor := utils.RandAddress(iotago.AddressEd25519).(*iotago.Ed25519Address)
+ issuer := utils.RandNFTID()
+ sender := utils.RandAnchorID()
+ amount := iotago_tpkg.RandBaseToken(iotago.MaxBaseToken)
+ index := utils.RandSlotIndex()
+
+ iotaOutput := &iotago.AnchorOutput{
+ Amount: amount,
+ AnchorID: aliasID,
+ Conditions: iotago.AnchorOutputUnlockConditions{
&iotago.StateControllerAddressUnlockCondition{
Address: stateController.ToAddress(),
},
@@ -309,12 +351,12 @@ func TestAccountOutputSerialization(t *testing.T) {
},
},
StateMetadata: make([]byte, 0),
- Features: iotago.AccountOutputFeatures{
+ Features: iotago.AnchorOutputFeatures{
&iotago.SenderFeature{
Address: sender.ToAddress(),
},
},
- ImmutableFeatures: iotago.AccountOutputImmFeatures{
+ ImmutableFeatures: iotago.AnchorOutputImmFeatures{
&iotago.IssuerFeature{
Address: issuer.ToAddress(),
},
diff --git a/pkg/testsuite/mock/utils.go b/pkg/testsuite/mock/utils.go
index f7cc8adcd..9689781dc 100644
--- a/pkg/testsuite/mock/utils.go
+++ b/pkg/testsuite/mock/utils.go
@@ -24,8 +24,8 @@ func WithInputs(inputs utxoledger.Outputs) options.Option[builder.TransactionBui
InputID: input.OutputID(),
Input: input.Output(),
})
- case iotago.OutputAccount:
- // For alias we need to unlock the state controller
+ case iotago.OutputAnchor:
+ // For anchor outputs we need to unlock the state controller
txBuilder.AddInput(&builder.TxInput{
UnlockTarget: input.Output().UnlockConditionSet().StateControllerAddress().Address,
InputID: input.OutputID(),
@@ -42,14 +42,12 @@ func WithInputs(inputs utxoledger.Outputs) options.Option[builder.TransactionBui
}
}
-func WithAccountInput(input *utxoledger.Output, governorTransition bool) options.Option[builder.TransactionBuilder] {
+func WithAccountInput(input *utxoledger.Output) options.Option[builder.TransactionBuilder] {
return func(txBuilder *builder.TransactionBuilder) {
switch input.OutputType() {
case iotago.OutputAccount:
- address := input.Output().UnlockConditionSet().StateControllerAddress().Address
- if governorTransition {
- address = input.Output().UnlockConditionSet().GovernorAddress().Address
- }
+ address := input.Output().UnlockConditionSet().Address().Address
+
txBuilder.AddInput(&builder.TxInput{
UnlockTarget: address,
InputID: input.OutputID(),
@@ -145,22 +143,19 @@ func WithBlockIssuerFeature(keys iotago.BlockIssuerKeys, expirySlot iotago.SlotI
func WithAddBlockIssuerKey(key iotago.BlockIssuerKey) options.Option[builder.AccountOutputBuilder] {
return func(accountBuilder *builder.AccountOutputBuilder) {
- transition := accountBuilder.GovernanceTransition()
- transition.BlockIssuerTransition().AddKeys(key)
+ accountBuilder.BlockIssuerTransition().AddKeys(key)
}
}
func WithBlockIssuerKeys(keys iotago.BlockIssuerKeys) options.Option[builder.AccountOutputBuilder] {
return func(accountBuilder *builder.AccountOutputBuilder) {
- transition := accountBuilder.GovernanceTransition()
- transition.BlockIssuerTransition().Keys(keys)
+ accountBuilder.BlockIssuerTransition().Keys(keys)
}
}
func WithBlockIssuerExpirySlot(expirySlot iotago.SlotIndex) options.Option[builder.AccountOutputBuilder] {
return func(accountBuilder *builder.AccountOutputBuilder) {
- transition := accountBuilder.GovernanceTransition()
- transition.BlockIssuerTransition().ExpirySlot(expirySlot)
+ accountBuilder.BlockIssuerTransition().ExpirySlot(expirySlot)
}
}
@@ -209,12 +204,9 @@ func WithAccountConditions(conditions iotago.AccountOutputUnlockConditions) opti
return func(accountBuilder *builder.AccountOutputBuilder) {
for _, condition := range conditions.MustSet() {
switch condition.Type() {
- case iotago.UnlockConditionStateControllerAddress:
- //nolint:forcetypeassert
- accountBuilder.StateController(condition.(*iotago.StateControllerAddressUnlockCondition).Address)
- case iotago.UnlockConditionGovernorAddress:
+ case iotago.UnlockConditionAddress:
//nolint:forcetypeassert
- accountBuilder.Governor(condition.(*iotago.GovernorAddressUnlockCondition).Address)
+ accountBuilder.Address(condition.(*iotago.AddressUnlockCondition).Address)
}
}
}
diff --git a/pkg/testsuite/mock/wallet_transactions.go b/pkg/testsuite/mock/wallet_transactions.go
index 6b3481046..3d275b7ce 100644
--- a/pkg/testsuite/mock/wallet_transactions.go
+++ b/pkg/testsuite/mock/wallet_transactions.go
@@ -18,7 +18,7 @@ import (
func (w *Wallet) CreateAccountFromInput(transactionName string, inputName string, recipientWallet *Wallet, creationSlot iotago.SlotIndex, opts ...options.Option[builder.AccountOutputBuilder]) *iotago.SignedTransaction {
input := w.Output(inputName)
- accountOutput := options.Apply(builder.NewAccountOutputBuilder(recipientWallet.Address(), recipientWallet.Address(), input.BaseTokenAmount()).
+ accountOutput := options.Apply(builder.NewAccountOutputBuilder(recipientWallet.Address(), input.BaseTokenAmount()).
Mana(input.StoredMana()),
opts).MustBuild()
@@ -151,7 +151,7 @@ func (w *Wallet) TransitionAccount(transactionName string, inputName string, opt
signedTransaction := lo.PanicOnErr(w.createSignedTransactionWithOptions(
transactionName,
- WithAccountInput(input, true),
+ WithAccountInput(input),
WithContextInputs(iotago.TxEssenceContextInputs{
&iotago.BlockIssuanceCreditInput{
AccountID: accountOutput.AccountID,
@@ -192,7 +192,7 @@ func (w *Wallet) DestroyAccount(transactionName string, inputName string, creati
CommitmentID: w.Node.Protocol.MainEngineInstance().Storage.Settings().LatestCommitment().Commitment().MustID(),
},
}),
- WithAccountInput(input, true),
+ WithAccountInput(input),
WithOutputs(destructionOutputs),
WithSlotCreated(creationSlot),
))
@@ -250,7 +250,7 @@ func (w *Wallet) TransitionImplicitAccountToAccountOutput(transactionName string
panic(fmt.Sprintf("output with alias %s is not an implicit account", inputName))
}
- accountOutput := options.Apply(builder.NewAccountOutputBuilder(w.Address(), w.Address(), MinIssuerAccountAmount).
+ accountOutput := options.Apply(builder.NewAccountOutputBuilder(w.Address(), MinIssuerAccountAmount).
AccountID(iotago.AccountIDFromOutputID(input.OutputID())),
opts).MustBuild()
diff --git a/pkg/testsuite/snapshotcreator/snapshotcreator.go b/pkg/testsuite/snapshotcreator/snapshotcreator.go
index f0d3ee8a0..c09f9d622 100644
--- a/pkg/testsuite/snapshotcreator/snapshotcreator.go
+++ b/pkg/testsuite/snapshotcreator/snapshotcreator.go
@@ -249,8 +249,7 @@ func createAccount(accountID iotago.AccountID, address iotago.Address, tokenAmou
Mana: mana,
AccountID: accountID,
Conditions: iotago.AccountOutputUnlockConditions{
- &iotago.StateControllerAddressUnlockCondition{Address: address},
- &iotago.GovernorAddressUnlockCondition{Address: address},
+ &iotago.AddressUnlockCondition{Address: address},
},
Features: iotago.AccountOutputFeatures{
&iotago.BlockIssuerFeature{
diff --git a/pkg/utils/rand.go b/pkg/utils/rand.go
index b6ff8e63a..9e35787f2 100644
--- a/pkg/utils/rand.go
+++ b/pkg/utils/rand.go
@@ -111,10 +111,17 @@ func RandNFTID() iotago.NFTID {
}
func RandAccountID() iotago.AccountID {
- alias := iotago.AccountID{}
- copy(alias[:], RandBytes(iotago.AccountIDLength))
+ accountID := iotago.AccountID{}
+ copy(accountID[:], RandBytes(iotago.AccountIDLength))
- return alias
+ return accountID
+}
+
+func RandAnchorID() iotago.AnchorID {
+ anchorID := iotago.AnchorID{}
+ copy(anchorID[:], RandBytes(iotago.AnchorIDLength))
+
+ return anchorID
}
func RandSlotIndex() iotago.SlotIndex {
@@ -146,7 +153,7 @@ func RandAddress(addressType iotago.AddressType) iotago.Address {
}
func RandOutputType() iotago.OutputType {
- outputTypes := []iotago.OutputType{iotago.OutputBasic, iotago.OutputAccount, iotago.OutputFoundry, iotago.OutputNFT, iotago.OutputDelegation}
+ outputTypes := []iotago.OutputType{iotago.OutputBasic, iotago.OutputAccount, iotago.OutputAnchor, iotago.OutputFoundry, iotago.OutputNFT, iotago.OutputDelegation}
return outputTypes[RandomIntn(len(outputTypes)-1)]
}
@@ -181,12 +188,27 @@ func RandOutputOnAddressWithAmount(outputType iotago.OutputType, address iotago.
},
Features: iotago.BasicOutputFeatures{},
}
+
case iotago.OutputAccount:
//nolint:forcetypeassert // we already checked the type
iotaOutput = &iotago.AccountOutput{
Amount: amount,
AccountID: RandAccountID(),
Conditions: iotago.AccountOutputUnlockConditions{
+ &iotago.AddressUnlockCondition{
+ Address: address,
+ },
+ },
+ Features: iotago.AccountOutputFeatures{},
+ ImmutableFeatures: iotago.AccountOutputImmFeatures{},
+ }
+
+ case iotago.OutputAnchor:
+ //nolint:forcetypeassert // we already checked the type
+ iotaOutput = &iotago.AnchorOutput{
+ Amount: amount,
+ AnchorID: RandAnchorID(),
+ Conditions: iotago.AnchorOutputUnlockConditions{
&iotago.StateControllerAddressUnlockCondition{
Address: address,
},
@@ -195,9 +217,10 @@ func RandOutputOnAddressWithAmount(outputType iotago.OutputType, address iotago.
},
},
StateMetadata: make([]byte, 0),
- Features: iotago.AccountOutputFeatures{},
- ImmutableFeatures: iotago.AccountOutputImmFeatures{},
+ Features: iotago.AnchorOutputFeatures{},
+ ImmutableFeatures: iotago.AnchorOutputImmFeatures{},
}
+
case iotago.OutputFoundry:
if address.Type() != iotago.AddressAccount {
panic("not an alias address")
@@ -221,6 +244,7 @@ func RandOutputOnAddressWithAmount(outputType iotago.OutputType, address iotago.
Features: iotago.FoundryOutputFeatures{},
ImmutableFeatures: iotago.FoundryOutputImmFeatures{},
}
+
case iotago.OutputNFT:
//nolint:forcetypeassert // we already checked the type
iotaOutput = &iotago.NFTOutput{
@@ -234,6 +258,7 @@ func RandOutputOnAddressWithAmount(outputType iotago.OutputType, address iotago.
Features: iotago.NFTOutputFeatures{},
ImmutableFeatures: iotago.NFTOutputImmFeatures{},
}
+
case iotago.OutputDelegation:
//nolint:forcetypeassert // we already checked the type
iotaOutput = &iotago.DelegationOutput{
@@ -249,6 +274,7 @@ func RandOutputOnAddressWithAmount(outputType iotago.OutputType, address iotago.
},
},
}
+
default:
panic("unhandled output type")
}
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 0fa7e6386..3abd303ab 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -73,7 +73,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d // indirect
- github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a // indirect
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 955217c98..969b82ae7 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -313,8 +313,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7I
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index aaab649cb..a92922bf8 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -10,7 +10,7 @@ require (
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967
github.com/mr-tron/base58 v1.2.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.14.0
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index 3dbf42773..880f443ff 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -52,8 +52,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a h1:WLW4iaJAx4N9Pujv+gzHklnnjCt5MPrtXyVcK3UXdNc=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231028103644-b834fd54b02a/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
From 754b4e7e1793c8a5b97597969ab56b61206c9493 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Tue, 31 Oct 2023 13:14:11 +0100
Subject: [PATCH 27/56] More fixes
---
deploy/ansible/hosts/feature.yml | 10 +++++-----
deploy/ansible/run.sh | 2 --
2 files changed, 5 insertions(+), 7 deletions(-)
diff --git a/deploy/ansible/hosts/feature.yml b/deploy/ansible/hosts/feature.yml
index 0dd0ade47..66e7b10ee 100644
--- a/deploy/ansible/hosts/feature.yml
+++ b/deploy/ansible/hosts/feature.yml
@@ -9,17 +9,17 @@ cores:
node-01.feature.shimmer.iota.cafe:
validatorAccountAddress: "{{ NODE_01_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_01_VALIDATOR_PRIVKEY }}"
- p2pIdentityPrvKey: "{{ NODE_01_P2PIDENTITYPRIVATEKEY }}"
+ p2pIdentityPrvKey: "{{ NODE_01_P2PIDENTITY_PRIVKEY }}"
node-02.feature.shimmer.iota.cafe:
validatorAccountAddress: "{{ NODE_02_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_02_VALIDATOR_PRIVKEY }}"
- p2pIdentityPrvKey: "{{ NODE_02_P2PIDENTITYPRIVATEKEY }}"
+ p2pIdentityPrvKey: "{{ NODE_02_P2PIDENTITY_PRIVKEY }}"
node-03.feature.shimmer.iota.cafe:
validatorAccountAddress: "{{ NODE_03_VALIDATOR_ACCOUNTADDRESS }}"
validatorPrvKey: "{{ NODE_03_VALIDATOR_PRIVKEY }}"
- p2pIdentityPrvKey: "{{ NODE_03_P2PIDENTITYPRIVATEKEY }}"
+ p2pIdentityPrvKey: "{{ NODE_03_P2PIDENTITY_PRIVKEY }}"
node-04.feature.shimmer.iota.cafe:
- p2pIdentityPrvKey: "{{ NODE_04_P2PIDENTITYPRIVATEKEY }}"
+ p2pIdentityPrvKey: "{{ NODE_04_P2PIDENTITY_PRIVKEY }}"
node-05.feature.shimmer.iota.cafe:
- p2pIdentityPrvKey: "{{ NODE_05_P2PIDENTITYPRIVATEKEY }}"
+ p2pIdentityPrvKey: "{{ NODE_05_P2PIDENTITY_PRIVKEY }}"
vars:
diff --git a/deploy/ansible/run.sh b/deploy/ansible/run.sh
index f2bbf73b0..62ee4840f 100755
--- a/deploy/ansible/run.sh
+++ b/deploy/ansible/run.sh
@@ -4,8 +4,6 @@ export ANSIBLE_STRATEGY=free
export ANSIBLE_PIPELINING=true
export ANSIBLE_PERSISTENT_CONTROL_PATH_DIR="/tmp/"
-set -x
-
ARGS=("$@")
ansible-playbook -u root -i deploy/ansible/hosts/"${1:-feature.yml}" \
--forks 20 --ssh-common-args "-o ControlMaster=auto -o ControlPersist=5m" \
From e37183697c8216e6ea8ddf53390fecf620baee59 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 31 Oct 2023 15:15:30 +0100
Subject: [PATCH 28/56] Fix deployment name.
---
.../iota-core-node/templates/docker-compose-iota-core.yml.j2 | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2 b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
index f8a9f58a4..2c18d1c6c 100644
--- a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
+++ b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
@@ -14,7 +14,7 @@ services:
# IOTA-CORE Nodes #
###################
- iota_core:
+ iota-core:
image: {{iota_core_docker_image_repo}}:{{iota_core_docker_image_tag}}
container_name: iota-core
stop_grace_period: 1m
From 151137215d6014b734609e73b65531930c4a4a92 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 31 Oct 2023 15:26:04 +0100
Subject: [PATCH 29/56] Fix iota-core deployment template.
---
.../iota-core-node/templates/docker-compose-iota-core.yml.j2 | 5 -----
1 file changed, 5 deletions(-)
diff --git a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2 b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
index 2c18d1c6c..40beab818 100644
--- a/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
+++ b/deploy/ansible/roles/iota-core-node/templates/docker-compose-iota-core.yml.j2
@@ -108,8 +108,6 @@ services:
condition: service_started
inx-blockissuer:
condition: service_started
- networks:
- - iota-core
ports:
- "8091:8091/tcp" # Faucet Frontend
environment:
@@ -128,12 +126,9 @@ services:
depends_on:
iota-core:
condition: service_started
- networks:
- - iota-core
environment:
- "VALIDATOR_PRV_KEY={{validatorPrvKey}}"
command: >
- --logger.level=debug
--inx.address=iota-core:9029
{% if 'node-01' in inventory_hostname %}
--validator.ignoreBootstrapped=true
From 283e0781ca1f895cb100bb30331de1d9faddfffe Mon Sep 17 00:00:00 2001
From: muXxer
Date: Tue, 31 Oct 2023 15:28:05 +0100
Subject: [PATCH 30/56] Update dependencies
---
go.mod | 44 ++++++++---------
go.sum | 90 +++++++++++++++++------------------
tools/gendoc/go.mod | 44 ++++++++---------
tools/gendoc/go.sum | 90 +++++++++++++++++------------------
tools/genesis-snapshot/go.mod | 16 +++----
tools/genesis-snapshot/go.sum | 32 ++++++-------
6 files changed, 156 insertions(+), 160 deletions(-)
diff --git a/go.mod b/go.mod
index a35086aea..1784168e9 100644
--- a/go.mod
+++ b/go.mod
@@ -7,28 +7,28 @@ replace github.com/goccy/go-graphviz => github.com/alexsporn/go-graphviz v0.0.0-
require (
github.com/goccy/go-graphviz v0.1.1
github.com/golang-jwt/jwt v3.2.2+incompatible
- github.com/google/uuid v1.3.1
+ github.com/google/uuid v1.4.0
github.com/gorilla/websocket v1.5.0
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
- github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b
- github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42
+ github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
- github.com/libp2p/go-libp2p v0.31.0
+ github.com/libp2p/go-libp2p v0.32.0
github.com/libp2p/go-libp2p-kad-dht v0.25.1
github.com/mr-tron/base58 v1.2.0
github.com/multiformats/go-multiaddr v0.12.0
@@ -69,19 +69,18 @@ require (
github.com/flynn/noise v1.0.0 // indirect
github.com/fogleman/gg v1.3.0 // indirect
github.com/francoispqt/gojay v1.2.13 // indirect
- github.com/fsnotify/fsnotify v1.6.0 // indirect
- github.com/go-logr/logr v1.2.4 // indirect
+ github.com/fsnotify/fsnotify v1.7.0 // indirect
+ github.com/go-logr/logr v1.3.0 // indirect
github.com/go-logr/stdr v1.2.2 // indirect
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect
github.com/godbus/dbus/v5 v5.1.0 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect
- github.com/golang/mock v1.6.0 // indirect
github.com/golang/protobuf v1.5.3 // indirect
github.com/google/go-github v17.0.0+incompatible // indirect
github.com/google/go-querystring v1.1.0 // indirect
github.com/google/gopacket v1.1.19 // indirect
- github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98 // indirect
+ github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b // indirect
github.com/hashicorp/errwrap v1.1.0 // indirect
github.com/hashicorp/go-multierror v1.1.1 // indirect
github.com/hashicorp/go-version v1.6.0 // indirect
@@ -90,7 +89,7 @@ require (
github.com/huin/goupnp v1.3.0 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
@@ -100,7 +99,7 @@ require (
github.com/jackpal/go-nat-pmp v1.0.2 // indirect
github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect
github.com/jbenet/goprocess v0.1.4 // indirect
- github.com/klauspost/compress v1.17.1 // indirect
+ github.com/klauspost/compress v1.17.2 // indirect
github.com/klauspost/cpuid/v2 v2.2.5 // indirect
github.com/knadh/koanf v1.5.0 // indirect
github.com/koron/go-ssdp v0.0.4 // indirect
@@ -135,7 +134,7 @@ require (
github.com/multiformats/go-multibase v0.2.0 // indirect
github.com/multiformats/go-multicodec v0.9.0 // indirect
github.com/multiformats/go-multihash v0.2.3 // indirect
- github.com/multiformats/go-multistream v0.4.1 // indirect
+ github.com/multiformats/go-multistream v0.5.0 // indirect
github.com/onsi/ginkgo/v2 v2.13.0 // indirect
github.com/opencontainers/runtime-spec v1.1.0 // indirect
github.com/opentracing/opentracing-go v1.2.0 // indirect
@@ -151,9 +150,9 @@ require (
github.com/prometheus/common v0.45.0 // indirect
github.com/prometheus/procfs v0.12.0 // indirect
github.com/quic-go/qpack v0.4.0 // indirect
- github.com/quic-go/qtls-go1-20 v0.3.4 // indirect
- github.com/quic-go/quic-go v0.38.1 // indirect
- github.com/quic-go/webtransport-go v0.5.3 // indirect
+ github.com/quic-go/qtls-go1-20 v0.4.1 // indirect
+ github.com/quic-go/quic-go v0.40.0 // indirect
+ github.com/quic-go/webtransport-go v0.6.0 // indirect
github.com/raulk/go-watchdog v1.3.0 // indirect
github.com/sasha-s/go-deadlock v0.3.1 // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
@@ -167,6 +166,7 @@ require (
go.opentelemetry.io/otel/metric v1.19.0 // indirect
go.opentelemetry.io/otel/trace v1.19.0 // indirect
go.uber.org/fx v1.20.1 // indirect
+ go.uber.org/mock v0.3.0 // indirect
go.uber.org/multierr v1.11.0 // indirect
go.uber.org/zap v1.26.0 // indirect
golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect
@@ -179,7 +179,7 @@ require (
golang.org/x/time v0.3.0 // indirect
golang.org/x/tools v0.14.0 // indirect
gonum.org/v1/gonum v0.14.0 // indirect
- google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect
+ google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 // indirect
gopkg.in/yaml.v2 v2.4.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
lukechampine.com/blake3 v1.2.1 // indirect
diff --git a/go.sum b/go.sum
index b52f0eaa4..2d6f0464b 100644
--- a/go.sum
+++ b/go.sum
@@ -119,8 +119,8 @@ github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHk
github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
-github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY=
-github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw=
+github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA=
+github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0=
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
@@ -132,8 +132,8 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
-github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ=
-github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
+github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY=
+github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
@@ -160,8 +160,6 @@ github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4er
github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
-github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc=
-github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
@@ -204,12 +202,12 @@ github.com/google/gopacket v1.1.19/go.mod h1:iJ8V8n6KS+z2U1A8pUwu8bW5SyEMkXJB8Yo
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20211214055906-6f57359322fd/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg=
-github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98 h1:pUa4ghanp6q4IJHwE9RwLgmVFfReJN+KbQ8ExNEUUoQ=
-github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik=
+github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b h1:RMpPgZTSApbPf7xaVel+QkoGPRLFLrwFO89uDUHEGf0=
+github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4=
-github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4=
+github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY=
github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg=
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
@@ -277,38 +275,38 @@ github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJ
github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b h1:D68khiAFb9DwTvjZc2nc4R0E6wUdKyYCUXkmdaMzuoQ=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b h1:mX3NXaTMLEwZnEs4IlxEvXY0YZo8qbb8M1xM39FS6qY=
-github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42 h1:EOfxTuAiBmED1VHuVh7/UIeB27cCRe13gdSzyioNMBw=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42 h1:xAER9M9Uoz2EOWT43E9wmXRe+RmAk8OBSUoboH4Su8M=
+github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42 h1:drmpgLlJy7kZ09Dt1qKSnbILU+27Qu2jp4VdPDNwbFk=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b h1:ZERXxhQBUBV1AqTE6cUI4vTxSx4JrnsMuLZFgj32xLM=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42 h1:BC5GkIHyXdoJGdw6Tu5ds2kjw9grFLtwQiuMaKfdLU8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42 h1:r8TkdQJB7/bJd8cF8z5GQ+rX/7JpbPdPoN7wMoV1OCM=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b h1:8b2sH+2Vf0y5BDYTMwKa09iQr3JF9JrzTI64DkXb+9U=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42 h1:ytzZZPtclAzLfjxv26frbinCGx3Z6ouUENbx5U7lFGg=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42 h1:QMxd32Y/veVhTDPCiOFgetjUbG7sr9MryF29/rSPkMA=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b h1:LusmtjpfG/q8lc15Fp9W3kABbN3tArKx/zw2ibdY1DU=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42 h1:/xPwStUckZ2V0XPoY496cXU+c5elpHyvYoT6JAmuvRY=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42 h1:AvNLzONVMspwx7nD/NyYUgb5Hi7/zgzIOegr1uRD/M8=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b h1:IwhoeOeRu25mBdrimuOOvbbhHYX0QipibV69ubn8nX0=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b h1:EhVgAU/f2J3VYZwP60dRdyfAeDU3c/gBzX9blKtQGKA=
-github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 h1:e1uJAlXE3zeXpa+c4uFOG+/AMFbUlLt2mcrSK5NMxVs=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42 h1:7wjs4t1snBDJ8LOTl+tZhr2ORywSOTgJMppxiIAMA0A=
+github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 h1:1QMJ39qXIx/IZVzus3+97IV7Pa++e+d340TvbMjhiBU=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b h1:zaXZn9yV/95SRDkgCZQeBbSbmcJTKSZbCB7oBd71Qwg=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42 h1:hZli4E9kJUAEQ7gzZR1XbPcpgqvqMPYq8YBPMbrBuos=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7IzdxrogQWGF1HfUUfR3KW8WhfxE4hUbfwZCDXFY=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 h1:17JDzMKTMXKF3xys6gPURRddkZhg1LY+xwfhbr/sVqg=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5/go.mod h1:LsJvoBUVVnY7tkwwByCVtAwmp5bFXdyJNGU/+KVQJVM=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac h1:c7R33+TQGMYP6pvLUQQaqpdDFl+GZbhAcfGMI0285fo=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac/go.mod h1:qPuMUvCTaghsnYRDnRoRuztTyEKFlmi2S7gb44rH7WM=
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
@@ -352,8 +350,8 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8
github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
-github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g=
-github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE=
+github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4=
+github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws=
github.com/knadh/koanf v1.5.0 h1:q2TSd/3Pyc/5yP9ldIrSdIz26MCcyNQzW0pEAugLPNs=
@@ -383,8 +381,8 @@ github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38y
github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic=
github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFGkx3Q3WM=
github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro=
-github.com/libp2p/go-libp2p v0.31.0 h1:LFShhP8F6xthWiBBq3euxbKjZsoRajVEyBS9snfHxYg=
-github.com/libp2p/go-libp2p v0.31.0/go.mod h1:W/FEK1c/t04PbRH3fA9i5oucu5YcgrG0JVoBWT1B7Eg=
+github.com/libp2p/go-libp2p v0.32.0 h1:86I4B7nBUPIyTgw3+5Ibq6K7DdKRCuZw8URCfPc1hQM=
+github.com/libp2p/go-libp2p v0.32.0/go.mod h1:hXXC3kXPlBZ1eu8Q2hptGrMB4mZ3048JUoS4EKaHW5c=
github.com/libp2p/go-libp2p-asn-util v0.3.0 h1:gMDcMyYiZKkocGXDQ5nsUQyquC9+H+iLEQHwOCZ7s8s=
github.com/libp2p/go-libp2p-asn-util v0.3.0/go.mod h1:B1mcOrKUE35Xq/ASTmQ4tN3LNzVVaMNmq2NACuqyB9w=
github.com/libp2p/go-libp2p-kad-dht v0.25.1 h1:ofFNrf6MMEy4vi3R1VbJ7LOcTn3Csh0cDcaWHTxtWNA=
@@ -486,8 +484,8 @@ github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI1
github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew=
github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U=
github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM=
-github.com/multiformats/go-multistream v0.4.1 h1:rFy0Iiyn3YT0asivDUIR05leAdwZq3de4741sbiSdfo=
-github.com/multiformats/go-multistream v0.4.1/go.mod h1:Mz5eykRVAjJWckE2U78c6xqdtyNUEhKSM0Lwar2p77Q=
+github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE=
+github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA=
github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE=
github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8=
github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU=
@@ -569,12 +567,12 @@ github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k
github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo=
github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo=
github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A=
-github.com/quic-go/qtls-go1-20 v0.3.4 h1:MfFAPULvst4yoMgY9QmtpYmfij/em7O8UUi+bNVm7Cg=
-github.com/quic-go/qtls-go1-20 v0.3.4/go.mod h1:X9Nh97ZL80Z+bX/gUXMbipO6OxdiDi58b/fMC9mAL+k=
-github.com/quic-go/quic-go v0.38.1 h1:M36YWA5dEhEeT+slOu/SwMEucbYd0YFidxG3KlGPZaE=
-github.com/quic-go/quic-go v0.38.1/go.mod h1:ijnZM7JsFIkp4cRyjxJNIzdSfCLmUMg9wdyhGmg+SN4=
-github.com/quic-go/webtransport-go v0.5.3 h1:5XMlzemqB4qmOlgIus5zB45AcZ2kCgCy2EptUrfOPWU=
-github.com/quic-go/webtransport-go v0.5.3/go.mod h1:OhmmgJIzTTqXK5xvtuX0oBpLV2GkLWNDA+UeTGJXErU=
+github.com/quic-go/qtls-go1-20 v0.4.1 h1:D33340mCNDAIKBqXuAvexTNMUByrYmFYVfKfDN5nfFs=
+github.com/quic-go/qtls-go1-20 v0.4.1/go.mod h1:X9Nh97ZL80Z+bX/gUXMbipO6OxdiDi58b/fMC9mAL+k=
+github.com/quic-go/quic-go v0.40.0 h1:GYd1iznlKm7dpHD7pOVpUvItgMPo/jrMgDWZhMCecqw=
+github.com/quic-go/quic-go v0.40.0/go.mod h1:PeN7kuVJ4xZbxSv/4OX6S1USOX8MJvydwpTx31vx60c=
+github.com/quic-go/webtransport-go v0.6.0 h1:CvNsKqc4W2HljHJnoT+rMmbRJybShZ0YPFDD3NxaZLY=
+github.com/quic-go/webtransport-go v0.6.0/go.mod h1:9KjU4AEBqEQidGHNDkZrb8CAa1abRaosM2yGOyiikEc=
github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk=
github.com/raulk/go-watchdog v1.3.0/go.mod h1:fIvOnLbF0b0ZwkB9YU4mOW9Did//4vPZtDqv66NfsMU=
github.com/rhnvrm/simples3 v0.6.1/go.mod h1:Y+3vYm2V7Y4VijFoJHHTrja6OgPrJ2cBti8dPGkC3sA=
@@ -694,6 +692,8 @@ go.uber.org/fx v1.20.1/go.mod h1:iSYNbHf2y55acNCwCXKx7LbWb5WG1Bnue5RDXz1OREg=
go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ=
go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk=
go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo=
+go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo=
+go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc=
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0=
@@ -820,7 +820,6 @@ golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211103235746-7861aae1554b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE=
@@ -857,7 +856,6 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn
golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
-golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc=
@@ -885,8 +883,8 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 h1:AB/lmRny7e2pLhFEYIbl5qkDAUt2h0ZRO4wGPhZf+ik=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405/go.mod h1:67X1fPuzjcrkymZzZV1vvkFeTn2Rvc6lYF9MYFGCcwE=
google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw=
google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio=
google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 3abd303ab..83ca86d17 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -5,7 +5,7 @@ go 1.21
replace github.com/iotaledger/iota-core => ../../
require (
- github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b
+ github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
)
@@ -32,8 +32,8 @@ require (
github.com/flynn/noise v1.0.0 // indirect
github.com/fogleman/gg v1.3.0 // indirect
github.com/francoispqt/gojay v1.2.13 // indirect
- github.com/fsnotify/fsnotify v1.6.0 // indirect
- github.com/go-logr/logr v1.2.4 // indirect
+ github.com/fsnotify/fsnotify v1.7.0 // indirect
+ github.com/go-logr/logr v1.3.0 // indirect
github.com/go-logr/stdr v1.2.2 // indirect
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect
github.com/goccy/go-graphviz v0.1.1 // indirect
@@ -41,13 +41,12 @@ require (
github.com/gogo/protobuf v1.3.2 // indirect
github.com/golang-jwt/jwt v3.2.2+incompatible // indirect
github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0 // indirect
- github.com/golang/mock v1.6.0 // indirect
github.com/golang/protobuf v1.5.3 // indirect
github.com/google/go-github v17.0.0+incompatible // indirect
github.com/google/go-querystring v1.1.0 // indirect
github.com/google/gopacket v1.1.19 // indirect
- github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98 // indirect
- github.com/google/uuid v1.3.1 // indirect
+ github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b // indirect
+ github.com/google/uuid v1.4.0 // indirect
github.com/gorilla/websocket v1.5.0 // indirect
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect
github.com/hashicorp/errwrap v1.1.0 // indirect
@@ -58,21 +57,21 @@ require (
github.com/huin/goupnp v1.3.0 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 // indirect
+ github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 // indirect
- github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d // indirect
+ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 // indirect
+ github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac // indirect
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
@@ -83,7 +82,7 @@ require (
github.com/jackpal/go-nat-pmp v1.0.2 // indirect
github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect
github.com/jbenet/goprocess v0.1.4 // indirect
- github.com/klauspost/compress v1.17.1 // indirect
+ github.com/klauspost/compress v1.17.2 // indirect
github.com/klauspost/cpuid/v2 v2.2.5 // indirect
github.com/knadh/koanf v1.5.0 // indirect
github.com/koron/go-ssdp v0.0.4 // indirect
@@ -93,7 +92,7 @@ require (
github.com/libp2p/go-buffer-pool v0.1.0 // indirect
github.com/libp2p/go-cidranger v1.1.0 // indirect
github.com/libp2p/go-flow-metrics v0.1.0 // indirect
- github.com/libp2p/go-libp2p v0.31.0 // indirect
+ github.com/libp2p/go-libp2p v0.32.0 // indirect
github.com/libp2p/go-libp2p-asn-util v0.3.0 // indirect
github.com/libp2p/go-libp2p-kad-dht v0.25.1 // indirect
github.com/libp2p/go-libp2p-kbucket v0.6.3 // indirect
@@ -124,7 +123,7 @@ require (
github.com/multiformats/go-multibase v0.2.0 // indirect
github.com/multiformats/go-multicodec v0.9.0 // indirect
github.com/multiformats/go-multihash v0.2.3 // indirect
- github.com/multiformats/go-multistream v0.4.1 // indirect
+ github.com/multiformats/go-multistream v0.5.0 // indirect
github.com/multiformats/go-varint v0.0.7 // indirect
github.com/onsi/ginkgo/v2 v2.13.0 // indirect
github.com/opencontainers/runtime-spec v1.1.0 // indirect
@@ -143,9 +142,9 @@ require (
github.com/prometheus/common v0.45.0 // indirect
github.com/prometheus/procfs v0.12.0 // indirect
github.com/quic-go/qpack v0.4.0 // indirect
- github.com/quic-go/qtls-go1-20 v0.3.4 // indirect
- github.com/quic-go/quic-go v0.38.1 // indirect
- github.com/quic-go/webtransport-go v0.5.3 // indirect
+ github.com/quic-go/qtls-go1-20 v0.4.1 // indirect
+ github.com/quic-go/quic-go v0.40.0 // indirect
+ github.com/quic-go/webtransport-go v0.6.0 // indirect
github.com/raulk/go-watchdog v1.3.0 // indirect
github.com/sasha-s/go-deadlock v0.3.1 // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
@@ -165,6 +164,7 @@ require (
go.uber.org/atomic v1.11.0 // indirect
go.uber.org/dig v1.17.1 // indirect
go.uber.org/fx v1.20.1 // indirect
+ go.uber.org/mock v0.3.0 // indirect
go.uber.org/multierr v1.11.0 // indirect
go.uber.org/zap v1.26.0 // indirect
golang.org/x/crypto v0.14.0 // indirect
@@ -178,7 +178,7 @@ require (
golang.org/x/time v0.3.0 // indirect
golang.org/x/tools v0.14.0 // indirect
gonum.org/v1/gonum v0.14.0 // indirect
- google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect
+ google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 // indirect
google.golang.org/grpc v1.59.0 // indirect
google.golang.org/protobuf v1.31.0 // indirect
gopkg.in/yaml.v2 v2.4.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index 969b82ae7..f3ed230d4 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -119,8 +119,8 @@ github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHk
github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0=
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
-github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY=
-github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw=
+github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA=
+github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM=
github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
github.com/gliderlabs/ssh v0.1.1/go.mod h1:U7qILu1NlMHj9FlMhZLlkCdDnU1DBEAqr0aevW3Awn0=
github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q=
@@ -132,8 +132,8 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
-github.com/go-logr/logr v1.2.4 h1:g01GSCwiDw2xSZfjJ2/T9M+S6pFdcNtFYsp+Y43HYDQ=
-github.com/go-logr/logr v1.2.4/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
+github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY=
+github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
@@ -162,8 +162,6 @@ github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4er
github.com/golang/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:tluoj9z5200jBnyusfRPU2LqT6J+DAorxEvtC7LHB+E=
github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
-github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc=
-github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
@@ -206,12 +204,12 @@ github.com/google/gopacket v1.1.19/go.mod h1:iJ8V8n6KS+z2U1A8pUwu8bW5SyEMkXJB8Yo
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
github.com/google/pprof v0.0.0-20211214055906-6f57359322fd/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg=
-github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98 h1:pUa4ghanp6q4IJHwE9RwLgmVFfReJN+KbQ8ExNEUUoQ=
-github.com/google/pprof v0.0.0-20230926050212-f7f687d19a98/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik=
+github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b h1:RMpPgZTSApbPf7xaVel+QkoGPRLFLrwFO89uDUHEGf0=
+github.com/google/pprof v0.0.0-20231023181126-ff6d637d2a7b/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik=
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4=
-github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4=
+github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/gax-go v2.0.0+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY=
github.com/googleapis/gax-go/v2 v2.0.3/go.mod h1:LLvjysVCY1JZeum8Z6l8qUty8fiNwE08qbEPm1M08qg=
github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY=
@@ -279,40 +277,40 @@ github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJ
github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b h1:D68khiAFb9DwTvjZc2nc4R0E6wUdKyYCUXkmdaMzuoQ=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
-github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b h1:mX3NXaTMLEwZnEs4IlxEvXY0YZo8qbb8M1xM39FS6qY=
-github.com/iotaledger/hive.go/app v0.0.0-20231020115340-13da292c580b/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42 h1:EOfxTuAiBmED1VHuVh7/UIeB27cCRe13gdSzyioNMBw=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42 h1:xAER9M9Uoz2EOWT43E9wmXRe+RmAk8OBSUoboH4Su8M=
+github.com/iotaledger/hive.go/app v0.0.0-20231027195901-620bd7470e42/go.mod h1:8ZbIKR84oQd/3iQ5eeT7xpudO9/ytzXP7veIYnk7Orc=
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3 h1:4aVJTc0KS77uEw0Tny4r0n1ORwcbAQDECaCclgf/6lE=
github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3/go.mod h1:TZeAqieDu+xDOZp2e9+S+8pZp1PrfgcwLUnxmd8IgLU=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42 h1:drmpgLlJy7kZ09Dt1qKSnbILU+27Qu2jp4VdPDNwbFk=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b h1:ZERXxhQBUBV1AqTE6cUI4vTxSx4JrnsMuLZFgj32xLM=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42 h1:BC5GkIHyXdoJGdw6Tu5ds2kjw9grFLtwQiuMaKfdLU8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42 h1:r8TkdQJB7/bJd8cF8z5GQ+rX/7JpbPdPoN7wMoV1OCM=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b h1:8b2sH+2Vf0y5BDYTMwKa09iQr3JF9JrzTI64DkXb+9U=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42 h1:ytzZZPtclAzLfjxv26frbinCGx3Z6ouUENbx5U7lFGg=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42 h1:QMxd32Y/veVhTDPCiOFgetjUbG7sr9MryF29/rSPkMA=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b h1:LusmtjpfG/q8lc15Fp9W3kABbN3tArKx/zw2ibdY1DU=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42 h1:/xPwStUckZ2V0XPoY496cXU+c5elpHyvYoT6JAmuvRY=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42 h1:AvNLzONVMspwx7nD/NyYUgb5Hi7/zgzIOegr1uRD/M8=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b h1:IwhoeOeRu25mBdrimuOOvbbhHYX0QipibV69ubn8nX0=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
-github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b h1:EhVgAU/f2J3VYZwP60dRdyfAeDU3c/gBzX9blKtQGKA=
-github.com/iotaledger/hive.go/logger v0.0.0-20231020115340-13da292c580b/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 h1:e1uJAlXE3zeXpa+c4uFOG+/AMFbUlLt2mcrSK5NMxVs=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42 h1:7wjs4t1snBDJ8LOTl+tZhr2ORywSOTgJMppxiIAMA0A=
+github.com/iotaledger/hive.go/logger v0.0.0-20231027195901-620bd7470e42/go.mod h1:aBfAfIB2GO/IblhYt5ipCbyeL9bXSNeAwtYVA3hZaHg=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 h1:1QMJ39qXIx/IZVzus3+97IV7Pa++e+d340TvbMjhiBU=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b h1:zaXZn9yV/95SRDkgCZQeBbSbmcJTKSZbCB7oBd71Qwg=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42 h1:hZli4E9kJUAEQ7gzZR1XbPcpgqvqMPYq8YBPMbrBuos=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26 h1:ZZs7IzdxrogQWGF1HfUUfR3KW8WhfxE4hUbfwZCDXFY=
-github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231028104239-869296c43f26/go.mod h1:aFS0dN6QgKGgZakGgEv57NOLw+pLGdEiGcfDZ3h9GL0=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d h1:0SVvkN04C+Ylc2puM/c77HuvRMmHRl0BkNjlZx1YWeA=
-github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231028104044-69b02af0058d/go.mod h1:WFa5hHen6fi3RBX4K6r4fzhGpoh+7KJVIyFztZHdM84=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 h1:17JDzMKTMXKF3xys6gPURRddkZhg1LY+xwfhbr/sVqg=
+github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5/go.mod h1:LsJvoBUVVnY7tkwwByCVtAwmp5bFXdyJNGU/+KVQJVM=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac h1:c7R33+TQGMYP6pvLUQQaqpdDFl+GZbhAcfGMI0285fo=
+github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac/go.mod h1:qPuMUvCTaghsnYRDnRoRuztTyEKFlmi2S7gb44rH7WM=
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
@@ -356,8 +354,8 @@ github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8
github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
-github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g=
-github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE=
+github.com/klauspost/compress v1.17.2 h1:RlWWUY/Dr4fL8qk9YG7DTZ7PDgME2V4csBXA8L/ixi4=
+github.com/klauspost/compress v1.17.2/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
github.com/klauspost/cpuid/v2 v2.2.5/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws=
github.com/knadh/koanf v1.5.0 h1:q2TSd/3Pyc/5yP9ldIrSdIz26MCcyNQzW0pEAugLPNs=
@@ -387,8 +385,8 @@ github.com/libp2p/go-cidranger v1.1.0 h1:ewPN8EZ0dd1LSnrtuwd4709PXVcITVeuwbag38y
github.com/libp2p/go-cidranger v1.1.0/go.mod h1:KWZTfSr+r9qEo9OkI9/SIEeAtw+NNoU0dXIXt15Okic=
github.com/libp2p/go-flow-metrics v0.1.0 h1:0iPhMI8PskQwzh57jB9WxIuIOQ0r+15PChFGkx3Q3WM=
github.com/libp2p/go-flow-metrics v0.1.0/go.mod h1:4Xi8MX8wj5aWNDAZttg6UPmc0ZrnFNsMtpsYUClFtro=
-github.com/libp2p/go-libp2p v0.31.0 h1:LFShhP8F6xthWiBBq3euxbKjZsoRajVEyBS9snfHxYg=
-github.com/libp2p/go-libp2p v0.31.0/go.mod h1:W/FEK1c/t04PbRH3fA9i5oucu5YcgrG0JVoBWT1B7Eg=
+github.com/libp2p/go-libp2p v0.32.0 h1:86I4B7nBUPIyTgw3+5Ibq6K7DdKRCuZw8URCfPc1hQM=
+github.com/libp2p/go-libp2p v0.32.0/go.mod h1:hXXC3kXPlBZ1eu8Q2hptGrMB4mZ3048JUoS4EKaHW5c=
github.com/libp2p/go-libp2p-asn-util v0.3.0 h1:gMDcMyYiZKkocGXDQ5nsUQyquC9+H+iLEQHwOCZ7s8s=
github.com/libp2p/go-libp2p-asn-util v0.3.0/go.mod h1:B1mcOrKUE35Xq/ASTmQ4tN3LNzVVaMNmq2NACuqyB9w=
github.com/libp2p/go-libp2p-kad-dht v0.25.1 h1:ofFNrf6MMEy4vi3R1VbJ7LOcTn3Csh0cDcaWHTxtWNA=
@@ -490,8 +488,8 @@ github.com/multiformats/go-multicodec v0.9.0/go.mod h1:L3QTQvMIaVBkXOXXtVmYE+LI1
github.com/multiformats/go-multihash v0.0.8/go.mod h1:YSLudS+Pi8NHE7o6tb3D8vrpKa63epEDmG8nTduyAew=
github.com/multiformats/go-multihash v0.2.3 h1:7Lyc8XfX/IY2jWb/gI7JP+o7JEq9hOa7BFvVU9RSh+U=
github.com/multiformats/go-multihash v0.2.3/go.mod h1:dXgKXCXjBzdscBLk9JkjINiEsCKRVch90MdaGiKsvSM=
-github.com/multiformats/go-multistream v0.4.1 h1:rFy0Iiyn3YT0asivDUIR05leAdwZq3de4741sbiSdfo=
-github.com/multiformats/go-multistream v0.4.1/go.mod h1:Mz5eykRVAjJWckE2U78c6xqdtyNUEhKSM0Lwar2p77Q=
+github.com/multiformats/go-multistream v0.5.0 h1:5htLSLl7lvJk3xx3qT/8Zm9J4K8vEOf/QGkvOGQAyiE=
+github.com/multiformats/go-multistream v0.5.0/go.mod h1:n6tMZiwiP2wUsR8DgfDWw1dydlEqV3l6N3/GBsX6ILA=
github.com/multiformats/go-varint v0.0.1/go.mod h1:3Ls8CIEsrijN6+B7PbrXRPxHRPuXSrVKRY101jdMZYE=
github.com/multiformats/go-varint v0.0.7 h1:sWSGR+f/eu5ABZA2ZpYKBILXTTs9JWpdEM/nEGOHFS8=
github.com/multiformats/go-varint v0.0.7/go.mod h1:r8PUYw/fD/SjBCiKOoDlGF6QawOELpZAu9eioSos/OU=
@@ -573,12 +571,12 @@ github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k
github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo=
github.com/quic-go/qpack v0.4.0 h1:Cr9BXA1sQS2SmDUWjSofMPNKmvF6IiIfDRmgU0w1ZCo=
github.com/quic-go/qpack v0.4.0/go.mod h1:UZVnYIfi5GRk+zI9UMaCPsmZ2xKJP7XBUvVyT1Knj9A=
-github.com/quic-go/qtls-go1-20 v0.3.4 h1:MfFAPULvst4yoMgY9QmtpYmfij/em7O8UUi+bNVm7Cg=
-github.com/quic-go/qtls-go1-20 v0.3.4/go.mod h1:X9Nh97ZL80Z+bX/gUXMbipO6OxdiDi58b/fMC9mAL+k=
-github.com/quic-go/quic-go v0.38.1 h1:M36YWA5dEhEeT+slOu/SwMEucbYd0YFidxG3KlGPZaE=
-github.com/quic-go/quic-go v0.38.1/go.mod h1:ijnZM7JsFIkp4cRyjxJNIzdSfCLmUMg9wdyhGmg+SN4=
-github.com/quic-go/webtransport-go v0.5.3 h1:5XMlzemqB4qmOlgIus5zB45AcZ2kCgCy2EptUrfOPWU=
-github.com/quic-go/webtransport-go v0.5.3/go.mod h1:OhmmgJIzTTqXK5xvtuX0oBpLV2GkLWNDA+UeTGJXErU=
+github.com/quic-go/qtls-go1-20 v0.4.1 h1:D33340mCNDAIKBqXuAvexTNMUByrYmFYVfKfDN5nfFs=
+github.com/quic-go/qtls-go1-20 v0.4.1/go.mod h1:X9Nh97ZL80Z+bX/gUXMbipO6OxdiDi58b/fMC9mAL+k=
+github.com/quic-go/quic-go v0.40.0 h1:GYd1iznlKm7dpHD7pOVpUvItgMPo/jrMgDWZhMCecqw=
+github.com/quic-go/quic-go v0.40.0/go.mod h1:PeN7kuVJ4xZbxSv/4OX6S1USOX8MJvydwpTx31vx60c=
+github.com/quic-go/webtransport-go v0.6.0 h1:CvNsKqc4W2HljHJnoT+rMmbRJybShZ0YPFDD3NxaZLY=
+github.com/quic-go/webtransport-go v0.6.0/go.mod h1:9KjU4AEBqEQidGHNDkZrb8CAa1abRaosM2yGOyiikEc=
github.com/raulk/go-watchdog v1.3.0 h1:oUmdlHxdkXRJlwfG0O9omj8ukerm8MEQavSiDTEtBsk=
github.com/raulk/go-watchdog v1.3.0/go.mod h1:fIvOnLbF0b0ZwkB9YU4mOW9Did//4vPZtDqv66NfsMU=
github.com/rhnvrm/simples3 v0.6.1/go.mod h1:Y+3vYm2V7Y4VijFoJHHTrja6OgPrJ2cBti8dPGkC3sA=
@@ -698,6 +696,8 @@ go.uber.org/fx v1.20.1/go.mod h1:iSYNbHf2y55acNCwCXKx7LbWb5WG1Bnue5RDXz1OREg=
go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ=
go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk=
go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo=
+go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo=
+go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc=
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0=
@@ -824,7 +824,6 @@ golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211103235746-7861aae1554b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE=
@@ -861,7 +860,6 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn
golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
-golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk=
golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc=
@@ -889,8 +887,8 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98
google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 h1:AB/lmRny7e2pLhFEYIbl5qkDAUt2h0ZRO4wGPhZf+ik=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405/go.mod h1:67X1fPuzjcrkymZzZV1vvkFeTn2Rvc6lYF9MYFGCcwE=
google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw=
google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio=
google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index a92922bf8..dbe403965 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -22,23 +22,23 @@ require (
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect
github.com/ethereum/go-ethereum v1.13.4 // indirect
- github.com/google/uuid v1.3.1 // indirect
+ github.com/google/uuid v1.4.0 // indirect
github.com/holiman/uint256 v1.2.3 // indirect
github.com/iancoleman/orderedmap v0.3.0 // indirect
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect
- github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42 // indirect
- github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b // indirect
- github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b // indirect
- github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b // indirect
+ github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42 // indirect
+ github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42 // indirect
+ github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42 // indirect
+ github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 // indirect
+ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42 // indirect
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/klauspost/cpuid/v2 v2.2.5 // indirect
github.com/kr/text v0.2.0 // indirect
github.com/libp2p/go-buffer-pool v0.1.0 // indirect
- github.com/libp2p/go-libp2p v0.31.0 // indirect
+ github.com/libp2p/go-libp2p v0.32.0 // indirect
github.com/minio/sha256-simd v1.0.1 // indirect
github.com/multiformats/go-base32 v0.1.0 // indirect
github.com/multiformats/go-base36 v0.2.0 // indirect
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index 880f443ff..4f07d0064 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -20,36 +20,36 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS
github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38=
github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
-github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4=
-github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
+github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4=
+github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/holiman/uint256 v1.2.3 h1:K8UWO1HUJpRMXBxbmaY1Y8IAMZC/RsKB+ArEnnK4l5o=
github.com/holiman/uint256 v1.2.3/go.mod h1:SC8Ryt4n+UBbPbIBKaG9zbbDlp4jOru9xFZmPzLUTxw=
github.com/iancoleman/orderedmap v0.3.0 h1:5cbR2grmZR/DiVt+VJopEhtVs9YGInGIxAoMJn+Ichc=
github.com/iancoleman/orderedmap v0.3.0/go.mod h1:XuLcCUkdL5owUCQeF2Ue9uuw1EptkJDkXXS7VoV7XGE=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PTNgli6EbS4tV9qu3QAm/kBU3XaYZV2xdzys=
github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b h1:D68khiAFb9DwTvjZc2nc4R0E6wUdKyYCUXkmdaMzuoQ=
-github.com/iotaledger/hive.go/ads v0.0.0-20231020115340-13da292c580b/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42 h1:EOfxTuAiBmED1VHuVh7/UIeB27cCRe13gdSzyioNMBw=
+github.com/iotaledger/hive.go/ads v0.0.0-20231027195901-620bd7470e42/go.mod h1:IFh0gDfeMgZtfCo+5afK59IDR4xXh+cTR9YtLnZPcbY=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42 h1:drmpgLlJy7kZ09Dt1qKSnbILU+27Qu2jp4VdPDNwbFk=
github.com/iotaledger/hive.go/constraints v0.0.0-20231027195901-620bd7470e42/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b h1:ZERXxhQBUBV1AqTE6cUI4vTxSx4JrnsMuLZFgj32xLM=
-github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231020115340-13da292c580b/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42 h1:BC5GkIHyXdoJGdw6Tu5ds2kjw9grFLtwQiuMaKfdLU8=
+github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231027195901-620bd7470e42/go.mod h1:Mc+ACqBGPxrPMIPUBOm6/HL0J6m0iVMwjtIEKW3uow8=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42 h1:r8TkdQJB7/bJd8cF8z5GQ+rX/7JpbPdPoN7wMoV1OCM=
github.com/iotaledger/hive.go/crypto v0.0.0-20231027195901-620bd7470e42/go.mod h1:h3o6okvMSEK3KOX6pOp3yq1h9ohTkTfo6X8MzEadeb0=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b h1:8b2sH+2Vf0y5BDYTMwKa09iQr3JF9JrzTI64DkXb+9U=
-github.com/iotaledger/hive.go/ds v0.0.0-20231020115340-13da292c580b/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42 h1:ytzZZPtclAzLfjxv26frbinCGx3Z6ouUENbx5U7lFGg=
+github.com/iotaledger/hive.go/ds v0.0.0-20231027195901-620bd7470e42/go.mod h1:3XkUSKfHaVxGbT0XAvjNlVYqPzhfLTGhDtdNA5UBPco=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42 h1:QMxd32Y/veVhTDPCiOFgetjUbG7sr9MryF29/rSPkMA=
github.com/iotaledger/hive.go/ierrors v0.0.0-20231027195901-620bd7470e42/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b h1:LusmtjpfG/q8lc15Fp9W3kABbN3tArKx/zw2ibdY1DU=
-github.com/iotaledger/hive.go/kvstore v0.0.0-20231020115340-13da292c580b/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42 h1:/xPwStUckZ2V0XPoY496cXU+c5elpHyvYoT6JAmuvRY=
+github.com/iotaledger/hive.go/kvstore v0.0.0-20231027195901-620bd7470e42/go.mod h1:O/U3jtiUDeqqM0MZQFu2UPqS9fUm0C5hNISxlmg/thE=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42 h1:AvNLzONVMspwx7nD/NyYUgb5Hi7/zgzIOegr1uRD/M8=
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42/go.mod h1:s4kzx9QY1MVWHJralj+3q5kI0eARtrJhphYD/iBbPfo=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b h1:IwhoeOeRu25mBdrimuOOvbbhHYX0QipibV69ubn8nX0=
-github.com/iotaledger/hive.go/log v0.0.0-20231020115340-13da292c580b/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42 h1:e1uJAlXE3zeXpa+c4uFOG+/AMFbUlLt2mcrSK5NMxVs=
+github.com/iotaledger/hive.go/log v0.0.0-20231027195901-620bd7470e42/go.mod h1:JvokzmpmFZPDskMlUqqjgHtD8usVJU4nAY/TNMGge8M=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42 h1:1QMJ39qXIx/IZVzus3+97IV7Pa++e+d340TvbMjhiBU=
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42/go.mod h1:jRw8yFipiPaqmTPHh7hTcxAP9u6pjRGpByS3REJKkbY=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b h1:zaXZn9yV/95SRDkgCZQeBbSbmcJTKSZbCB7oBd71Qwg=
-github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231020115340-13da292c580b/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42 h1:hZli4E9kJUAEQ7gzZR1XbPcpgqvqMPYq8YBPMbrBuos=
+github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
@@ -64,8 +64,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/libp2p/go-buffer-pool v0.1.0 h1:oK4mSFcQz7cTQIfqbe4MIj9gLW+mnanjyFtc6cdF0Y8=
github.com/libp2p/go-buffer-pool v0.1.0/go.mod h1:N+vh8gMqimBzdKkSMVuydVDq+UV5QTWy5HSiZacSbPg=
-github.com/libp2p/go-libp2p v0.31.0 h1:LFShhP8F6xthWiBBq3euxbKjZsoRajVEyBS9snfHxYg=
-github.com/libp2p/go-libp2p v0.31.0/go.mod h1:W/FEK1c/t04PbRH3fA9i5oucu5YcgrG0JVoBWT1B7Eg=
+github.com/libp2p/go-libp2p v0.32.0 h1:86I4B7nBUPIyTgw3+5Ibq6K7DdKRCuZw8URCfPc1hQM=
+github.com/libp2p/go-libp2p v0.32.0/go.mod h1:hXXC3kXPlBZ1eu8Q2hptGrMB4mZ3048JUoS4EKaHW5c=
github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM=
github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8=
github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o=
From 8ed294dfd3624419a1e5526eb0bded22edce97c8 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Tue, 31 Oct 2023 16:25:11 +0100
Subject: [PATCH 31/56] Update default snapshot for feature network
---
.github/workflows/feature-network-deploy.yml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/.github/workflows/feature-network-deploy.yml b/.github/workflows/feature-network-deploy.yml
index c4c71e425..812411683 100644
--- a/.github/workflows/feature-network-deploy.yml
+++ b/.github/workflows/feature-network-deploy.yml
@@ -70,7 +70,7 @@ jobs:
- name: Ansible deploy
env:
CUSTOM_SNAPSHOT_URL: '${{ github.event.inputs.snapshotUrl }}'
- DEFAULT_SNAPSHOT_URL: 'https://0x0.st/HJXh.bin'
+ DEFAULT_SNAPSHOT_URL: 'https://0x0.st/HywH.bin'
NETWORK_ENVIRONMENT: '${{ secrets.NETWORK_ENVIRONMENT }}'
IOTA_CORE_DOCKER_IMAGE_REPO: 'iotaledger/iota-core'
IOTA_CORE_DOCKER_IMAGE_TAG: 'feature'
From 83b575c24c45c5dda21173136ba7c1725acde6d2 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 11:40:13 +0100
Subject: [PATCH 32/56] Action for running docker network and checking is
health
---
.github/workflows/docker-network-health.yml | 46 +++++++++++++++++++++
1 file changed, 46 insertions(+)
create mode 100644 .github/workflows/docker-network-health.yml
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
new file mode 100644
index 000000000..bc1d9e2f7
--- /dev/null
+++ b/.github/workflows/docker-network-health.yml
@@ -0,0 +1,46 @@
+name: Run Docker Network and Check Health
+
+on:
+ pull_request:
+
+jobs:
+ run-and-check:
+ runs-on: ubuntu-latest
+
+ steps:
+ - name: Checkout code
+ uses: actions/checkout@v2
+
+ - name: Execute run.sh in Background with Timeout
+ run: |
+ cd ./tools/docker-network
+ timeout 300s ./run.sh 0 0 &
+ RUN_PID=$!
+ echo "RUN_PID=$RUN_PID" >> $GITHUB_ENV
+
+ - name: Polling health and killing run.sh if healthy
+ run: |
+ SUCCESS=false
+ while true; do
+ OUTPUT=$(curl -o /dev/null -s -w "%{http_code}\n" http://localhost:8080/health)
+ if [[ $OUTPUT -eq 200 ]]; then
+ SUCCESS=true
+ kill -s SIGINT $RUN_PID
+ break
+ # curl will return a connection refused when the network is tear down from the timeout.
+ elif [[ $OUTPUT -eq 000 ]]; then
+ echo "Connection refused. Failing the action."
+ break
+ fi
+ sleep 5
+ done
+ if [[ ! $SUCCESS ]]; then
+ echo "Health check never returned 200. Failing the action."
+ exit 1
+ fi
+
+ - name: Cleanup
+ run: |
+ cd ./tools/docker-network
+ docker compose kill
+ docker compose down -v
From 43e6c442f0e0a7a96ec6f3bda5eef22e389bf4f9 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 11:41:03 +0100
Subject: [PATCH 33/56] Ignore paths
---
.github/workflows/docker-network-health.yml | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index bc1d9e2f7..01213c028 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -2,6 +2,10 @@ name: Run Docker Network and Check Health
on:
pull_request:
+ paths-ignore:
+ - 'documentation/**'
+ - 'scripts/**'
+ - 'tools/**'
jobs:
run-and-check:
From 3ba26b89013d89ea4094c9da82cee709c4d54d35 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 11:57:59 +0100
Subject: [PATCH 34/56] Wait for node-4 to be created before querying it
---
.github/workflows/docker-network-health.yml | 3 +++
1 file changed, 3 insertions(+)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 01213c028..0074fed9e 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -24,6 +24,9 @@ jobs:
- name: Polling health and killing run.sh if healthy
run: |
+ # wait for node-4 to be created before querying it
+ timeout 300s bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done'
+ sleep 10 # Additional 10 seconds wait
SUCCESS=false
while true; do
OUTPUT=$(curl -o /dev/null -s -w "%{http_code}\n" http://localhost:8080/health)
From bb7e47aaed15e177ac6b2514c0caa65464f91294 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 12:07:56 +0100
Subject: [PATCH 35/56] Aggregate parallel steps into one
---
.github/workflows/docker-network-health.yml | 21 ++++++++++++++-------
1 file changed, 14 insertions(+), 7 deletions(-)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 0074fed9e..f53816c20 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -15,18 +15,24 @@ jobs:
- name: Checkout code
uses: actions/checkout@v2
- - name: Execute run.sh in Background with Timeout
+ - name: Run network, wait and check health
run: |
+ set +x
+
+ # Run network
cd ./tools/docker-network
timeout 300s ./run.sh 0 0 &
RUN_PID=$!
- echo "RUN_PID=$RUN_PID" >> $GITHUB_ENV
-
- - name: Polling health and killing run.sh if healthy
- run: |
- # wait for node-4 to be created before querying it
- timeout 300s bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done'
+
+ # Wait for node-4 to be created before querying it
+ timeout 300s bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done' &
+ WAIT_FOR_CONTAINER_PID=$!
+
+ wait $WAIT_FOR_CONTAINER_PID || exit 1
+
sleep 10 # Additional 10 seconds wait
+
+ # Health check
SUCCESS=false
while true; do
OUTPUT=$(curl -o /dev/null -s -w "%{http_code}\n" http://localhost:8080/health)
@@ -41,6 +47,7 @@ jobs:
fi
sleep 5
done
+
if [[ ! $SUCCESS ]]; then
echo "Health check never returned 200. Failing the action."
exit 1
From 8b6d522469eb44c869508ece99471f98533ce711 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 12:10:32 +0100
Subject: [PATCH 36/56] Log executed commands
---
.github/workflows/docker-network-health.yml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index f53816c20..7f834540c 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -17,7 +17,7 @@ jobs:
- name: Run network, wait and check health
run: |
- set +x
+ set -x
# Run network
cd ./tools/docker-network
From 3ac1b98fea78e28c39357ea492783c518020a0a4 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 12:12:40 +0100
Subject: [PATCH 37/56] Avoid clashing by allowing no action concurrency
---
.github/workflows/docker-network-health.yml | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 7f834540c..e8bf5818b 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -7,6 +7,10 @@ on:
- 'scripts/**'
- 'tools/**'
+concurrency:
+ group: run-and-check-group
+ cancel-in-progress: false
+
jobs:
run-and-check:
runs-on: ubuntu-latest
From a8a4dd52d6825d04b0367754021ef28bb5f4a6e0 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 12:17:05 +0100
Subject: [PATCH 38/56] Allow ON workflow_dispatch
---
.github/workflows/docker-network-health.yml | 1 +
1 file changed, 1 insertion(+)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index e8bf5818b..d0f96a10b 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -1,6 +1,7 @@
name: Run Docker Network and Check Health
on:
+ workflow_dispatch:
pull_request:
paths-ignore:
- 'documentation/**'
From f4351844e62c2b1685bc983c74c20543dd98b7f6 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 12:23:14 +0100
Subject: [PATCH 39/56] Increase timeouts
---
.github/workflows/docker-network-health.yml | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index d0f96a10b..46a160843 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -26,11 +26,11 @@ jobs:
# Run network
cd ./tools/docker-network
- timeout 300s ./run.sh 0 0 &
+ timeout 10m ./run.sh 0 0 &
RUN_PID=$!
# Wait for node-4 to be created before querying it
- timeout 300s bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done' &
+ timeout 10m bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done' &
WAIT_FOR_CONTAINER_PID=$!
wait $WAIT_FOR_CONTAINER_PID || exit 1
From d3cf8d092eb8a67f8a2c8a64d8abfaa524d3b599 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:16:04 +0100
Subject: [PATCH 40/56] Do not depend on Go to be installed on the host to
create snapshot
---
tools/docker-network/run.sh | 35 +++++++++++++++++------------------
1 file changed, 17 insertions(+), 18 deletions(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 7ac82d24a..78e5a8a32 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -12,12 +12,6 @@ fi
REPLICAS=${1:-1}
MONITORING=${2:-0}
-FEATURE=${3:-0}
-
-DOCKER_COMPOSE_FILE=docker-compose.yml
-if [ $FEATURE -ne 0 ]; then
- DOCKER_COMPOSE_FILE=docker-compose-feature.yml
-fi
export DOCKER_BUILDKIT=1
export COMPOSE_DOCKER_CLI_BUILD=1
@@ -34,7 +28,7 @@ fi
# Allow docker compose to build and cache an image
echo $DOCKER_BUILD_CONTEXT $DOCKERFILE_PATH
-docker compose -f $DOCKER_COMPOSE_FILE build --build-arg WITH_GO_WORK=${WITH_GO_WORK:-0} --build-arg DOCKER_BUILD_CONTEXT=${DOCKER_BUILD_CONTEXT} --build-arg DOCKERFILE_PATH=${DOCKERFILE_PATH}
+docker compose build --build-arg WITH_GO_WORK=${WITH_GO_WORK:-0} --build-arg DOCKER_BUILD_CONTEXT=${DOCKER_BUILD_CONTEXT} --build-arg DOCKERFILE_PATH=${DOCKERFILE_PATH}
docker compose pull inx-indexer inx-blockissuer inx-faucet inx-validator-1
@@ -46,18 +40,23 @@ fi
# create snapshot file
echo "Create snapshot"
-if [ $FEATURE -ne 0 ]; then
- pushd ../genesis-snapshot
- go run -tags=rocksdb . --config feature
-else
- pushd ../genesis-snapshot
- go run -tags=rocksdb . --config docker --seed 7R1itJx5hVuo9w9hjg5cwKFmek4HMSoBDgJZN8hKGxih
-fi
-popd
-mv ../genesis-snapshot/*.snapshot .
+
+# Run Go command in Docker container
+docker run --rm \
+ --user $(id -u) \
+ -v "$(realpath $(pwd)/../../):/workspace" \
+ -v "$(go env GOCACHE):/go-cache" \
+ -v "$(go env GOMODCACHE):/go-mod-cache" \
+ -e GOCACHE="/go-cache" \
+ -e GOMODCACHE="/go-mod-cache" \
+ -w "/workspace/tools/genesis-snapshot" \
+ golang:1.21 go run -tags=rocksdb . --config docker --seed 7R1itJx5hVuo9w9hjg5cwKFmek4HMSoBDgJZN8hKGxih
+
+# Move and set permissions for the .snapshot file
+mv -f ../genesis-snapshot/*.snapshot .
chmod o+r *.snapshot
-echo "Run iota-core network with ${DOCKER_COMPOSE_FILE}"
+echo "Run iota-core network"
# IOTA_CORE_PEER_REPLICAS is used in docker-compose.yml to determine how many replicas to create
export IOTA_CORE_PEER_REPLICAS=$REPLICAS
# Profiles is created to set which docker profiles to run
@@ -68,7 +67,7 @@ if [ $MONITORING -ne 0 ]; then
fi
export COMPOSE_PROFILES=$(join , ${PROFILES[@]})
-docker compose -f $DOCKER_COMPOSE_FILE up
+docker compose up
echo "Clean up docker resources"
docker compose down -v
From b98d2c397da7b86dc8be0e622e3bda69da73d882 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:30:38 +0100
Subject: [PATCH 41/56] Run all actions on self-hosted runners
---
.github/workflows/build_tools.yml | 2 +-
.github/workflows/docker-network-health.yml | 2 +-
.github/workflows/feature-network-deploy.yml | 2 +-
3 files changed, 3 insertions(+), 3 deletions(-)
diff --git a/.github/workflows/build_tools.yml b/.github/workflows/build_tools.yml
index 262c56563..18481f055 100644
--- a/.github/workflows/build_tools.yml
+++ b/.github/workflows/build_tools.yml
@@ -8,7 +8,7 @@ jobs:
build:
name: Import Check
- runs-on: ubuntu-latest
+ runs-on: self-hosted
steps:
- name: Checkout repository
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 46a160843..32a40097b 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -14,7 +14,7 @@ concurrency:
jobs:
run-and-check:
- runs-on: ubuntu-latest
+ runs-on: self-hosted
steps:
- name: Checkout code
diff --git a/.github/workflows/feature-network-deploy.yml b/.github/workflows/feature-network-deploy.yml
index 812411683..42bce3b5d 100644
--- a/.github/workflows/feature-network-deploy.yml
+++ b/.github/workflows/feature-network-deploy.yml
@@ -13,7 +13,7 @@ on:
jobs:
deploy:
environment: feature
- runs-on: ubuntu-latest
+ runs-on: self-hosted
env:
DOCKER_BUILDKIT: 1
steps:
From cf54f7981f930d545356653d10dd545940d5241b Mon Sep 17 00:00:00 2001
From: jkrvivian
Date: Wed, 1 Nov 2023 20:33:56 +0800
Subject: [PATCH 42/56] Fix inx port of inx-indexer in docker-compose
---
tools/docker-network/docker-compose.yml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tools/docker-network/docker-compose.yml b/tools/docker-network/docker-compose.yml
index a16297065..3cfd3f017 100644
--- a/tools/docker-network/docker-compose.yml
+++ b/tools/docker-network/docker-compose.yml
@@ -191,7 +191,7 @@ services:
networks:
- iota-core
command: >
- --inx.address=node-1-validator:9019
+ --inx.address=node-1-validator:9029
--restAPI.bindAddress=inx-indexer:9011
inx-blockissuer:
From e1668d7f6db321d6444a63e27e49bc4fa461de55 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:39:03 +0100
Subject: [PATCH 43/56] Do not assume go is installed or cached on the runner
---
tools/docker-network/run.sh | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 78e5a8a32..6cac8241e 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -45,8 +45,8 @@ echo "Create snapshot"
docker run --rm \
--user $(id -u) \
-v "$(realpath $(pwd)/../../):/workspace" \
- -v "$(go env GOCACHE):/go-cache" \
- -v "$(go env GOMODCACHE):/go-mod-cache" \
+ -v "${HOME}/.cache/go-build:/go-cache" \
+ -v "${HOME}/go/pkg/mod:/go-mod-cache" \
-e GOCACHE="/go-cache" \
-e GOMODCACHE="/go-mod-cache" \
-w "/workspace/tools/genesis-snapshot" \
From 9faf3cccc04e023bee74e2af0cd62a7659ad766f Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:51:28 +0100
Subject: [PATCH 44/56] Fail fast if main run.sh exits
---
.github/workflows/docker-network-health.yml | 8 +++++---
1 file changed, 5 insertions(+), 3 deletions(-)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 32a40097b..2c4bf728a 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -32,10 +32,12 @@ jobs:
# Wait for node-4 to be created before querying it
timeout 10m bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done' &
WAIT_FOR_CONTAINER_PID=$!
+
+ # Wait for any of the two processes to exit
+ wait -n || exit 1
- wait $WAIT_FOR_CONTAINER_PID || exit 1
-
- sleep 10 # Additional 10 seconds wait
+ # Additional 10 seconds wait to allow the API to come up
+ sleep 10
# Health check
SUCCESS=false
From 84f65ad82fae0a5854e01f805766ed72b94b0e4f Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:51:57 +0100
Subject: [PATCH 45/56] Test: REVERT THIS
---
tools/docker-network/run.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 6cac8241e..86f5e261f 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -45,7 +45,7 @@ echo "Create snapshot"
docker run --rm \
--user $(id -u) \
-v "$(realpath $(pwd)/../../):/workspace" \
- -v "${HOME}/.cache/go-build:/go-cache" \
+ -v "$(go env BLA):/go-cache" \
-v "${HOME}/go/pkg/mod:/go-mod-cache" \
-e GOCACHE="/go-cache" \
-e GOMODCACHE="/go-mod-cache" \
From 018eda1265b8afd25cd0cb6773be279005ffc33c Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:53:16 +0100
Subject: [PATCH 46/56] Simplify
---
.github/workflows/docker-network-health.yml | 1 -
1 file changed, 1 deletion(-)
diff --git a/.github/workflows/docker-network-health.yml b/.github/workflows/docker-network-health.yml
index 2c4bf728a..ee5b1b08b 100644
--- a/.github/workflows/docker-network-health.yml
+++ b/.github/workflows/docker-network-health.yml
@@ -31,7 +31,6 @@ jobs:
# Wait for node-4 to be created before querying it
timeout 10m bash -c 'until docker ps | grep docker-network-node-4; do sleep 5; done' &
- WAIT_FOR_CONTAINER_PID=$!
# Wait for any of the two processes to exit
wait -n || exit 1
From f204c7be7d2ecd17ac77c4de31b6eaa1d758e8a4 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Wed, 1 Nov 2023 13:53:22 +0100
Subject: [PATCH 47/56] Revert "Test: REVERT THIS"
This reverts commit 84f65ad82fae0a5854e01f805766ed72b94b0e4f.
---
tools/docker-network/run.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 86f5e261f..6cac8241e 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -45,7 +45,7 @@ echo "Create snapshot"
docker run --rm \
--user $(id -u) \
-v "$(realpath $(pwd)/../../):/workspace" \
- -v "$(go env BLA):/go-cache" \
+ -v "${HOME}/.cache/go-build:/go-cache" \
-v "${HOME}/go/pkg/mod:/go-mod-cache" \
-e GOCACHE="/go-cache" \
-e GOMODCACHE="/go-mod-cache" \
From 4f88b45f4fb9a90a6c51cd351784feba9c6c57e0 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Thu, 2 Nov 2023 12:51:34 +0100
Subject: [PATCH 48/56] Use safemath in accounts
---
components/debugapi/node.go | 7 +++-
components/restapi/core/accounts.go | 19 +++++++----
components/restapi/core/component.go | 5 ++-
pkg/core/account/accounts.go | 34 ++++++++++++++-----
pkg/core/account/accounts_test.go | 6 ++--
pkg/core/account/seated_accounts.go | 9 +++--
pkg/core/account/seated_accounts_test.go | 9 ++---
pkg/protocol/commitment_verifier.go | 7 +++-
.../slotattestation/testframework_test.go | 4 ++-
.../engine/filter/blockfilter/filter_test.go | 4 ++-
.../conflictdag/tests/accounts_framework.go | 4 ++-
.../seatmanager/mock/mockseatmanager.go | 18 ++++++----
.../sybilprotection/seatmanager/poa/poa.go | 18 +++++++---
.../seatmanager/topstakers/topstakers.go | 27 +++++++++++----
.../seatmanager/topstakers/topstakers_test.go | 29 +++++++++-------
.../performance/testsuite_test.go | 6 ++--
.../sybilprotectionv1/sybilprotection.go | 16 +++++++--
pkg/storage/testframework_test.go | 4 ++-
pkg/testsuite/sybilprotection.go | 14 +++++---
19 files changed, 169 insertions(+), 71 deletions(-)
diff --git a/components/debugapi/node.go b/components/debugapi/node.go
index 280311b76..e625cfdc6 100644
--- a/components/debugapi/node.go
+++ b/components/debugapi/node.go
@@ -17,7 +17,12 @@ func validatorsSummary() (*ValidatorsSummaryResponse, error) {
}
var validatorSeats []*Validator
- latestCommittee.Accounts().ForEach(func(id iotago.AccountID, pool *account.Pool) bool {
+ accounts, err := latestCommittee.Accounts()
+ if err != nil {
+ return nil, ierrors.Wrapf(err, "failed to get accounts from committee for slot %d", latestSlotIndex)
+ }
+
+ accounts.ForEach(func(id iotago.AccountID, pool *account.Pool) bool {
validatorSeats = append(validatorSeats, &Validator{
AccountID: id,
SeatIndex: uint8(lo.Return1(latestCommittee.GetSeat(id))),
diff --git a/components/restapi/core/accounts.go b/components/restapi/core/accounts.go
index 91c6211c3..e6080f473 100644
--- a/components/restapi/core/accounts.go
+++ b/components/restapi/core/accounts.go
@@ -188,7 +188,7 @@ func rewardsByOutputID(c echo.Context) (*apimodels.ManaRewardsResponse, error) {
}, nil
}
-func selectedCommittee(c echo.Context) *apimodels.CommitteeResponse {
+func selectedCommittee(c echo.Context) (*apimodels.CommitteeResponse, error) {
timeProvider := deps.Protocol.CommittedAPI().TimeProvider()
var slot iotago.SlotIndex
@@ -206,11 +206,16 @@ func selectedCommittee(c echo.Context) *apimodels.CommitteeResponse {
if !exists {
return &apimodels.CommitteeResponse{
Epoch: epoch,
- }
+ }, nil
+ }
+
+ accounts, err := seatedAccounts.Accounts()
+ if err != nil {
+ return nil, ierrors.Wrapf(err, "failed to get accounts from committee for slot %d", slot)
}
- committee := make([]*apimodels.CommitteeMemberResponse, 0, seatedAccounts.Accounts().Size())
- seatedAccounts.Accounts().ForEach(func(accountID iotago.AccountID, seat *account.Pool) bool {
+ committee := make([]*apimodels.CommitteeMemberResponse, 0, accounts.Size())
+ accounts.ForEach(func(accountID iotago.AccountID, seat *account.Pool) bool {
committee = append(committee, &apimodels.CommitteeMemberResponse{
AccountID: accountID,
PoolStake: seat.PoolStake,
@@ -224,7 +229,7 @@ func selectedCommittee(c echo.Context) *apimodels.CommitteeResponse {
return &apimodels.CommitteeResponse{
Epoch: epoch,
Committee: committee,
- TotalStake: seatedAccounts.Accounts().TotalStake(),
- TotalValidatorStake: seatedAccounts.Accounts().TotalValidatorStake(),
- }
+ TotalStake: accounts.TotalStake(),
+ TotalValidatorStake: accounts.TotalValidatorStake(),
+ }, nil
}
diff --git a/components/restapi/core/component.go b/components/restapi/core/component.go
index 1d1784943..8361b8d8f 100644
--- a/components/restapi/core/component.go
+++ b/components/restapi/core/component.go
@@ -357,7 +357,10 @@ func configure() error {
}, checkNodeSynced())
routeGroup.GET(RouteCommittee, func(c echo.Context) error {
- resp := selectedCommittee(c)
+ resp, err := selectedCommittee(c)
+ if err != nil {
+ return err
+ }
return responseByHeader(c, resp)
}, checkNodeSynced())
diff --git a/pkg/core/account/accounts.go b/pkg/core/account/accounts.go
index 45fdc7f3c..770a72de1 100644
--- a/pkg/core/account/accounts.go
+++ b/pkg/core/account/accounts.go
@@ -6,6 +6,7 @@ import (
"io"
"sync/atomic"
+ "github.com/iotaledger/hive.go/core/safemath"
"github.com/iotaledger/hive.go/ds/shrinkingmap"
"github.com/iotaledger/hive.go/ierrors"
"github.com/iotaledger/hive.go/runtime/syncutils"
@@ -71,31 +72,44 @@ func (a *Accounts) Get(id iotago.AccountID) (pool *Pool, exists bool) {
}
// setWithoutLocking sets the weight of the given identity.
-func (a *Accounts) setWithoutLocking(id iotago.AccountID, pool *Pool) {
+func (a *Accounts) setWithoutLocking(id iotago.AccountID, pool *Pool) error {
value, created := a.accountPools.GetOrCreate(id, func() *Pool {
return pool
})
+ var safeMathErr error
+
if !created {
// if there was already an entry, we need to subtract the former
// stake first and set the new value
- // TODO: use safemath
- a.totalStake -= value.PoolStake
- a.totalValidatorStake -= value.ValidatorStake
+ if a.totalStake, safeMathErr = safemath.SafeSub(a.totalStake, value.PoolStake); safeMathErr != nil {
+ return ierrors.Wrapf(safeMathErr, "failed to subtract pool stake from total stake for account %s", id.String())
+ }
+
+ if a.totalValidatorStake, safeMathErr = safemath.SafeSub(a.totalValidatorStake, value.ValidatorStake); safeMathErr != nil {
+ return ierrors.Wrapf(safeMathErr, "failed to subtract validator stake from total validator stake for account %s", id.String())
+ }
a.accountPools.Set(id, pool)
}
- a.totalStake += pool.PoolStake
- a.totalValidatorStake += pool.ValidatorStake
+ if a.totalStake, safeMathErr = safemath.SafeAdd(a.totalStake, pool.PoolStake); safeMathErr != nil {
+ return ierrors.Wrapf(safeMathErr, "failed to add pool stake to total stake for account %s", id.String())
+ }
+
+ if a.totalValidatorStake, safeMathErr = safemath.SafeAdd(a.totalValidatorStake, pool.ValidatorStake); safeMathErr != nil {
+ return ierrors.Wrapf(safeMathErr, "failed to add validator stake to total validator stake for account %s", id.String())
+ }
+
+ return nil
}
// Set sets the weight of the given identity.
-func (a *Accounts) Set(id iotago.AccountID, pool *Pool) {
+func (a *Accounts) Set(id iotago.AccountID, pool *Pool) error {
a.mutex.Lock()
defer a.mutex.Unlock()
- a.setWithoutLocking(id, pool)
+ return a.setWithoutLocking(id, pool)
}
func (a *Accounts) TotalStake() iotago.BaseToken {
@@ -168,7 +182,9 @@ func (a *Accounts) readFromReadSeeker(reader io.ReadSeeker) (n int, err error) {
return 0, ierrors.Wrap(err, "invalid pool bytes length")
}
- a.setWithoutLocking(accountID, pool)
+ if err := a.setWithoutLocking(accountID, pool); err != nil {
+ return 0, ierrors.Wrapf(err, "failed to set pool for account %s", accountID.String())
+ }
}
var reused bool
diff --git a/pkg/core/account/accounts_test.go b/pkg/core/account/accounts_test.go
index 7838ddf6b..ffc7dbe8e 100644
--- a/pkg/core/account/accounts_test.go
+++ b/pkg/core/account/accounts_test.go
@@ -20,11 +20,13 @@ func TestAccounts(t *testing.T) {
// check "Set"
for id, stake := range issuers {
- accounts.Set(id, &account.Pool{
+ if err := accounts.Set(id, &account.Pool{
PoolStake: iotago.BaseToken(stake),
ValidatorStake: iotago.BaseToken(stake) * 2,
FixedCost: iotago.Mana(stake) * 3,
- })
+ }); err != nil {
+ t.Fatal(err)
+ }
}
// check "Size"
diff --git a/pkg/core/account/seated_accounts.go b/pkg/core/account/seated_accounts.go
index 4e8abffb1..070eda6e4 100644
--- a/pkg/core/account/seated_accounts.go
+++ b/pkg/core/account/seated_accounts.go
@@ -78,19 +78,22 @@ func (s *SeatedAccounts) SeatCount() int {
return s.seatsByAccount.Size()
}
-func (s *SeatedAccounts) Accounts() *Accounts {
+func (s *SeatedAccounts) Accounts() (*Accounts, error) {
accounts := NewAccounts()
+ var err error
s.seatsByAccount.ForEachKey(func(id iotago.AccountID) bool {
pool, exists := s.accounts.Get(id)
if !exists {
panic("account not found")
}
- accounts.Set(id, pool)
+ if err = accounts.Set(id, pool); err != nil {
+ return false
+ }
return true
})
- return accounts
+ return accounts, err
}
func (s *SeatedAccounts) String() string {
diff --git a/pkg/core/account/seated_accounts_test.go b/pkg/core/account/seated_accounts_test.go
index 2158d34d6..c87b8abda 100644
--- a/pkg/core/account/seated_accounts_test.go
+++ b/pkg/core/account/seated_accounts_test.go
@@ -19,9 +19,9 @@ func TestSelectedAccounts(t *testing.T) {
account3 := iotago.AccountID([32]byte{3})
account4 := iotago.AccountID([32]byte{4})
- accounts.Set(account1, &account.Pool{})
- accounts.Set(account2, &account.Pool{})
- accounts.Set(account3, &account.Pool{})
+ require.NoError(t, accounts.Set(account1, &account.Pool{}))
+ require.NoError(t, accounts.Set(account2, &account.Pool{}))
+ require.NoError(t, accounts.Set(account3, &account.Pool{}))
// Create a new set of selected accounts
seatedAccounts := account.NewSeatedAccounts(accounts, account1, account3)
@@ -65,7 +65,8 @@ func TestSelectedAccounts(t *testing.T) {
require.True(t, has)
// Test the "Members" method
- members := seatedAccounts.Accounts()
+ members, err := seatedAccounts.Accounts()
+ require.NoError(t, err)
require.Equal(t, 2, members.Size())
require.True(t, members.Has(account2))
require.True(t, members.Has(account3))
diff --git a/pkg/protocol/commitment_verifier.go b/pkg/protocol/commitment_verifier.go
index 1315071e3..7289fd86d 100644
--- a/pkg/protocol/commitment_verifier.go
+++ b/pkg/protocol/commitment_verifier.go
@@ -25,10 +25,15 @@ func NewCommitmentVerifier(mainEngine *engine.Engine, lastCommonCommitmentBefore
return nil, ierrors.Errorf("committee in slot %d does not exist", lastCommonCommitmentBeforeFork.Slot())
}
+ accountsAtForkingPoint, err := committeeAtForkingPoint.Accounts()
+ if err != nil {
+ return nil, ierrors.Wrapf(err, "failed to get accounts from committee for slot %d", lastCommonCommitmentBeforeFork.Slot())
+ }
+
return &CommitmentVerifier{
engine: mainEngine,
cumulativeWeight: lastCommonCommitmentBeforeFork.CumulativeWeight(),
- validatorAccountsAtFork: lo.PanicOnErr(mainEngine.Ledger.PastAccounts(committeeAtForkingPoint.Accounts().IDs(), lastCommonCommitmentBeforeFork.Slot())),
+ validatorAccountsAtFork: lo.PanicOnErr(mainEngine.Ledger.PastAccounts(accountsAtForkingPoint.IDs(), lastCommonCommitmentBeforeFork.Slot())),
// TODO: what happens if the committee rotated after the fork?
}, nil
}
diff --git a/pkg/protocol/engine/attestation/slotattestation/testframework_test.go b/pkg/protocol/engine/attestation/slotattestation/testframework_test.go
index ad77b48fe..1214dd6b3 100644
--- a/pkg/protocol/engine/attestation/slotattestation/testframework_test.go
+++ b/pkg/protocol/engine/attestation/slotattestation/testframework_test.go
@@ -61,7 +61,9 @@ func NewTestFramework(test *testing.T) *TestFramework {
accounts := account.NewAccounts()
var members []iotago.AccountID
t.issuerByAlias.ForEach(func(alias string, issuer *issuer) bool {
- accounts.Set(issuer.accountID, &account.Pool{}) // we don't care about pools with PoA
+ if err := accounts.Set(issuer.accountID, &account.Pool{}); err != nil { // we don't care about pools with PoA
+ test.Fatal(err)
+ }
members = append(members, issuer.accountID)
return true
})
diff --git a/pkg/protocol/engine/filter/blockfilter/filter_test.go b/pkg/protocol/engine/filter/blockfilter/filter_test.go
index 6f1b4bdc4..2ab3aac3f 100644
--- a/pkg/protocol/engine/filter/blockfilter/filter_test.go
+++ b/pkg/protocol/engine/filter/blockfilter/filter_test.go
@@ -98,7 +98,9 @@ func (t *TestFramework) IssueBlockAtSlotWithVersion(alias string, slot iotago.Sl
func mockedCommitteeFunc(validatorAccountID iotago.AccountID) func(iotago.SlotIndex) (*account.SeatedAccounts, bool) {
mockedAccounts := account.NewAccounts()
- mockedAccounts.Set(validatorAccountID, new(account.Pool))
+ if err := mockedAccounts.Set(validatorAccountID, new(account.Pool)); err != nil {
+ panic(err)
+ }
seatedAccounts := account.NewSeatedAccounts(mockedAccounts)
seatedAccounts.Set(account.SeatIndex(0), validatorAccountID)
diff --git a/pkg/protocol/engine/mempool/conflictdag/tests/accounts_framework.go b/pkg/protocol/engine/mempool/conflictdag/tests/accounts_framework.go
index bc54c0fc8..6a32efefb 100644
--- a/pkg/protocol/engine/mempool/conflictdag/tests/accounts_framework.go
+++ b/pkg/protocol/engine/mempool/conflictdag/tests/accounts_framework.go
@@ -74,7 +74,9 @@ func (f *AccountsTestFramework) CreateID(alias string) iotago.AccountID {
validatorID := iotago.AccountIDFromData(hashedAlias[:])
validatorID.RegisterAlias(alias)
- f.Instance.Set(validatorID, &account.Pool{}) // we don't care about pools when doing PoA
+ if err := f.Instance.Set(validatorID, &account.Pool{}); err != nil { // we don't care about pools when doing PoA
+ f.test.Fatal(err)
+ }
f.Committee.Set(account.SeatIndex(f.Committee.SeatCount()), validatorID)
f.identitiesByAlias[alias] = validatorID
diff --git a/pkg/protocol/sybilprotection/seatmanager/mock/mockseatmanager.go b/pkg/protocol/sybilprotection/seatmanager/mock/mockseatmanager.go
index d8b53b4a1..7e90c5bc4 100644
--- a/pkg/protocol/sybilprotection/seatmanager/mock/mockseatmanager.go
+++ b/pkg/protocol/sybilprotection/seatmanager/mock/mockseatmanager.go
@@ -61,11 +61,13 @@ func NewManualPOAProvider() module.Provider[*engine.Engine, seatmanager.SeatMana
func (m *ManualPOA) AddRandomAccount(alias string) iotago.AccountID {
id := iotago.AccountID(tpkg.Rand32ByteArray())
id.RegisterAlias(alias)
- m.accounts.Set(id, &account.Pool{
+ if err := m.accounts.Set(id, &account.Pool{ // We don't care about pools with PoA, but need to set something to avoid division by zero errors.
PoolStake: 1,
ValidatorStake: 1,
FixedCost: 1,
- }) // We don't care about pools with PoA, but need to set something to avoid division by zero errors.
+ }); err != nil {
+ panic(err)
+ }
m.aliases.Set(alias, id)
@@ -79,11 +81,13 @@ func (m *ManualPOA) AddRandomAccount(alias string) iotago.AccountID {
}
func (m *ManualPOA) AddAccount(id iotago.AccountID, alias string) iotago.AccountID {
- m.accounts.Set(id, &account.Pool{
+ if err := m.accounts.Set(id, &account.Pool{ // We don't care about pools with PoA, but need to set something to avoid division by zero errors.
PoolStake: 1,
ValidatorStake: 1,
FixedCost: 1,
- }) // We don't care about pools with PoA, but need to set something to avoid division by zero errors.
+ }); err != nil {
+ panic(err)
+ }
m.aliases.Set(alias, id)
m.committee = m.accounts.SelectCommittee(m.accounts.IDs()...)
@@ -164,11 +168,13 @@ func (m *ManualPOA) RotateCommittee(epoch iotago.EpochIndex, validators accounts
m.accounts = account.NewAccounts()
for _, validatorData := range validators {
- m.accounts.Set(validatorData.ID, &account.Pool{
+ if err := m.accounts.Set(validatorData.ID, &account.Pool{
PoolStake: validatorData.ValidatorStake + validatorData.DelegationStake,
ValidatorStake: validatorData.ValidatorStake,
FixedCost: validatorData.FixedCost,
- })
+ }); err != nil {
+ return nil, ierrors.Wrapf(err, "error while setting pool for epoch %d for validator %s", epoch, validatorData.ID.String())
+ }
}
m.committee = m.accounts.SelectCommittee(m.accounts.IDs()...)
}
diff --git a/pkg/protocol/sybilprotection/seatmanager/poa/poa.go b/pkg/protocol/sybilprotection/seatmanager/poa/poa.go
index d73ce7191..f5c113c76 100644
--- a/pkg/protocol/sybilprotection/seatmanager/poa/poa.go
+++ b/pkg/protocol/sybilprotection/seatmanager/poa/poa.go
@@ -89,17 +89,23 @@ func (s *SeatManager) RotateCommittee(epoch iotago.EpochIndex, validators accoun
committeeAccounts := account.NewAccounts()
for _, validatorData := range validators {
- committeeAccounts.Set(validatorData.ID, &account.Pool{
+ if err := committeeAccounts.Set(validatorData.ID, &account.Pool{
PoolStake: validatorData.ValidatorStake + validatorData.DelegationStake,
ValidatorStake: validatorData.ValidatorStake,
FixedCost: validatorData.FixedCost,
- })
+ }); err != nil {
+ return nil, ierrors.Wrapf(err, "error while setting committee for epoch %d for validator %s", epoch, validatorData.ID.String())
+ }
}
s.committee = committeeAccounts.SelectCommittee(committeeAccounts.IDs()...)
}
- err := s.committeeStore.Store(epoch, s.committee.Accounts())
+ accounts, err := s.committee.Accounts()
if err != nil {
+ return nil, ierrors.Wrapf(err, "error while getting accounts from committee for epoch %d", epoch)
+ }
+
+ if err := s.committeeStore.Store(epoch, accounts); err != nil {
return nil, ierrors.Wrapf(err, "error while storing committee for epoch %d", epoch)
}
@@ -186,8 +192,12 @@ func (s *SeatManager) SetCommittee(epoch iotago.EpochIndex, validators *account.
s.committee = validators.SelectCommittee(validators.IDs()...)
- err := s.committeeStore.Store(epoch, s.committee.Accounts())
+ accounts, err := s.committee.Accounts()
if err != nil {
+ return ierrors.Wrapf(err, "failed to get accounts from committee for epoch %d", epoch)
+ }
+
+ if err := s.committeeStore.Store(epoch, accounts); err != nil {
return ierrors.Wrapf(err, "failed to set committee for epoch %d", epoch)
}
diff --git a/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers.go b/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers.go
index 4d572c888..464b83c9c 100644
--- a/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers.go
+++ b/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers.go
@@ -98,18 +98,29 @@ func (s *SeatManager) RotateCommittee(epoch iotago.EpochIndex, candidates accoun
return nil, ierrors.Errorf("cannot re-use previous committee from epoch %d as it does not exist", epoch-1)
}
- err := s.committeeStore.Store(epoch, committee.Accounts())
+ accounts, err := committee.Accounts()
if err != nil {
+ return nil, ierrors.Wrapf(err, "error while getting accounts from committee for epoch %d", epoch-1)
+ }
+
+ if err := s.committeeStore.Store(epoch, accounts); err != nil {
return nil, ierrors.Wrapf(err, "error while storing committee for epoch %d", epoch)
}
return committee, nil
}
- committee := s.selectNewCommittee(candidates)
+ committee, err := s.selectNewCommittee(candidates)
+ if err != nil {
+ return nil, ierrors.Wrap(err, "error while selecting new committee")
+ }
- err := s.committeeStore.Store(epoch, committee.Accounts())
+ accounts, err := committee.Accounts()
if err != nil {
+ return nil, ierrors.Wrapf(err, "error while getting accounts for newly selected committee for epoch %d", epoch)
+ }
+
+ if err := s.committeeStore.Store(epoch, accounts); err != nil {
return nil, ierrors.Wrapf(err, "error while storing committee for epoch %d", epoch)
}
@@ -203,7 +214,7 @@ func (s *SeatManager) SetCommittee(epoch iotago.EpochIndex, validators *account.
return nil
}
-func (s *SeatManager) selectNewCommittee(candidates accounts.AccountsData) *account.SeatedAccounts {
+func (s *SeatManager) selectNewCommittee(candidates accounts.AccountsData) (*account.SeatedAccounts, error) {
sort.Slice(candidates, func(i, j int) bool {
// Prioritize the candidate that has a larger pool stake.
if candidates[i].ValidatorStake+candidates[i].DelegationStake != candidates[j].ValidatorStake+candidates[j].DelegationStake {
@@ -233,13 +244,15 @@ func (s *SeatManager) selectNewCommittee(candidates accounts.AccountsData) *acco
newCommitteeAccounts := account.NewAccounts()
for _, candidateData := range candidates[:s.optsSeatCount] {
- newCommitteeAccounts.Set(candidateData.ID, &account.Pool{
+ if err := newCommitteeAccounts.Set(candidateData.ID, &account.Pool{
PoolStake: candidateData.ValidatorStake + candidateData.DelegationStake,
ValidatorStake: candidateData.ValidatorStake,
FixedCost: candidateData.FixedCost,
- })
+ }); err != nil {
+ return nil, ierrors.Wrapf(err, "error while setting pool for committee candidate %s", candidateData.ID.String())
+ }
}
committee := newCommitteeAccounts.SelectCommittee(newCommitteeAccounts.IDs()...)
- return committee
+ return committee, nil
}
diff --git a/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers_test.go b/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers_test.go
index 4b539e72f..54790e263 100644
--- a/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers_test.go
+++ b/pkg/protocol/sybilprotection/seatmanager/topstakers/topstakers_test.go
@@ -35,11 +35,13 @@ func TestTopStakers_InitializeCommittee(t *testing.T) {
// Create committee for epoch 0
initialCommittee := account.NewAccounts()
for i := 0; i < 3; i++ {
- initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
+ if err := initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
PoolStake: 1900,
ValidatorStake: 900,
FixedCost: 11,
- })
+ }); err != nil {
+ t.Fatal(err)
+ }
}
// Try setting committee that is too small - should return an error.
err := topStakersSeatManager.SetCommittee(0, initialCommittee)
@@ -53,7 +55,6 @@ func TestTopStakers_InitializeCommittee(t *testing.T) {
require.NoError(t, topStakersSeatManager.InitializeCommittee(0, time.Time{}))
assertOnlineCommittee(t, topStakersSeatManager.OnlineCommittee(), lo.Return1(weightedSeats.GetSeat(initialCommitteeAccountIDs[0])), lo.Return1(weightedSeats.GetSeat(initialCommitteeAccountIDs[2])), lo.Return1(weightedSeats.GetSeat(initialCommitteeAccountIDs[2])))
-
}
func TestTopStakers_RotateCommittee(t *testing.T) {
@@ -77,27 +78,27 @@ func TestTopStakers_RotateCommittee(t *testing.T) {
// Create committee for epoch 0
initialCommittee := account.NewAccounts()
- initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
+ require.NoError(t, initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
PoolStake: 1900,
ValidatorStake: 900,
FixedCost: 11,
- })
+ }))
- initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
+ require.NoError(t, initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
PoolStake: 1900,
ValidatorStake: 900,
FixedCost: 11,
- })
+ }))
// Try setting committee that is too small - should return an error.
err := topStakersSeatManager.SetCommittee(0, initialCommittee)
require.Error(t, err)
- initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
+ require.NoError(t, initialCommittee.Set(tpkg.RandAccountID(), &account.Pool{
PoolStake: 1900,
ValidatorStake: 900,
FixedCost: 11,
- })
+ }))
// Set committee with the correct size
err = topStakersSeatManager.SetCommittee(0, initialCommittee)
@@ -187,7 +188,9 @@ func TestTopStakers_RotateCommittee(t *testing.T) {
// Make sure that after committee rotation, the online committee is not changed.
assertOnlineCommittee(t, topStakersSeatManager.OnlineCommittee(), lo.Return1(weightedSeats.GetSeat(initialCommitteeAccountIDs[2])))
- newCommitteeMemberIDs := newCommittee.Accounts().IDs()
+ accounts, err := newCommittee.Accounts()
+ require.NoError(t, err)
+ newCommitteeMemberIDs := accounts.IDs()
// A new committee member appears online and makes the previously active committee seat inactive.
topStakersSeatManager.activityTracker.MarkSeatActive(lo.Return1(weightedSeats.GetSeat(newCommitteeMemberIDs[0])), newCommitteeMemberIDs[0], tpkg.TestAPI.TimeProvider().SlotEndTime(14))
@@ -213,9 +216,11 @@ func TestTopStakers_RotateCommittee(t *testing.T) {
}
func assertCommittee(t *testing.T, expectedCommittee *account.Accounts, actualCommittee *account.SeatedAccounts) {
- require.Equal(t, actualCommittee.Accounts().Size(), expectedCommittee.Size())
+ actualAccounts, err := actualCommittee.Accounts()
+ require.NoError(t, err)
+ require.Equal(t, actualAccounts.Size(), expectedCommittee.Size())
for _, memberID := range expectedCommittee.IDs() {
- require.Truef(t, actualCommittee.Accounts().Has(memberID), "expected account %s to be part of committee, but it is not, actual committee members: %s", memberID, actualCommittee.Accounts().IDs())
+ require.Truef(t, actualAccounts.Has(memberID), "expected account %s to be part of committee, but it is not, actual committee members: %s", memberID, actualAccounts.IDs())
}
}
diff --git a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/testsuite_test.go b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/testsuite_test.go
index 684e21916..62c5c1e48 100644
--- a/pkg/protocol/sybilprotection/sybilprotectionv1/performance/testsuite_test.go
+++ b/pkg/protocol/sybilprotection/sybilprotectionv1/performance/testsuite_test.go
@@ -106,11 +106,13 @@ func (t *TestSuite) ApplyEpochActions(epoch iotago.EpochIndex, actions map[strin
action.validate(t.T, t.api)
accountID := t.Account(alias, true)
- committee.Set(accountID, &account.Pool{
+ if err := committee.Set(accountID, &account.Pool{
PoolStake: action.PoolStake,
ValidatorStake: action.ValidatorStake,
FixedCost: action.FixedCost,
- })
+ }); err != nil {
+ t.T.Fatal(err)
+ }
}
// Store directly on the committee store, because in actual code the SeatManager is responsible for adding the storage entry.
diff --git a/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go b/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
index 2ebd75b0c..6389a2608 100644
--- a/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
+++ b/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
@@ -161,7 +161,11 @@ func (o *SybilProtection) CommitSlot(slot iotago.SlotIndex) (committeeRoot, rewa
panic(fmt.Sprintf("committee for current epoch %d not found", currentEpoch))
}
- committeeAccounts := committee.Accounts()
+ committeeAccounts, err := committee.Accounts()
+ if err != nil {
+ return iotago.Identifier{}, iotago.Identifier{}, ierrors.Wrapf(err, "failed to get accounts from committee for epoch %d", currentEpoch)
+ }
+
committeeAccounts.SetReused()
if err = o.seatManager.SetCommittee(nextEpoch, committeeAccounts); err != nil {
return iotago.Identifier{}, iotago.Identifier{}, ierrors.Wrapf(err, "failed to set committee for epoch %d", nextEpoch)
@@ -276,7 +280,13 @@ func (o *SybilProtection) slotFinalized(slot iotago.SlotIndex) {
epochEndSlot := timeProvider.EpochEnd(epoch)
if slot+apiForSlot.ProtocolParameters().EpochNearingThreshold() == epochEndSlot &&
epochEndSlot > o.lastCommittedSlot+apiForSlot.ProtocolParameters().MaxCommittableAge() {
- newCommittee := o.selectNewCommittee(slot)
+ newCommittee, err := o.selectNewCommittee(slot)
+ if err != nil {
+ // TODO: should we fail "harder" here?
+ o.errHandler(ierrors.Wrap(err, "error while selecting new committee"))
+
+ return
+ }
o.events.CommitteeSelected.Trigger(newCommittee, epoch+1)
}
}
@@ -371,7 +381,7 @@ func (o *SybilProtection) OrderedRegisteredCandidateValidatorsList(epoch iotago.
return validatorResp, nil
}
-func (o *SybilProtection) selectNewCommittee(slot iotago.SlotIndex) *account.Accounts {
+func (o *SybilProtection) selectNewCommittee(slot iotago.SlotIndex) (*account.Accounts, error) {
timeProvider := o.apiProvider.APIForSlot(slot).TimeProvider()
currentEpoch := timeProvider.EpochFromSlot(slot)
nextEpoch := currentEpoch + 1
diff --git a/pkg/storage/testframework_test.go b/pkg/storage/testframework_test.go
index 290f180dd..847cc0ef4 100644
--- a/pkg/storage/testframework_test.go
+++ b/pkg/storage/testframework_test.go
@@ -152,7 +152,9 @@ func (t *TestFramework) GenerateSemiPermanentData(epoch iotago.EpochIndex) {
createdBytes += int64(len(lo.PanicOnErr(versionAndHash.Bytes()))) + 8 // for epoch key
accounts := account.NewAccounts()
- accounts.Set(tpkg.RandAccountID(), &account.Pool{})
+ if err := accounts.Set(tpkg.RandAccountID(), &account.Pool{}); err != nil {
+ t.t.Fatal(err)
+ }
err = committeeStore.Store(epoch, accounts)
require.NoError(t.t, err)
createdBytes += int64(len(lo.PanicOnErr(accounts.Bytes()))) + 8 // for epoch key
diff --git a/pkg/testsuite/sybilprotection.go b/pkg/testsuite/sybilprotection.go
index 3eac2c524..39673cb29 100644
--- a/pkg/testsuite/sybilprotection.go
+++ b/pkg/testsuite/sybilprotection.go
@@ -17,13 +17,17 @@ func (t *TestSuite) AssertSybilProtectionCommittee(epoch iotago.EpochIndex, expe
for _, node := range nodes {
t.Eventually(func() error {
- accounts := lo.Return1(node.Protocol.MainEngineInstance().SybilProtection.SeatManager().CommitteeInEpoch(epoch)).Accounts().IDs()
- if !assert.ElementsMatch(t.fakeTesting, expectedAccounts, accounts) {
- return ierrors.Errorf("AssertSybilProtectionCommittee: %s: expected %s, got %s", node.Name, expectedAccounts, accounts)
+ accounts, err := lo.Return1(node.Protocol.MainEngineInstance().SybilProtection.SeatManager().CommitteeInEpoch(epoch)).Accounts()
+ if err != nil {
+ t.Testing.Fatal(err)
+ }
+ accountIDs := accounts.IDs()
+ if !assert.ElementsMatch(t.fakeTesting, expectedAccounts, accountIDs) {
+ return ierrors.Errorf("AssertSybilProtectionCommittee: %s: expected %s, got %s", node.Name, expectedAccounts, accountIDs)
}
- if len(expectedAccounts) != len(accounts) {
- return ierrors.Errorf("AssertSybilProtectionCommittee: %s: expected %v, got %v", node.Name, len(expectedAccounts), len(accounts))
+ if len(expectedAccounts) != len(accountIDs) {
+ return ierrors.Errorf("AssertSybilProtectionCommittee: %s: expected %v, got %v", node.Name, len(expectedAccounts), len(accountIDs))
}
return nil
From e81779ebfa449183c06d437bf8c93ce6eb5efc82 Mon Sep 17 00:00:00 2001
From: Andrew
Date: Thu, 2 Nov 2023 12:04:06 +0000
Subject: [PATCH 49/56] rename max parents
---
components/restapi/core/blocks.go | 2 +-
go.mod | 2 +-
go.sum | 4 +--
pkg/storage/testframework_test.go | 2 +-
pkg/testsuite/mock/blockissuer.go | 45 ++++++++++++++++---------
pkg/testsuite/testsuite_issue_blocks.go | 2 +-
tools/gendoc/go.mod | 2 +-
tools/gendoc/go.sum | 4 +--
tools/genesis-snapshot/go.mod | 2 +-
tools/genesis-snapshot/go.sum | 4 +--
10 files changed, 42 insertions(+), 27 deletions(-)
diff --git a/components/restapi/core/blocks.go b/components/restapi/core/blocks.go
index 554fdfa08..2b58599ea 100644
--- a/components/restapi/core/blocks.go
+++ b/components/restapi/core/blocks.go
@@ -45,7 +45,7 @@ func blockMetadataByID(c echo.Context) (*apimodels.BlockMetadataResponse, error)
}
func blockIssuanceBySlot(slotIndex iotago.SlotIndex) (*apimodels.IssuanceBlockHeaderResponse, error) {
- references := deps.Protocol.MainEngineInstance().TipSelection.SelectTips(iotago.BlockMaxParents)
+ references := deps.Protocol.MainEngineInstance().TipSelection.SelectTips(iotago.BasicBlockMaxParents)
var slotCommitment *model.Commitment
var err error
diff --git a/go.mod b/go.mod
index 1784168e9..006f26e31 100644
--- a/go.mod
+++ b/go.mod
@@ -25,7 +25,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac
- github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e
github.com/labstack/echo/v4 v4.11.2
github.com/labstack/gommon v0.4.0
github.com/libp2p/go-libp2p v0.32.0
diff --git a/go.sum b/go.sum
index 2d6f0464b..2af6a376f 100644
--- a/go.sum
+++ b/go.sum
@@ -307,8 +307,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 h1:17JDz
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5/go.mod h1:LsJvoBUVVnY7tkwwByCVtAwmp5bFXdyJNGU/+KVQJVM=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac h1:c7R33+TQGMYP6pvLUQQaqpdDFl+GZbhAcfGMI0285fo=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac/go.mod h1:qPuMUvCTaghsnYRDnRoRuztTyEKFlmi2S7gb44rH7WM=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e h1:ZYRC1MHn/ghsqtjIpYGTxLQrh5n5eUmC0/YWnJiTRhk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/pkg/storage/testframework_test.go b/pkg/storage/testframework_test.go
index 290f180dd..374d22fc4 100644
--- a/pkg/storage/testframework_test.go
+++ b/pkg/storage/testframework_test.go
@@ -91,7 +91,7 @@ func (t *TestFramework) GeneratePrunableData(epoch iotago.EpochIndex, size int64
var createdBytes int64
for createdBytes < size {
block := tpkg.RandBlock(&iotago.BasicBlockBody{
- StrongParents: tpkg.SortedRandBlockIDs(1 + rand.Intn(iotago.BlockMaxParents)),
+ StrongParents: tpkg.SortedRandBlockIDs(1 + rand.Intn(iotago.BasicBlockMaxParents)),
Payload: &iotago.TaggedData{Data: make([]byte, 8192)},
MaxBurnedMana: 1000,
}, apiForEpoch, 0)
diff --git a/pkg/testsuite/mock/blockissuer.go b/pkg/testsuite/mock/blockissuer.go
index 1e6fad7fa..509b152f2 100644
--- a/pkg/testsuite/mock/blockissuer.go
+++ b/pkg/testsuite/mock/blockissuer.go
@@ -125,8 +125,7 @@ func (i *BlockIssuer) CreateValidationBlock(ctx context.Context, alias string, i
}
if blockParams.BlockHeader.References == nil {
- // TODO: change this to get references for validator block
- references, err := i.getReferences(ctx, nil, node, blockParams.BlockHeader.ParentsCount)
+ references, err := i.getReferencesValidationBlock(ctx, node, blockParams.BlockHeader.ParentsCount)
require.NoError(i.Testing, err)
blockParams.BlockHeader.References = references
@@ -226,7 +225,7 @@ func (i *BlockIssuer) CreateBasicBlock(ctx context.Context, alias string, node *
}
if blockParams.BlockHeader.References == nil {
- references, err := i.getReferences(ctx, blockParams.Payload, node, blockParams.BlockHeader.ParentsCount)
+ references, err := i.getReferencesBasicBlock(ctx, node, blockParams.BlockHeader.ParentsCount)
require.NoError(i.Testing, err)
blockParams.BlockHeader.References = references
}
@@ -388,9 +387,9 @@ func (i *BlockIssuer) AttachBlock(ctx context.Context, iotaBlock *iotago.Block,
resign = true
}
- switch innerBlock := iotaBlock.Body.(type) {
+ switch blockBody := iotaBlock.Body.(type) {
case *iotago.BasicBlockBody:
- switch payload := innerBlock.Payload.(type) {
+ switch payload := blockBody.Payload.(type) {
case *iotago.SignedTransaction:
if payload.Transaction.NetworkID != protoParams.NetworkID() {
return iotago.EmptyBlockID, ierrors.Wrapf(ErrBlockAttacherInvalidBlock, "invalid payload, error: wrong networkID: %d", payload.Transaction.NetworkID)
@@ -398,21 +397,28 @@ func (i *BlockIssuer) AttachBlock(ctx context.Context, iotaBlock *iotago.Block,
}
if len(iotaBlock.Parents()) == 0 {
- references, referencesErr := i.getReferences(ctx, innerBlock.Payload, node)
+ references, referencesErr := i.getReferencesBasicBlock(ctx, node)
if referencesErr != nil {
return iotago.EmptyBlockID, ierrors.Wrapf(ErrBlockAttacherAttachingNotPossible, "tipselection failed, error: %w", referencesErr)
}
- innerBlock.StrongParents = references[iotago.StrongParentType]
- innerBlock.WeakParents = references[iotago.WeakParentType]
- innerBlock.ShallowLikeParents = references[iotago.ShallowLikeParentType]
+ blockBody.StrongParents = references[iotago.StrongParentType]
+ blockBody.WeakParents = references[iotago.WeakParentType]
+ blockBody.ShallowLikeParents = references[iotago.ShallowLikeParentType]
resign = true
}
case *iotago.ValidationBlockBody:
- //nolint:revive,staticcheck //temporarily disable
if len(iotaBlock.Parents()) == 0 {
- // TODO: implement tipselection for validator blocks
+ references, referencesErr := i.getReferencesValidationBlock(ctx, node)
+ if referencesErr != nil {
+ return iotago.EmptyBlockID, ierrors.Wrapf(ErrBlockAttacherAttachingNotPossible, "tipselection failed, error: %w", referencesErr)
+ }
+
+ blockBody.StrongParents = references[iotago.StrongParentType]
+ blockBody.WeakParents = references[iotago.WeakParentType]
+ blockBody.ShallowLikeParents = references[iotago.ShallowLikeParentType]
+ resign = true
}
}
@@ -544,13 +550,22 @@ func (i *BlockIssuer) getAddressableCommitment(currentAPI iotago.API, blockIssui
return commitment, nil
}
-func (i *BlockIssuer) getReferences(ctx context.Context, p iotago.Payload, node *Node, strongParentsCountOpt ...int) (model.ParentReferences, error) {
- strongParentsCount := iotago.BlockMaxParents
+func (i *BlockIssuer) getReferencesBasicBlock(ctx context.Context, node *Node, strongParentsCountOpt ...int) (model.ParentReferences, error) {
+ strongParentsCount := iotago.BasicBlockMaxParents
+ if len(strongParentsCountOpt) > 0 && strongParentsCountOpt[0] > 0 {
+ strongParentsCount = strongParentsCountOpt[0]
+ }
+
+ return i.getReferencesWithRetry(ctx, strongParentsCount, node)
+}
+
+func (i *BlockIssuer) getReferencesValidationBlock(ctx context.Context, node *Node, strongParentsCountOpt ...int) (model.ParentReferences, error) {
+ strongParentsCount := iotago.ValidationBlockMaxParents
if len(strongParentsCountOpt) > 0 && strongParentsCountOpt[0] > 0 {
strongParentsCount = strongParentsCountOpt[0]
}
- return i.getReferencesWithRetry(ctx, p, strongParentsCount, node)
+ return i.getReferencesWithRetry(ctx, strongParentsCount, node)
}
func (i *BlockIssuer) validateReferences(issuingTime time.Time, slotCommitmentIndex iotago.SlotIndex, references model.ParentReferences, node *Node) error {
@@ -594,7 +609,7 @@ func (i *BlockIssuer) CopyIdentityFromBlockIssuer(otherBlockIssuer *BlockIssuer)
// getReferencesWithRetry tries to get references for the given payload. If it fails, it will retry at regular intervals until
// the timeout is reached.
-func (i *BlockIssuer) getReferencesWithRetry(ctx context.Context, _ iotago.Payload, parentsCount int, node *Node) (references model.ParentReferences, err error) {
+func (i *BlockIssuer) getReferencesWithRetry(ctx context.Context, parentsCount int, node *Node) (references model.ParentReferences, err error) {
timeout := time.NewTimer(i.optsTipSelectionTimeout)
interval := time.NewTicker(i.optsTipSelectionRetryInterval)
defer timeutil.CleanupTimer(timeout)
diff --git a/pkg/testsuite/testsuite_issue_blocks.go b/pkg/testsuite/testsuite_issue_blocks.go
index 982f4a714..14210a24c 100644
--- a/pkg/testsuite/testsuite_issue_blocks.go
+++ b/pkg/testsuite/testsuite_issue_blocks.go
@@ -207,7 +207,7 @@ func (t *TestSuite) IssueBlockRowInSlot(prefix string, slot iotago.SlotIndex, ro
}
tx := t.DefaultWallet().CreateBasicOutputsEquallyFromInputs(txName, 1, inputName)
- issuingOptionsCopy[node.Name] = t.limitParentsCountInBlockOptions(issuingOptionsCopy[node.Name], iotago.BlockMaxParents)
+ issuingOptionsCopy[node.Name] = t.limitParentsCountInBlockOptions(issuingOptionsCopy[node.Name], iotago.BasicBlockMaxParents)
blockHeaderOptions := append(issuingOptionsCopy[node.Name], mock.WithIssuingTime(issuingTime))
t.assertParentsCommitmentExistFromBlockOptions(blockHeaderOptions, node)
diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod
index 83ca86d17..4fdc2c0b0 100644
--- a/tools/gendoc/go.mod
+++ b/tools/gendoc/go.mod
@@ -72,7 +72,7 @@ require (
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 // indirect
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 // indirect
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac // indirect
- github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 // indirect
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e // indirect
github.com/ipfs/boxo v0.13.1 // indirect
github.com/ipfs/go-cid v0.4.1 // indirect
github.com/ipfs/go-datastore v0.6.0 // indirect
diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum
index f3ed230d4..b16ac173a 100644
--- a/tools/gendoc/go.sum
+++ b/tools/gendoc/go.sum
@@ -311,8 +311,8 @@ github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5 h1:17JDz
github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231031135002-4c79ea5193f5/go.mod h1:LsJvoBUVVnY7tkwwByCVtAwmp5bFXdyJNGU/+KVQJVM=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac h1:c7R33+TQGMYP6pvLUQQaqpdDFl+GZbhAcfGMI0285fo=
github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231031134131-b6ad918dc1ac/go.mod h1:qPuMUvCTaghsnYRDnRoRuztTyEKFlmi2S7gb44rH7WM=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e h1:ZYRC1MHn/ghsqtjIpYGTxLQrh5n5eUmC0/YWnJiTRhk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI=
github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod
index dbe403965..1ec9104e2 100644
--- a/tools/genesis-snapshot/go.mod
+++ b/tools/genesis-snapshot/go.mod
@@ -10,7 +10,7 @@ require (
github.com/iotaledger/hive.go/lo v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/hive.go/runtime v0.0.0-20231027195901-620bd7470e42
github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000
- github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967
+ github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e
github.com/mr-tron/base58 v1.2.0
github.com/spf13/pflag v1.0.5
golang.org/x/crypto v0.14.0
diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum
index 4f07d0064..00953620a 100644
--- a/tools/genesis-snapshot/go.sum
+++ b/tools/genesis-snapshot/go.sum
@@ -52,8 +52,8 @@ github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd74
github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231027195901-620bd7470e42/go.mod h1:SdK26z8/VhWtxaqCuQrufm80SELgowQPmu9T/8eUQ8g=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42 h1:OlDhgvJ48bZxcvTeebJ1b96xtNnJAddejd2Q4rlH1mU=
github.com/iotaledger/hive.go/stringify v0.0.0-20231027195901-620bd7470e42/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967 h1:qiBW4TiRdzVJshLu7RoWn9ur64SFLCOQ0oFtQWtz4bk=
-github.com/iotaledger/iota.go/v4 v4.0.0-20231031113109-5d7d59311967/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e h1:ZYRC1MHn/ghsqtjIpYGTxLQrh5n5eUmC0/YWnJiTRhk=
+github.com/iotaledger/iota.go/v4 v4.0.0-20231102113728-20b8d01e826e/go.mod h1:jqbLYq4a/FwuiPBqFfkAwwxU8vs3+kReRq2/tyX5qRA=
github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s=
github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk=
github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg=
From cf40c206d35267aef097f341695c8700e63d1ec8 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 2 Nov 2023 14:54:53 +0100
Subject: [PATCH 50/56] Generate a snapshot with current timestamp.
---
tools/genesis-snapshot/presets/presets.go | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tools/genesis-snapshot/presets/presets.go b/tools/genesis-snapshot/presets/presets.go
index 29babde9f..315aebac9 100644
--- a/tools/genesis-snapshot/presets/presets.go
+++ b/tools/genesis-snapshot/presets/presets.go
@@ -244,7 +244,7 @@ var Feature = []options.Option[snapshotcreator.Options]{
iotago.NewV3ProtocolParameters(
iotago.WithNetworkOptions("feature", "rms"),
iotago.WithSupplyOptions(4_600_000_000_000_000, 100, 1, 10, 100, 100, 100),
- iotago.WithTimeProviderOptions(1697631694, 10, 13),
+ iotago.WithTimeProviderOptions(time.Now().Unix(), 10, 13),
iotago.WithLivenessOptions(30, 30, 10, 20, 30),
// increase/decrease threshold = fraction * slotDurationInSeconds * schedulerRate
iotago.WithCongestionControlOptions(500, 500, 500, 800000, 500000, 100000, 1000, 100),
From 763e7681b6790f93fe9b841c553adb9582318e86 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 2 Nov 2023 14:56:13 +0100
Subject: [PATCH 51/56] Fix node shutdown.
---
components/metrics/collector/metric.go | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/components/metrics/collector/metric.go b/components/metrics/collector/metric.go
index 4ea8e0adc..1696b8d80 100644
--- a/components/metrics/collector/metric.go
+++ b/components/metrics/collector/metric.go
@@ -198,7 +198,7 @@ func (m *Metric) schedulePruning(labelValues []string) {
func (m *Metric) shutdown() {
if m.pruningExecutor != nil {
- m.pruningExecutor.Shutdown()
+ m.pruningExecutor.Shutdown(timed.CancelPendingElements)
}
}
From f56c9d1f1c82daa3242183ea23abde1841ccefd6 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 2 Nov 2023 14:57:03 +0100
Subject: [PATCH 52/56] Cleanup run.sh script
---
tools/docker-network/run.sh | 7 +++----
1 file changed, 3 insertions(+), 4 deletions(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index 6cac8241e..a47388ec1 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -1,17 +1,16 @@
#!/bin/bash
-set -e
+
# Create a function to join an array of strings by a given character
function join { local IFS="$1"; shift; echo "$*"; }
# All parameters can be optional now, just make sure we don't have too many
if [[ $# -gt 4 ]] ; then
- echo 'Call with ./run [replicas=1|2|3|...] [monitoring=0|1] [feature=0|1]'
+ echo 'Call with ./run [monitoring=0|1]'
exit 0
fi
-REPLICAS=${1:-1}
-MONITORING=${2:-0}
+MONITORING=${1:-0}
export DOCKER_BUILDKIT=1
export COMPOSE_DOCKER_CLI_BUILD=1
From 5558da729147183b2b8132d55a4eb1efffe7a2f8 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Thu, 2 Nov 2023 15:09:28 +0100
Subject: [PATCH 53/56] ansible-playbook is not available on self-hosted
actions runner
---
.github/workflows/feature-network-deploy.yml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/.github/workflows/feature-network-deploy.yml b/.github/workflows/feature-network-deploy.yml
index 42bce3b5d..812411683 100644
--- a/.github/workflows/feature-network-deploy.yml
+++ b/.github/workflows/feature-network-deploy.yml
@@ -13,7 +13,7 @@ on:
jobs:
deploy:
environment: feature
- runs-on: self-hosted
+ runs-on: ubuntu-latest
env:
DOCKER_BUILDKIT: 1
steps:
From d9c93abf7e78896faa44ec41ec8680b3ada86c08 Mon Sep 17 00:00:00 2001
From: Piotr Macek <4007944+piotrm50@users.noreply.github.com>
Date: Thu, 2 Nov 2023 15:13:12 +0100
Subject: [PATCH 54/56] Fix run.sh after review.
---
tools/docker-network/run.sh | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/tools/docker-network/run.sh b/tools/docker-network/run.sh
index a47388ec1..323a6bdbb 100755
--- a/tools/docker-network/run.sh
+++ b/tools/docker-network/run.sh
@@ -5,8 +5,8 @@
function join { local IFS="$1"; shift; echo "$*"; }
# All parameters can be optional now, just make sure we don't have too many
-if [[ $# -gt 4 ]] ; then
- echo 'Call with ./run [monitoring=0|1]'
+if [[ $# -gt 2 ]] ; then
+ echo 'Call with ./run.sh [monitoring=0|1]'
exit 0
fi
From 6937f94cde18819fbbf2cad358d228e840cca191 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Thu, 2 Nov 2023 15:33:07 +0100
Subject: [PATCH 55/56] Panic when failing to select new committee
---
.../sybilprotectionv1/sybilprotection.go | 11 ++++-------
1 file changed, 4 insertions(+), 7 deletions(-)
diff --git a/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go b/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
index 6389a2608..fd3ff4c70 100644
--- a/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
+++ b/pkg/protocol/sybilprotection/sybilprotectionv1/sybilprotection.go
@@ -282,10 +282,7 @@ func (o *SybilProtection) slotFinalized(slot iotago.SlotIndex) {
epochEndSlot > o.lastCommittedSlot+apiForSlot.ProtocolParameters().MaxCommittableAge() {
newCommittee, err := o.selectNewCommittee(slot)
if err != nil {
- // TODO: should we fail "harder" here?
- o.errHandler(ierrors.Wrap(err, "error while selecting new committee"))
-
- return
+ panic(ierrors.Wrap(err, "error while selecting new committee"))
}
o.events.CommitteeSelected.Trigger(newCommittee, epoch+1)
}
@@ -387,7 +384,7 @@ func (o *SybilProtection) selectNewCommittee(slot iotago.SlotIndex) (*account.Ac
nextEpoch := currentEpoch + 1
candidates, err := o.performanceTracker.EligibleValidatorCandidates(nextEpoch)
if err != nil {
- panic(ierrors.Wrapf(err, "failed to retrieve candidates for epoch %d", nextEpoch))
+ return nil, ierrors.Wrapf(err, "failed to retrieve candidates for epoch %d", nextEpoch)
}
candidateAccounts := make(accounts.AccountsData, 0)
@@ -404,12 +401,12 @@ func (o *SybilProtection) selectNewCommittee(slot iotago.SlotIndex) (*account.Ac
return nil
}); err != nil {
- panic(ierrors.Wrap(err, "failed to iterate through candidates"))
+ return nil, ierrors.Wrap(err, "failed to iterate through candidates")
}
newCommittee, err := o.seatManager.RotateCommittee(nextEpoch, candidateAccounts)
if err != nil {
- panic(ierrors.Wrap(err, "failed to rotate committee"))
+ return nil, ierrors.Wrap(err, "failed to rotate committee")
}
o.performanceTracker.ClearCandidates()
From 8226e8a6536c58e22e387eb4b6c9ab875d715481 Mon Sep 17 00:00:00 2001
From: Andrea V <1577639+karimodm@users.noreply.github.com>
Date: Fri, 3 Nov 2023 10:34:36 +0100
Subject: [PATCH 56/56] Configure default genesis seed and fix fatal error
---
tools/genesis-snapshot/main.go | 14 ++++++--------
1 file changed, 6 insertions(+), 8 deletions(-)
diff --git a/tools/genesis-snapshot/main.go b/tools/genesis-snapshot/main.go
index 63d303a83..05728937e 100644
--- a/tools/genesis-snapshot/main.go
+++ b/tools/genesis-snapshot/main.go
@@ -37,7 +37,7 @@ func main() {
func parseFlags() (opt []options.Option[snapshotcreator.Options], conf string) {
filename := flag.String("filename", "", "the name of the generated snapshot file")
config := flag.String("config", "", "use ready config: devnet, feature, docker")
- genesisSeedStr := flag.String("seed", "", "the genesis seed provided in base58 format.")
+ genesisSeedStr := flag.String("seed", "7R1itJx5hVuo9w9hjg5cwKFmek4HMSoBDgJZN8hKGxih", "the genesis seed provided in base58 format.")
flag.Parse()
opt = []options.Option[snapshotcreator.Options]{}
@@ -45,14 +45,12 @@ func parseFlags() (opt []options.Option[snapshotcreator.Options], conf string) {
opt = append(opt, snapshotcreator.WithFilePath(*filename))
}
- if *genesisSeedStr != "" {
- genesisSeed, err := base58.Decode(*genesisSeedStr)
- if err != nil {
- log.Fatal(ierrors.Errorf("failed to decode base58 seed, using the default one: %w", err))
- }
- keyManager := mock.NewKeyManager(genesisSeed[:], 0)
- opt = append(opt, snapshotcreator.WithGenesisKeyManager(keyManager))
+ genesisSeed, err := base58.Decode(*genesisSeedStr)
+ if err != nil {
+ log.Fatal(ierrors.Wrap(err, "failed to decode base58 seed"))
}
+ keyManager := mock.NewKeyManager(genesisSeed[:], 0)
+ opt = append(opt, snapshotcreator.WithGenesisKeyManager(keyManager))
return opt, *config
}