From 6755fa1c1395d9be2698bf7e443c36410c4ba83f Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Fri, 7 Nov 2025 16:13:03 -0500 Subject: [PATCH 01/12] refactor: separate tx tracker out of tx client --- go.mod | 2 +- go.sum | 4 +- pkg/user/pruning_test.go | 25 ++++--------- pkg/user/tx_client.go | 69 +++++++---------------------------- pkg/user/tx_client_test.go | 10 ++--- pkg/user/v2/tx_client.go | 1 + tools/talis/docker/Dockerfile | 6 +-- 7 files changed, 32 insertions(+), 85 deletions(-) diff --git a/go.mod b/go.mod index b9d8aa2a7c..0850ebf814 100644 --- a/go.mod +++ b/go.mod @@ -277,7 +277,7 @@ require ( replace ( cosmossdk.io/api => github.com/celestiaorg/cosmos-sdk/api v0.7.6 cosmossdk.io/x/upgrade => github.com/celestiaorg/cosmos-sdk/x/upgrade v0.2.0 - github.com/cometbft/cometbft => github.com/celestiaorg/celestia-core v0.39.13 + github.com/cometbft/cometbft => github.com/celestiaorg/celestia-core v1.54.0-tm-v0.38.17.0.20251105163759-5adfb0ae1193 github.com/cosmos/cosmos-sdk => github.com/celestiaorg/cosmos-sdk v0.51.6 github.com/cosmos/ibc-go/v8 => github.com/celestiaorg/ibc-go/v8 v8.7.2 // Use ledger-cosmos-go v0.16.0 because v0.15.0 causes "hidapi: unknown failure" diff --git a/go.sum b/go.sum index 6cbc71f4e0..cf97475805 100644 --- a/go.sum +++ b/go.sum @@ -781,8 +781,8 @@ github.com/bytedance/sonic v1.14.1/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnN github.com/bytedance/sonic/loader v0.3.0 h1:dskwH8edlzNMctoruo8FPTJDF3vLtDT0sXZwvZJyqeA= github.com/bytedance/sonic/loader v0.3.0/go.mod h1:N8A3vUdtUebEY2/VQC0MyhYeKUFosQU6FxH2JmUe6VI= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= -github.com/celestiaorg/celestia-core v0.39.13 h1:WJjQ+Mq5DkDTjNuZYY7jU3V7obbOnGVdDwIOiLTvREE= -github.com/celestiaorg/celestia-core v0.39.13/go.mod h1:uJPRDBfk83X/bJ/MhfBLJj33T6UyhDS3vY2rW8eI1Mc= +github.com/celestiaorg/celestia-core v1.54.0-tm-v0.38.17.0.20251105163759-5adfb0ae1193 h1:AHx8Mog5b398FK0klEQipwnFC7ojLmG16jPubYBjb+M= +github.com/celestiaorg/celestia-core v1.54.0-tm-v0.38.17.0.20251105163759-5adfb0ae1193/go.mod h1:GxCnrVUUJZ3dtpNlKfHw3otb5TqyOdwjXef1/pNqSzA= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35 h1:FREwqZwPvYsodr1AqqEIyW+VsBnwTzJNtC6NFdZX8rs= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35/go.mod h1:SI38xqZZ4ccoAxszUJqsJ/a5rOkzQRijzHQQlLKkyUc= github.com/celestiaorg/cosmos-sdk v0.51.6 h1:sUDSbbg6SjPjWHW7mow4j6i+I8X8nHDmpojnPOeAwYI= diff --git a/pkg/user/pruning_test.go b/pkg/user/pruning_test.go index 7385f561f3..4b50cf54ab 100644 --- a/pkg/user/pruning_test.go +++ b/pkg/user/pruning_test.go @@ -3,14 +3,13 @@ package user import ( "fmt" "testing" - "time" "github.com/stretchr/testify/require" ) func TestPruningInTxTracker(t *testing.T) { txClient := &TxClient{ - txTracker: make(map[string]txInfo), + TxTracker: NewTxTracker(), } numTransactions := 10 @@ -20,31 +19,21 @@ func TestPruningInTxTracker(t *testing.T) { for i := range numTransactions { // 5 transactions will be pruned if i%2 == 0 { - txClient.txTracker["tx"+fmt.Sprint(i)] = txInfo{ - signer: "signer" + fmt.Sprint(i), - sequence: uint64(i), - timestamp: time.Now(). - Add(-10 * time.Minute), - } + txClient.TxTracker.trackTransaction("signer"+fmt.Sprint(i), uint64(i), "tx"+fmt.Sprint(i), []byte(fmt.Sprintf("tx%d", i))) txsToBePruned++ } else { - txClient.txTracker["tx"+fmt.Sprint(i)] = txInfo{ - signer: "signer" + fmt.Sprint(i), - sequence: uint64(i), - timestamp: time.Now(). - Add(-5 * time.Minute), - } + txClient.TxTracker.trackTransaction("signer"+fmt.Sprint(i), uint64(i), "tx"+fmt.Sprint(i), []byte(fmt.Sprintf("tx%d", i))) txsNotReadyToBePruned++ } } - txTrackerBeforePruning := len(txClient.txTracker) + txTrackerBeforePruning := len(txClient.TxTracker.TxQueue) // All transactions were indexed - require.Equal(t, numTransactions, len(txClient.txTracker)) - txClient.pruneTxTracker() + require.Equal(t, numTransactions, len(txClient.TxTracker.TxQueue)) + txClient.TxTracker.pruneTxTracker() // Prunes the transactions that are 10 minutes old // 5 transactions will be pruned require.Equal(t, txsNotReadyToBePruned, txTrackerBeforePruning-txsToBePruned) - require.Equal(t, len(txClient.txTracker), txsNotReadyToBePruned) + require.Equal(t, len(txClient.TxTracker.TxQueue), txsNotReadyToBePruned) } diff --git a/pkg/user/tx_client.go b/pkg/user/tx_client.go index 577529eb30..c7f6d668e4 100644 --- a/pkg/user/tx_client.go +++ b/pkg/user/tx_client.go @@ -58,15 +58,6 @@ var ( type Option func(client *TxClient) -// txInfo is a struct that holds the sequence and the signer of a transaction -// in the local tx pool. -type txInfo struct { - sequence uint64 - signer string - timestamp time.Time - txBytes []byte -} - // TxResponse is a response from the chain after // a transaction has been submitted. type TxResponse struct { @@ -230,7 +221,7 @@ type TxClient struct { defaultAddress sdktypes.AccAddress // txTracker maps the tx hash to the Sequence and signer of the transaction // that was submitted to the chain - txTracker map[string]txInfo + TxTracker *txTracker gasEstimationClient gasestimation.GasEstimatorClient // txQueue manages parallel transaction submission when enabled txQueue *txQueue @@ -265,7 +256,7 @@ func NewTxClient( pollTime: DefaultPollTime, defaultAccount: records[0].Name, defaultAddress: addr, - txTracker: make(map[string]txInfo), + TxTracker: NewTxTracker(), cdc: cdc, gasEstimationClient: gasestimation.NewGasEstimatorClient(conn), } @@ -462,7 +453,7 @@ func (client *TxClient) BroadcastTx(ctx context.Context, msgs []sdktypes.Msg, op // prune transactions that are older than 10 minutes // pruning has to be done in broadcast, since users // might not always call ConfirmTx(). - client.pruneTxTracker() + client.TxTracker.pruneTxTracker() account, err := client.getAccountNameFromMsgs(msgs) if err != nil { @@ -583,7 +574,8 @@ func (client *TxClient) submitToSingleConnection(ctx context.Context, txBytes [] } // Save the sequence, signer and txBytes of the in the local txTracker // before the sequence is incremented - client.trackTransaction(signer, resp.TxHash, txBytes) + sequence := client.signer.Account(signer).Sequence() + client.TxTracker.trackTransaction(signer, sequence, resp.TxHash, txBytes) // Increment sequence after successful submission if err := client.signer.IncrementSequence(signer); err != nil { @@ -720,7 +712,8 @@ func (client *TxClient) submitToMultipleConnections(ctx context.Context, txBytes // Return first successful response, if any if resp, ok := <-respCh; ok && resp != nil { - client.trackTransaction(signer, resp.TxHash, txBytes) + sequence := client.signer.Account(signer).Sequence() + client.TxTracker.trackTransaction(signer, sequence, resp.TxHash, txBytes) if err := client.signer.IncrementSequence(signer); err != nil { return nil, fmt.Errorf("increment sequencing: %w", err) @@ -736,15 +729,6 @@ func (client *TxClient) submitToMultipleConnections(ctx context.Context, txBytes return nil, errors.Join(errs...) } -// pruneTxTracker removes transactions from the local tx tracker that are older than 10 minutes -func (client *TxClient) pruneTxTracker() { - for hash, txInfo := range client.txTracker { - if time.Since(txInfo.timestamp) >= txTrackerPruningInterval { - delete(client.txTracker, hash) - } - } -} - // ConfirmTx periodically pings the provided node for the commitment of a transaction by its // hash. It will continually loop until the context is cancelled, the tx is found or an error // is encountered. @@ -780,15 +764,15 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon )) if resp.ExecutionCode != abci.CodeTypeOK { span.RecordError(fmt.Errorf("txclient/ConfirmTx: execution error: %s", resp.Error)) - client.deleteFromTxTracker(txHash) + client.TxTracker.deleteFromTxTracker(txHash) return nil, client.buildExecutionError(txHash, resp) } span.AddEvent("txclient/ConfirmTx: transaction confirmed successfully") - client.deleteFromTxTracker(txHash) + client.TxTracker.deleteFromTxTracker(txHash) return client.buildTxResponse(txHash, resp), nil case core.TxStatusEvicted: - _, _, txBytes, exists := client.GetTxFromTxTracker(txHash) + _, _, txBytes, exists := client.TxTracker.GetTxFromTxTracker(txHash) if !exists { return nil, fmt.Errorf("tx: %s not found in txTracker; likely failed during broadcast", txHash) } @@ -819,7 +803,7 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon span.AddEvent("txclient/ConfirmTx: transaction resubmitted successfully after eviction") case core.TxStatusRejected: span.RecordError(fmt.Errorf("txclient/ConfirmTx: transaction rejected: %s", resp.Error)) - sequence, signer, _, exists := client.GetTxFromTxTracker(txHash) + sequence, signer, _, exists := client.TxTracker.GetTxFromTxTracker(txHash) if !exists { return nil, fmt.Errorf("tx: %s not found in tx client txTracker; likely failed during broadcast", txHash) } @@ -828,11 +812,11 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon if err := client.signer.SetSequence(signer, sequence); err != nil { return nil, fmt.Errorf("setting sequence: %w", err) } - client.deleteFromTxTracker(txHash) + client.TxTracker.deleteFromTxTracker(txHash) return nil, fmt.Errorf("tx with hash %s was rejected by the node with execution code: %d and log: %s", txHash, resp.ExecutionCode, resp.Error) default: span.RecordError(fmt.Errorf("txclient/ConfirmTx: unknown tx status for tx: %s", txHash)) - client.deleteFromTxTracker(txHash) + client.TxTracker.deleteFromTxTracker(txHash) if ctx.Err() != nil { return nil, ctx.Err() } @@ -861,13 +845,6 @@ func extractSequenceError(fullError string) string { return s } -// deleteFromTxTracker safely deletes a transaction from the local tx tracker. -func (client *TxClient) deleteFromTxTracker(txHash string) { - client.mtx.Lock() - defer client.mtx.Unlock() - delete(client.txTracker, txHash) -} - // EstimateGasPriceAndUsage returns the estimated gas price based on the provided priority, // and also the gas limit/used for the provided transaction. // The gas limit is calculated by simulating the transaction and then calculating the amount of gas that was consumed during execution. @@ -1045,26 +1022,6 @@ func (client *TxClient) getAccountNameFromMsgs(msgs []sdktypes.Msg) (string, err return record.Name, nil } -// trackTransaction tracks a transaction without acquiring the mutex. -// This should only be called when the caller already holds the mutex. -func (client *TxClient) trackTransaction(signer, txHash string, txBytes []byte) { - sequence := client.signer.Account(signer).Sequence() - client.txTracker[txHash] = txInfo{ - sequence: sequence, - signer: signer, - timestamp: time.Now(), - txBytes: txBytes, - } -} - -// GetTxFromTxTracker gets transaction info from the tx client's local tx tracker by its hash -func (client *TxClient) GetTxFromTxTracker(hash string) (sequence uint64, signer string, txBytes []byte, exists bool) { - client.mtx.Lock() - defer client.mtx.Unlock() - txInfo, exists := client.txTracker[hash] - return txInfo.sequence, txInfo.signer, txInfo.txBytes, exists -} - // Signer exposes the tx clients underlying signer func (client *TxClient) Signer() *Signer { return client.signer diff --git a/pkg/user/tx_client_test.go b/pkg/user/tx_client_test.go index 192cb221eb..c56d3a985d 100644 --- a/pkg/user/tx_client_test.go +++ b/pkg/user/tx_client_test.go @@ -342,7 +342,7 @@ func TestRejections(t *testing.T) { seqAfterConfirmation := sender.Sequence() require.Equal(t, seqBeforeSubmission+1, seqAfterConfirmation) // Was removed from the tx tracker - _, _, _, exists := txClient.GetTxFromTxTracker(resp.TxHash) + _, _, _, exists := txClient.TxTracker.GetTxFromTxTracker(resp.TxHash) require.False(t, exists) } @@ -393,7 +393,7 @@ func TestEvictions(t *testing.T) { require.NoError(t, err) require.Equal(t, res.Code, abci.CodeTypeOK) // They should be removed from the tx tracker after confirmation - _, _, _, exists := txClient.GetTxFromTxTracker(resp.TxHash) + _, _, _, exists := txClient.TxTracker.GetTxFromTxTracker(resp.TxHash) require.False(t, exists) } @@ -565,13 +565,13 @@ func (suite *TxClientTestSuite) queryCurrentBalance(t *testing.T) int64 { } func wasRemovedFromTxTracker(txHash string, txClient *user.TxClient) bool { - seq, signer, txBytes, exists := txClient.GetTxFromTxTracker(txHash) + seq, signer, txBytes, exists := txClient.TxTracker.GetTxFromTxTracker(txHash) return !exists && seq == 0 && signer == "" && len(txBytes) == 0 } // assertTxInTxTracker verifies that a tx was indexed in the tx tracker and that the sequence increases by one after broadcast. func assertTxInTxTracker(t *testing.T, txClient *user.TxClient, txHash, expectedSigner string, seqBeforeBroadcast uint64) { - seqFromTxTracker, signer, txBytes, exists := txClient.GetTxFromTxTracker(txHash) + seqFromTxTracker, signer, txBytes, exists := txClient.TxTracker.GetTxFromTxTracker(txHash) require.True(t, exists) require.Equal(t, expectedSigner, signer) seqAfterBroadcast := txClient.Signer().Account(expectedSigner).Sequence() @@ -827,7 +827,7 @@ func (suite *TxClientTestSuite) TestSequenceIncrementOnlyOnceInMultiConnBroadcas require.Equal(t, seqBefore+1, seqAfter, "Sequence should be incremented by exactly 1, not by number of connections") // Verify the transaction is tracked - trackedSeq, trackedSigner, _, exists := multiConnClient.GetTxFromTxTracker(resp.TxHash) + trackedSeq, trackedSigner, _, exists := multiConnClient.TxTracker.GetTxFromTxTracker(resp.TxHash) require.True(t, exists, "Transaction should be in tracker") require.Equal(t, seqBefore, trackedSeq, "Tracked sequence should be the sequence before increment") require.Equal(t, multiConnClient.DefaultAccountName(), trackedSigner, "Tracked signer should match") diff --git a/pkg/user/v2/tx_client.go b/pkg/user/v2/tx_client.go index 4cca1021bc..931e7128a1 100644 --- a/pkg/user/v2/tx_client.go +++ b/pkg/user/v2/tx_client.go @@ -18,6 +18,7 @@ import ( type TxClient struct { // Embed the underlying client to automatically delegate all methods *user.TxClient + TxQueue map[string]user.TxInfo } // NewTxClient creates a new v2 TxClient by wrapping the original NewTxClient function. diff --git a/tools/talis/docker/Dockerfile b/tools/talis/docker/Dockerfile index 5564c69589..6adf7be151 100644 --- a/tools/talis/docker/Dockerfile +++ b/tools/talis/docker/Dockerfile @@ -12,6 +12,6 @@ RUN go mod download COPY . . -RUN GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/txsim ./test/cmd/txsim \ - && GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 GOOS=linux go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/celestia-appd ./cmd/celestia-appd \ - && GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/latency-monitor ./tools/latency-monitor +# RUN GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/txsim ./test/cmd/txsim \ +RUN GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 GOOS=linux go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/celestia-appd ./cmd/celestia-appd +# && GOOS=${GOOS} GOARCH=${GOARCH} CGO_ENABLED=0 go build -tags="ledger" -ldflags="${LDFLAGS}" -o /out/latency-monitor ./tools/latency-monitor From 76a8e267d75bbe40fe73205ce8c5aa1d8211fdaa Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Mon, 17 Nov 2025 22:15:36 -0500 Subject: [PATCH 02/12] feat: sequential submission wip --- pkg/user/tx_client.go | 54 ++--- pkg/user/v2/sequential_queue.go | 350 ++++++++++++++++++++++++++++++++ pkg/user/v2/tx_client.go | 178 +++++++++++++++- test/docker-e2e/go.mod | 4 + 4 files changed, 553 insertions(+), 33 deletions(-) create mode 100644 pkg/user/v2/sequential_queue.go diff --git a/pkg/user/tx_client.go b/pkg/user/tx_client.go index c7f6d668e4..97c3e0d343 100644 --- a/pkg/user/tx_client.go +++ b/pkg/user/tx_client.go @@ -155,9 +155,9 @@ func WithDefaultAccount(name string) Option { c.defaultAddress = addr // Update worker 0's account if tx queue already exists - if c.txQueue != nil && len(c.txQueue.workers) > 0 { - c.txQueue.workers[0].accountName = name - c.txQueue.workers[0].address = addr.String() + if c.parallelTxQueue != nil && len(c.parallelTxQueue.workers) > 0 { + c.parallelTxQueue.workers[0].accountName = name + c.parallelTxQueue.workers[0].address = addr.String() } } } @@ -190,7 +190,7 @@ func WithTxWorkers(numWorkers int) Option { } return func(c *TxClient) { - c.txQueue = newTxQueue(c, numWorkers) + c.parallelTxQueue = newTxQueue(c, numWorkers) } } @@ -198,8 +198,8 @@ func WithTxWorkers(numWorkers int) Option { // Default is 100 if not specified. func WithParallelQueueSize(size int) Option { return func(c *TxClient) { - if c.txQueue != nil { - c.txQueue.jobQueue = make(chan *SubmissionJob, size) + if c.parallelTxQueue != nil { + c.parallelTxQueue.jobQueue = make(chan *SubmissionJob, size) } } } @@ -221,10 +221,10 @@ type TxClient struct { defaultAddress sdktypes.AccAddress // txTracker maps the tx hash to the Sequence and signer of the transaction // that was submitted to the chain - TxTracker *txTracker + TxTracker *TxTracker gasEstimationClient gasestimation.GasEstimatorClient - // txQueue manages parallel transaction submission when enabled - txQueue *txQueue + // parallelTxQueue manages parallel transaction submission when enabled + parallelTxQueue *txQueue } // NewTxClient returns a new TxClient @@ -267,8 +267,8 @@ func NewTxClient( // Always create a tx queue with at least 1 worker (the default account) // unless already configured by WithTxWorkers option - if txClient.txQueue == nil { - txClient.txQueue = newTxQueue(txClient, 1) + if txClient.parallelTxQueue == nil { + txClient.parallelTxQueue = newTxQueue(txClient, 1) } return txClient, nil @@ -326,7 +326,7 @@ func SetupTxClient( return nil, err } - if err := txClient.txQueue.start(ctx); err != nil { + if err := txClient.parallelTxQueue.start(ctx); err != nil { return nil, fmt.Errorf("failed to start tx queue: %w", err) } @@ -362,12 +362,12 @@ func (client *TxClient) SubmitPayForBlobToQueue(ctx context.Context, blobs []*sh // to the provided channel when the transaction is confirmed. The caller is responsible for creating and // closing the result channel. func (client *TxClient) QueueBlob(ctx context.Context, resultC chan SubmissionResult, blobs []*share.Blob, opts ...TxOption) { - if client.txQueue == nil { + if client.parallelTxQueue == nil { resultC <- SubmissionResult{Error: errTxQueueNotConfigured} return } - if !client.txQueue.isStarted() { + if !client.parallelTxQueue.isStarted() { resultC <- SubmissionResult{Error: errTxQueueNotStarted} return } @@ -379,7 +379,7 @@ func (client *TxClient) QueueBlob(ctx context.Context, resultC chan SubmissionRe ResultsC: resultC, } - client.txQueue.submitJob(job) + client.parallelTxQueue.submitJob(job) } // SubmitPayForBlobWithAccount forms a transaction from the provided blobs, signs it with the provided account, and submits it to the chain. @@ -1030,51 +1030,51 @@ func (client *TxClient) Signer() *Signer { // StartTxQueueForTest starts the tx queue for testing purposes. // This function is only intended for use in tests. func (client *TxClient) StartTxQueueForTest(ctx context.Context) error { - if client.txQueue == nil { + if client.parallelTxQueue == nil { return nil } - return client.txQueue.start(ctx) + return client.parallelTxQueue.start(ctx) } // StopTxQueueForTest stops the tx queue for testing purposes. // This function is only intended for use in tests. func (client *TxClient) StopTxQueueForTest() { - if client.txQueue != nil { - client.txQueue.stop() + if client.parallelTxQueue != nil { + client.parallelTxQueue.stop() } } // IsTxQueueStartedForTest returns whether the tx queue is started, for testing purposes. // This function is only intended for use in tests. func (client *TxClient) IsTxQueueStartedForTest() bool { - if client.txQueue == nil { + if client.parallelTxQueue == nil { return false } - return client.txQueue.isStarted() + return client.parallelTxQueue.isStarted() } // TxQueueWorkerCount returns the number of workers in the tx queue func (client *TxClient) TxQueueWorkerCount() int { - if client.txQueue == nil { + if client.parallelTxQueue == nil { return 0 } - return len(client.txQueue.workers) + return len(client.parallelTxQueue.workers) } // TxQueueWorkerAddress returns the address for the worker at the given index func (client *TxClient) TxQueueWorkerAddress(index int) string { - if client.txQueue == nil || index < 0 || index >= len(client.txQueue.workers) { + if client.parallelTxQueue == nil || index < 0 || index >= len(client.parallelTxQueue.workers) { return "" } - return client.txQueue.workers[index].address + return client.parallelTxQueue.workers[index].address } // TxQueueWorkerAccountName returns the account name for the worker at the given index func (client *TxClient) TxQueueWorkerAccountName(index int) string { - if client.txQueue == nil || index < 0 || index >= len(client.txQueue.workers) { + if client.parallelTxQueue == nil || index < 0 || index >= len(client.parallelTxQueue.workers) { return "" } - return client.txQueue.workers[index].accountName + return client.parallelTxQueue.workers[index].accountName } // QueryMinimumGasPrice queries both the nodes local and network wide diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go new file mode 100644 index 0000000000..662c893caa --- /dev/null +++ b/pkg/user/v2/sequential_queue.go @@ -0,0 +1,350 @@ +package v2 + +import ( + "context" + "fmt" + "sync" + "time" + + "github.com/celestiaorg/celestia-app/v6/app/grpc/tx" + "github.com/celestiaorg/celestia-app/v6/pkg/user" + "github.com/celestiaorg/go-square/v3/share" + abci "github.com/cometbft/cometbft/abci/types" + "github.com/cometbft/cometbft/rpc/core" + sdktypes "github.com/cosmos/cosmos-sdk/types" +) + +// SequentialSubmissionJob represents a transaction submission task +type SequentialSubmissionJob struct { + Blobs []*share.Blob + Options []user.TxOption + Ctx context.Context + ResultsC chan SequentialSubmissionResult +} + +// SequentialSubmissionResult contains the result of a transaction submission +type SequentialSubmissionResult struct { + TxResponse *sdktypes.TxResponse + Error error +} + +// sequentialQueue manages single-threaded transaction submission +type sequentialQueue struct { + wg sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + client *TxClient + accountName string + pollTime time.Duration + + // Transaction processing + mu sync.RWMutex + queuedTxs map[uint64]*queuedTx // Transactions waiting to be broadcast + pendingTxs map[uint64]*pendingTx // Transactions broadcast, waiting confirmation +} + +// queuedTx represents a transaction waiting to be broadcast +type queuedTx struct { + sequence uint64 + blobs []*share.Blob + options []user.TxOption + resultsC chan SequentialSubmissionResult + needsResign bool // True if needs resigning (e.g., after prior rejection) +} + +// pendingTx tracks a transaction that has been broadcast +type pendingTx struct { + sequence uint64 + status string + txHash string + txBytes []byte // Stored for resubmission without resigning + blobs []*share.Blob + options []user.TxOption + resultsC chan SequentialSubmissionResult + submittedAt time.Time + attempts int +} + +const ( + defaultSequentialQueueSize = 100 + maxResubmitAttempts = 5 +) + +func newSequentialQueue(client *TxClient, accountName string, pollTime time.Duration) *sequentialQueue { + if pollTime == 0 { + pollTime = user.DefaultPollTime + } + + return &sequentialQueue{ + client: client, + accountName: accountName, + pollTime: pollTime, + queuedTxs: make(map[uint64]*queuedTx), + pendingTxs: make(map[uint64]*pendingTx), + } +} + +// start initiates the sequential queue processor +func (q *sequentialQueue) start(ctx context.Context) error { + q.ctx, q.cancel = context.WithCancel(ctx) + + // Start the processing loop (broadcasts queued txs in order) + q.wg.Add(1) + go func() { + defer q.wg.Done() + q.processLoop() + }() + + // Start the monitoring loop (confirms pending txs) + q.wg.Add(1) + go func() { + defer q.wg.Done() + q.monitorLoop() + }() + + return nil +} + +// stop shuts down the sequential queue +func (q *sequentialQueue) stop() { + if q.ctx == nil { + return + } + + q.cancel() + q.wg.Wait() + q.ctx, q.cancel = nil, nil +} + +// isStarted returns whether the queue is started +func (q *sequentialQueue) isStarted() bool { + return q.ctx != nil && q.cancel != nil +} + +// submitJob submits a job to the sequential queue (directly, no channel) +func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { + if !q.isStarted() { + job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("sequential queue not started")} + return + } + + // Check if job context is cancelled + if job.Ctx.Err() != nil { + job.ResultsC <- SequentialSubmissionResult{Error: job.Ctx.Err()} + return + } + + // Get current sequence + acc := q.client.Account(q.accountName) + if acc == nil { + job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("account %s not found", q.accountName)} + return + } + sequence := acc.Sequence() + + // Add to queued transactions + q.mu.Lock() + q.queuedTxs[sequence] = &queuedTx{ + sequence: sequence, + blobs: job.Blobs, + options: job.Options, + resultsC: job.ResultsC, + needsResign: false, + } + q.mu.Unlock() + + // Increment sequence for next job + if err := q.client.Signer().IncrementSequence(q.accountName); err != nil { + job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("error incrementing sequence: %w", err)} + } +} + +// processLoop is a process continuely submitting txs from Q +func (q *sequentialQueue) processLoop() { + // TODO: Maybe this should be like 1 second? so tx client can actually be the one to control submission cadence rather than user? + ticker := time.NewTicker(q.pollTime / 2) // ARBITRARY DELAY + defer ticker.Stop() + + for { + select { + case <-ticker.C: + q.submitTxsInQueue() + case <-q.ctx.Done(): + return + } + } +} + +// submitTxsInQueue broadcasts the next transaction in sequence order +// this needs to change to only lock per transaction submission rather than the whole queue +func (q *sequentialQueue) submitTxsInQueue() { + q.mu.Lock() + defer q.mu.Unlock() + + // Get the first transaction from the queue + // keep one second delay between submissions + for _, queued := range q.queuedTxs { + + // Broadcast the transaction + var resp *sdktypes.TxResponse + // TODO: figure out how to set tx bytes + var txBytes []byte + var err error + + resp, err = q.client.BroadcastPayForBlobWithoutRetry(q.ctx, q.accountName, queued.blobs, queued.options...) + if err != nil { + // check if the error is sequence mismatch and if tx should be resigned. (never resign evicted txs) + queued.resultsC <- SequentialSubmissionResult{Error: fmt.Errorf("broadcast failed: %w", err)} + delete(q.queuedTxs, queued.sequence) + return + } + + // Move from queued to pending + q.pendingTxs[queued.sequence] = &pendingTx{ + sequence: queued.sequence, + txHash: resp.TxHash, + txBytes: txBytes, + blobs: queued.blobs, + options: queued.options, + resultsC: queued.resultsC, + submittedAt: time.Now(), + attempts: 1, + } + delete(q.queuedTxs, queued.sequence) + + time.Sleep(time.Second) // this is submission delay but could be done differently + } +} + +// monitorLoop periodically checks pending transactions +func (q *sequentialQueue) monitorLoop() { + ticker := time.NewTicker(q.pollTime) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + q.checkPendingTransactions() + case <-q.ctx.Done(): + return + } + } +} + +// checkPendingTransactions checks status of all pending transactions +func (q *sequentialQueue) checkPendingTransactions() { + // todo; possibly error prone + txClient := tx.NewTxClient(q.client.GetConn()) + + // Check all pending transactions + for _, pending := range q.pendingTxs { + seq := pending.sequence + + // Check transaction status + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: pending.txHash}) + if err != nil { + // Network error, skip this check + continue + } + + switch statusResp.Status { + case core.TxStatusPending: + // Still pending, continue monitoring + continue + + case core.TxStatusCommitted: + // Check execution code + if statusResp.ExecutionCode != abci.CodeTypeOK { + // Execution failed - treat like rejection, roll back signer sequence + // should prolly use execution error type same as in tx client + pending.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("tx execution failed with code %d: %s", statusResp.ExecutionCode, statusResp.Error), + } + continue + } + // use populate tx response from tx client TODO: reuse the function that populates + pending.resultsC <- SequentialSubmissionResult{ + TxResponse: &sdktypes.TxResponse{ + Height: statusResp.Height, + TxHash: pending.txHash, + Code: statusResp.ExecutionCode, + Codespace: statusResp.Codespace, + GasWanted: statusResp.GasWanted, + GasUsed: statusResp.GasUsed, + Signers: statusResp.Signers, + }, + Error: nil, + } + + case core.TxStatusEvicted: + // Transaction evicted - put back in queue for resubmission WITHOUT resigning + // Add back to queuedTxs with same sequence + q.mu.Lock() + q.queuedTxs[seq] = &queuedTx{ + sequence: seq, + blobs: pending.blobs, + options: pending.options, + resultsC: pending.resultsC, + needsResign: false, // Keep existing tx bytes, no resigning + } + delete(q.pendingTxs, seq) + q.mu.Unlock() + case core.TxStatusRejected: + // tx rejected - roll back the signer sequence + // check if the tx before was also rejected, if so, no need to roll back the signer sequence + // because it would have been rolled back already + prevTx, exists := q.pendingTxs[seq-1] // TODO: we need accessors with mutexes for this + if !exists || prevTx.status != core.TxStatusRejected { + // no previous tx, so we can roll back the signer sequence + q.client.TxClient.Signer().SetSequence(q.accountName, seq) + continue + } + pending.status = core.TxStatusRejected + // TODO: how do i do this? maybe i should keep track of tx hashes and just tx status them + pending.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("tx rejected: %s", statusResp.Error), + } + // TODO: remove the bytes from the tx tracker and just keep the sequence, hash and status + // this is to save storage space + + // Mark subsequent transactions for resigning + q.markForResign(seq) + } + } + +} + +// markForResign marks all subsequent queued transactions for resigning +// idk if this is the best way, we shall see +func (q *sequentialQueue) markForResign(fromSeq uint64) { + for seq, queued := range q.queuedTxs { + if seq > fromSeq { + queued.needsResign = true + } + } +} + +// purgeOldTxs removes rejected/confirmed transactions that are older than 10 minutes (possible refactor this to height since time is the worst in blockchain env +// but here it feels not too bad since its only for cleanup) +// TODO: find out how much storage is required for this to work. +// we could possibly remove everything but sequence, status and hash for confirmed/rejected txs for lightweight storage. +func (q *sequentialQueue) purgeOldTxs() { + q.mu.Lock() + defer q.mu.Unlock() + + // remove all transactions that are older than 10 minutes + for seq, pending := range q.pendingTxs { + if time.Since(pending.submittedAt) > 10*time.Minute { + delete(q.pendingTxs, seq) + delete(q.queuedTxs, seq) + } + } +} + +// GetQueueSize returns the number of pending transactions +func (q *sequentialQueue) GetQueueSize() int { + q.mu.RLock() + defer q.mu.RUnlock() + return len(q.queuedTxs) + len(q.pendingTxs) +} diff --git a/pkg/user/v2/tx_client.go b/pkg/user/v2/tx_client.go index 931e7128a1..52336cc7aa 100644 --- a/pkg/user/v2/tx_client.go +++ b/pkg/user/v2/tx_client.go @@ -2,6 +2,9 @@ package v2 import ( "context" + "fmt" + "sync" + "time" "github.com/celestiaorg/celestia-app/v6/app/encoding" "github.com/celestiaorg/celestia-app/v6/pkg/user" @@ -18,7 +21,10 @@ import ( type TxClient struct { // Embed the underlying client to automatically delegate all methods *user.TxClient - TxQueue map[string]user.TxInfo + + // Sequential queues per account + sequentialQueues map[string]*sequentialQueue + queueMu sync.RWMutex } // NewTxClient creates a new v2 TxClient by wrapping the original NewTxClient function. @@ -33,8 +39,14 @@ func NewTxClient( if err != nil { return nil, err } - - return &TxClient{TxClient: v1Client}, nil + v2Client := &TxClient{ + TxClient: v1Client, + sequentialQueues: make(map[string]*sequentialQueue), + } + if err := v2Client.StartSequentialQueue(context.Background(), v1Client.DefaultAccountName()); err != nil { + return nil, err + } + return v2Client, nil } // SetupTxClient creates and initializes a new v2 TxClient by wrapping the original setupTxClient method. @@ -49,13 +61,22 @@ func SetupTxClient( if err != nil { return nil, err } - - return &TxClient{TxClient: v1Client}, nil + v2Client := &TxClient{ + TxClient: v1Client, + sequentialQueues: make(map[string]*sequentialQueue), + } + if err := v2Client.StartSequentialQueue(ctx, v1Client.DefaultAccountName()); err != nil { + return nil, err + } + return v2Client, nil } // Wrapv1TxClient wraps a v1 TxClient and returns a v2 TxClient. func Wrapv1TxClient(v1Client *user.TxClient) *TxClient { - return &TxClient{TxClient: v1Client} + return &TxClient{ + TxClient: v1Client, + sequentialQueues: make(map[string]*sequentialQueue), + } } func (c *TxClient) buildSDKTxResponse(legacyResp *user.TxResponse) *sdktypes.TxResponse { @@ -121,3 +142,148 @@ func (c *TxClient) ConfirmTx(ctx context.Context, txHash string) (*sdktypes.TxRe return c.buildSDKTxResponse(legacyResp), nil } + +// StartSequentialQueue starts a sequential submission queue for the given account. +func (c *TxClient) StartSequentialQueue(ctx context.Context, accountName string) error { + return c.StartSequentialQueueWithPollTime(ctx, accountName, user.DefaultPollTime) +} + +// StartSequentialQueueWithPollTime starts a sequential queue with a custom poll time. +func (c *TxClient) StartSequentialQueueWithPollTime(ctx context.Context, accountName string, pollTime time.Duration) error { + c.queueMu.Lock() + defer c.queueMu.Unlock() + + if _, exists := c.sequentialQueues[accountName]; exists { + return fmt.Errorf("sequential queue already running for account %s", accountName) + } + + queue := newSequentialQueue(c, accountName, pollTime) + if err := queue.start(ctx); err != nil { + return fmt.Errorf("failed to start sequential queue: %w", err) + } + + c.sequentialQueues[accountName] = queue + return nil +} + +// StopSequentialQueue stops the sequential queue for the given account. +func (c *TxClient) StopSequentialQueue(accountName string) { + c.queueMu.Lock() + defer c.queueMu.Unlock() + + if queue, exists := c.sequentialQueues[accountName]; exists { + queue.stop() + delete(c.sequentialQueues, accountName) + } +} + +// StopAllSequentialQueues stops all running sequential queues. +func (c *TxClient) StopAllSequentialQueues() { + c.queueMu.Lock() + defer c.queueMu.Unlock() + + for accountName, queue := range c.sequentialQueues { + queue.stop() + delete(c.sequentialQueues, accountName) + } +} + +// SubmitPFBToSequentialQueue submits blobs using the sequential queue for the default account. +func (c *TxClient) SubmitPFBToSequentialQueue(ctx context.Context, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, error) { + return c.SubmitPFBToSequentialQueueWithAccount(ctx, c.DefaultAccountName(), blobs, opts...) +} + +// SubmitPFBToSequentialQueueWithAccount submits blobs using the sequential queue for the specified account. +func (c *TxClient) SubmitPFBToSequentialQueueWithAccount(ctx context.Context, accountName string, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, error) { + c.queueMu.RLock() + queue, exists := c.sequentialQueues[accountName] + c.queueMu.RUnlock() + + if !exists { + return nil, fmt.Errorf("sequential queue not started for account %s", accountName) + } + + resultsC := make(chan SequentialSubmissionResult, 1) + defer close(resultsC) + + job := &SequentialSubmissionJob{ + Blobs: blobs, + Options: opts, + Ctx: ctx, + ResultsC: resultsC, + } + + queue.submitJob(job) + + // Wait for result + select { + case result := <-resultsC: + if result.Error != nil { + return nil, result.Error + } + return result.TxResponse, nil + case <-ctx.Done(): + return nil, ctx.Err() + } +} + +// GetSequentialQueueSize returns the number of pending transactions in the queue for the given account. +func (c *TxClient) GetSequentialQueueSize(accountName string) (int, error) { + c.queueMu.RLock() + queue, exists := c.sequentialQueues[accountName] + c.queueMu.RUnlock() + + if !exists { + return 0, fmt.Errorf("sequential queue not started for account %s", accountName) + } + + return queue.GetQueueSize(), nil +} + +// BroadcastPayForBlobWithoutRetry broadcasts a PayForBlob transaction without automatic retry logic. +// TTODO: this will do for now, but i should refactor tx client functions without breaking the API to make it more usable +func (c *TxClient) BroadcastPayForBlobWithoutRetry(ctx context.Context, accountName string, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, error) { + // Lock the client for the duration of build+sign+send + c.TxClient.mtx.Lock() + defer c.TxClient.mtx.Unlock() + + // Check account is loaded + if err := c.TxClient.CheckAccountLoadedExported(ctx, accountName); err != nil { + return nil, err + } + + acc := c.Signer().Account(accountName) + if acc == nil { + return nil, fmt.Errorf("account %s not found", accountName) + } + + // Build the transaction bytes + txBytes, _, err := c.Signer().CreatePayForBlobs(accountName, blobs, opts...) + if err != nil { + return nil, err + } + + // Get the sequence before sending + sequence := c.Signer().Account(accountName).Sequence() + + // Send directly without retry logic + conn := c.GetConn() + if conn == nil { + return nil, fmt.Errorf("no connection available") + } + + resp, err := c.TxClient.SendTxToConnectionExported(ctx, conn, txBytes) + if err != nil { + return nil, err + } + + // Track the transaction + c.TxClient.GetTxTrackerExported().TrackTransaction(accountName, sequence, resp.TxHash, txBytes) + + // Increment sequence after successful broadcast + if err := c.Signer().IncrementSequence(accountName); err != nil { + return nil, fmt.Errorf("error incrementing sequence: %w", err) + } + + return resp, nil +} diff --git a/test/docker-e2e/go.mod b/test/docker-e2e/go.mod index f85ad869bd..60e2c7a0ef 100644 --- a/test/docker-e2e/go.mod +++ b/test/docker-e2e/go.mod @@ -275,7 +275,11 @@ replace ( cosmossdk.io/api => github.com/celestiaorg/cosmos-sdk/api v0.7.6 cosmossdk.io/x/upgrade => github.com/celestiaorg/cosmos-sdk/x/upgrade v0.2.0 github.com/celestiaorg/celestia-app/v6 => ../.. +<<<<<<< Updated upstream github.com/cometbft/cometbft => github.com/celestiaorg/celestia-core v0.39.13 +======= + github.com/cometbft/cometbft => /Users/ninabarbakadze/Desktop/celestia/celestia-core +>>>>>>> Stashed changes github.com/cosmos/cosmos-sdk => github.com/celestiaorg/cosmos-sdk v0.51.6 github.com/cosmos/ibc-go/v8 => github.com/celestiaorg/ibc-go/v8 v8.7.2 // Use ledger-cosmos-go v0.16.0 because v0.15.0 causes "hidapi: unknown failure" From db2eca12c0d03452d646aa83f9fb0783f419b165 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Fri, 21 Nov 2025 17:56:56 -0500 Subject: [PATCH 03/12] refactor: change to single queue; it works --- pkg/user/tx_tracker.go | 104 ++++++ pkg/user/v2/sequential_queue.go | 542 +++++++++++++++++----------- pkg/user/v2/tx_client.go | 95 +++-- scripts/single-node-all-upgrades.sh | 2 +- tools/spam_txclient/main.go | 114 +++--- 5 files changed, 565 insertions(+), 292 deletions(-) create mode 100644 pkg/user/tx_tracker.go diff --git a/pkg/user/tx_tracker.go b/pkg/user/tx_tracker.go new file mode 100644 index 0000000000..af8e380657 --- /dev/null +++ b/pkg/user/tx_tracker.go @@ -0,0 +1,104 @@ +package user + +import ( + "sync" + "time" +) + +// TxTracker tracks transaction metadata for submitted transactions. +// It stores the hash, sequence, signer, and raw bytes for each transaction. +// This is used for: +// - Resubmitting evicted transactions with the same bytes +// - Rolling back sequence numbers on rejection +// - Pruning old transactions to prevent memory leaks +type TxTracker struct { + mu sync.RWMutex + TxQueue map[string]*txInfo +} + +// txInfo contains metadata about a submitted transaction +type txInfo struct { + signer string + sequence uint64 + txBytes []byte + timestamp time.Time +} + +// NewTxTracker creates a new TxTracker instance +func NewTxTracker() *TxTracker { + return &TxTracker{ + TxQueue: make(map[string]*txInfo), + } +} + +// trackTransaction adds a transaction to the tracker +func (t *TxTracker) trackTransaction(signer string, sequence uint64, txHash string, txBytes []byte) { + t.mu.Lock() + defer t.mu.Unlock() + + t.TxQueue[txHash] = &txInfo{ + signer: signer, + sequence: sequence, + txBytes: txBytes, + timestamp: time.Now(), + } +} + +// GetTxFromTxTracker retrieves transaction metadata by hash +// Returns: sequence, signer, txBytes, exists +func (t *TxTracker) GetTxFromTxTracker(txHash string) (uint64, string, []byte, bool) { + t.mu.RLock() + defer t.mu.RUnlock() + + info, exists := t.TxQueue[txHash] + if !exists { + return 0, "", nil, false + } + + return info.sequence, info.signer, info.txBytes, true +} + +// RemoveTxFromTxTracker removes a transaction from the tracker +func (t *TxTracker) RemoveTxFromTxTracker(txHash string) { + t.mu.Lock() + defer t.mu.Unlock() + + delete(t.TxQueue, txHash) +} + +// deleteFromTxTracker removes a transaction from the tracker (alias for RemoveTxFromTxTracker) +func (t *TxTracker) deleteFromTxTracker(txHash string) { + t.RemoveTxFromTxTracker(txHash) +} + +// GetTxBytes retrieves the transaction bytes for a given account and sequence +// Returns nil if not found +func (t *TxTracker) GetTxBytes(accountName string, sequence uint64) []byte { + t.mu.RLock() + defer t.mu.RUnlock() + + // Search through all transactions to find one matching the account and sequence + for _, info := range t.TxQueue { + if info.signer == accountName && info.sequence == sequence { + return info.txBytes + } + } + + return nil +} + +// pruneTxTracker removes transactions older than txTrackerPruningInterval +// This prevents memory leaks from accumulating transaction history +func (t *TxTracker) pruneTxTracker() { + t.mu.Lock() + defer t.mu.Unlock() + + cutoffTime := time.Now().Add(-txTrackerPruningInterval) + + for txHash, info := range t.TxQueue { + if info.timestamp.Before(cutoffTime) { + delete(t.TxQueue, txHash) + } + } +} + diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index 662c893caa..4ecda959e6 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -3,6 +3,7 @@ package v2 import ( "context" "fmt" + "strings" "sync" "time" @@ -28,7 +29,7 @@ type SequentialSubmissionResult struct { Error error } -// sequentialQueue manages single-threaded transaction submission +// sequentialQueue manages single-threaded transaction submission with a unified queue type sequentialQueue struct { wg sync.WaitGroup ctx context.Context @@ -38,37 +39,33 @@ type sequentialQueue struct { accountName string pollTime time.Duration - // Transaction processing - mu sync.RWMutex - queuedTxs map[uint64]*queuedTx // Transactions waiting to be broadcast - pendingTxs map[uint64]*pendingTx // Transactions broadcast, waiting confirmation -} + // Single unified queue - transactions stay here until confirmed + mu sync.RWMutex + queue []*queuedTx // All transactions from submission to confirmation + ResignChan chan *queuedTx // Channel for all rejected transactions that need to be resigned + ResubmitChan chan *queuedTx // Channel for all evicted transactions that need to be resubmitted -// queuedTx represents a transaction waiting to be broadcast -type queuedTx struct { - sequence uint64 - blobs []*share.Blob - options []user.TxOption - resultsC chan SequentialSubmissionResult - needsResign bool // True if needs resigning (e.g., after prior rejection) + // Track last confirmed sequence for rollback logic + lastConfirmedSeq uint64 } -// pendingTx tracks a transaction that has been broadcast -type pendingTx struct { - sequence uint64 - status string - txHash string - txBytes []byte // Stored for resubmission without resigning - blobs []*share.Blob - options []user.TxOption - resultsC chan SequentialSubmissionResult - submittedAt time.Time - attempts int +// queuedTx represents a transaction in the queue (from submission to confirmation) +type queuedTx struct { + // Original submission data + blobs []*share.Blob + options []user.TxOption + resultsC chan SequentialSubmissionResult + + // Set after broadcast + txHash string // Empty until broadcast + txBytes []byte // Set after broadcast, used for eviction resubmission + sequence uint64 // Set after broadcast + submittedAt time.Time // Set after broadcast + shouldResign bool // Set after broadcast } const ( defaultSequentialQueueSize = 100 - maxResubmitAttempts = 5 ) func newSequentialQueue(client *TxClient, accountName string, pollTime time.Duration) *sequentialQueue { @@ -76,275 +73,392 @@ func newSequentialQueue(client *TxClient, accountName string, pollTime time.Dura pollTime = user.DefaultPollTime } - return &sequentialQueue{ - client: client, - accountName: accountName, - pollTime: pollTime, - queuedTxs: make(map[uint64]*queuedTx), - pendingTxs: make(map[uint64]*pendingTx), + ctx, cancel := context.WithCancel(context.Background()) + q := &sequentialQueue{ + client: client, + accountName: accountName, + pollTime: pollTime, + ctx: ctx, + cancel: cancel, + queue: make([]*queuedTx, 0, defaultSequentialQueueSize), + ResignChan: make(chan *queuedTx, 10), // Buffered channel for resign requests + ResubmitChan: make(chan *queuedTx, 10), // Buffered channel for resubmit requests } + return q } -// start initiates the sequential queue processor -func (q *sequentialQueue) start(ctx context.Context) error { - q.ctx, q.cancel = context.WithCancel(ctx) - - // Start the processing loop (broadcasts queued txs in order) - q.wg.Add(1) +// start begins the sequential queue processing +func (q *sequentialQueue) start() { + q.wg.Add(2) go func() { defer q.wg.Done() - q.processLoop() + q.coordinate() }() - - // Start the monitoring loop (confirms pending txs) - q.wg.Add(1) go func() { defer q.wg.Done() q.monitorLoop() }() - - return nil } -// stop shuts down the sequential queue -func (q *sequentialQueue) stop() { - if q.ctx == nil { - return +// submitJob adds a new transaction to the queue +func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { + q.mu.Lock() + defer q.mu.Unlock() + + qTx := &queuedTx{ + blobs: job.Blobs, + options: job.Options, + resultsC: job.ResultsC, } - q.cancel() - q.wg.Wait() - q.ctx, q.cancel = nil, nil + q.queue = append(q.queue, qTx) } -// isStarted returns whether the queue is started -func (q *sequentialQueue) isStarted() bool { - return q.ctx != nil && q.cancel != nil +// GetQueueSize returns the number of transactions in the queue +func (q *sequentialQueue) GetQueueSize() int { + q.mu.RLock() + defer q.mu.RUnlock() + return len(q.queue) } -// submitJob submits a job to the sequential queue (directly, no channel) -func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { - if !q.isStarted() { - job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("sequential queue not started")} - return +// processNextTx signs and broadcasts the next unbroadcast transaction in queue +func (q *sequentialQueue) processNextTx() { + // Find first unbroadcast transaction (txHash is empty) + fmt.Println("Processing next tx") + var qTx *queuedTx + q.mu.RLock() + for _, tx := range q.queue { + if tx.txHash == "" { + qTx = tx + break + } } + q.mu.RUnlock() - // Check if job context is cancelled - if job.Ctx.Err() != nil { - job.ResultsC <- SequentialSubmissionResult{Error: job.Ctx.Err()} + if qTx == nil { return } - // Get current sequence - acc := q.client.Account(q.accountName) - if acc == nil { - job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("account %s not found", q.accountName)} + resp, err := q.client.BroadcastPayForBlobWithoutRetry( + q.ctx, + q.accountName, + qTx.blobs, + qTx.options..., + ) + + if err != nil || resp.Code != 0 { + // Check if this is a sequence mismatch AND we're blocked + // This means the sequence was rolled back while we were broadcasting + // TODO: ma\ybe we can check if q is blocked and if so, return + // otherwise it could mean client is stalled + if IsSequenceMismatchError(err) { + fmt.Println("Sequence mismatch error") + // return we probably need to resign earlier transactions + // come back to this later + return + } + + // Other broadcast errors - send error and remove from queue + select { + case qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("broadcast failed: %w", err), + }: + case <-q.ctx.Done(): + } + q.removeFromQueue(qTx) return } - sequence := acc.Sequence() - // Add to queued transactions q.mu.Lock() - q.queuedTxs[sequence] = &queuedTx{ - sequence: sequence, - blobs: job.Blobs, - options: job.Options, - resultsC: job.ResultsC, - needsResign: false, - } - q.mu.Unlock() - - // Increment sequence for next job - if err := q.client.Signer().IncrementSequence(q.accountName); err != nil { - job.ResultsC <- SequentialSubmissionResult{Error: fmt.Errorf("error incrementing sequence: %w", err)} - } + defer q.mu.Unlock() + // Broadcast successful - mark as broadcast in queue + sequence := q.client.Signer().Account(q.accountName).Sequence() + txBytes := q.client.TxClient.TxTracker.GetTxBytes(q.accountName, sequence) + + qTx.txHash = resp.TxHash + qTx.txBytes = txBytes + qTx.sequence = sequence + qTx.submittedAt = time.Now() + fmt.Println("Broadcast successful - marking as broadcast in queue") } -// processLoop is a process continuely submitting txs from Q -func (q *sequentialQueue) processLoop() { - // TODO: Maybe this should be like 1 second? so tx client can actually be the one to control submission cadence rather than user? - ticker := time.NewTicker(q.pollTime / 2) // ARBITRARY DELAY +// monitorLoop periodically checks the status of broadcast transactions +func (q *sequentialQueue) monitorLoop() { + ticker := time.NewTicker(1 * time.Second) defer ticker.Stop() for { select { - case <-ticker.C: - q.submitTxsInQueue() case <-q.ctx.Done(): return + case <-ticker.C: + q.checkBroadcastTransactions() } } } -// submitTxsInQueue broadcasts the next transaction in sequence order -// this needs to change to only lock per transaction submission rather than the whole queue -func (q *sequentialQueue) submitTxsInQueue() { - q.mu.Lock() - defer q.mu.Unlock() - - // Get the first transaction from the queue - // keep one second delay between submissions - for _, queued := range q.queuedTxs { - - // Broadcast the transaction - var resp *sdktypes.TxResponse - // TODO: figure out how to set tx bytes - var txBytes []byte - var err error +// coordinate coordinates transaction submission with confirmation +func (q *sequentialQueue) coordinate() { + ticker := time.NewTicker(time.Second) + defer ticker.Stop() - resp, err = q.client.BroadcastPayForBlobWithoutRetry(q.ctx, q.accountName, queued.blobs, queued.options...) - if err != nil { - // check if the error is sequence mismatch and if tx should be resigned. (never resign evicted txs) - queued.resultsC <- SequentialSubmissionResult{Error: fmt.Errorf("broadcast failed: %w", err)} - delete(q.queuedTxs, queued.sequence) + for { + select { + case <-q.ctx.Done(): return + case <-q.ResignChan: + fmt.Println("Resigning rejected tx") + q.ResignRejected() + case qTx := <-q.ResubmitChan: + fmt.Println("Resubmitting evicted tx") + q.ResubmitEvicted(qTx) + case <-ticker.C: + q.processNextTx() } + } +} - // Move from queued to pending - q.pendingTxs[queued.sequence] = &pendingTx{ - sequence: queued.sequence, - txHash: resp.TxHash, - txBytes: txBytes, - blobs: queued.blobs, - options: queued.options, - resultsC: queued.resultsC, - submittedAt: time.Now(), - attempts: 1, +// ResignRejected resigns a rejected transaction +func (q *sequentialQueue) ResignRejected() { + fmt.Println("Resigning rejected tx") + q.mu.RLock() + var txsToResign []*queuedTx + for _, qTx := range q.queue { + if qTx.shouldResign { + txsToResign = append(txsToResign, qTx) + } + } + q.mu.RUnlock() + + for _, qTx := range txsToResign { + if qTx.shouldResign { + // resign the tx + resp, err := q.client.BroadcastPayForBlobWithoutRetry( + q.ctx, + q.accountName, + qTx.blobs, + qTx.options..., + ) + if err != nil { + // send error and remove from queue + select { + case qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("rejected and failed to resign: %w", err), + }: + case <-q.ctx.Done(): + } + q.removeFromQueue(qTx) + return + } + sequence := q.client.Signer().Account(q.accountName).Sequence() + txBytes := q.client.TxClient.TxTracker.GetTxBytes(q.accountName, sequence) + + qTx.txHash = resp.TxHash + qTx.txBytes = txBytes + qTx.sequence = sequence + qTx.shouldResign = false + fmt.Printf("Resigned and submitted tx successfully: %s\n", resp.TxHash) } - delete(q.queuedTxs, queued.sequence) - - time.Sleep(time.Second) // this is submission delay but could be done differently } } -// monitorLoop periodically checks pending transactions -func (q *sequentialQueue) monitorLoop() { - ticker := time.NewTicker(q.pollTime) - defer ticker.Stop() +// TODO: come back to this and see if it makes sense +// func (q *sequentialQueue) setTxInfo(qTx *queuedTx, resp *sdktypes.TxResponse, txBytes []byte, sequence uint64) { +// q.mu.Lock() +// defer q.mu.Unlock() - for { +// qTx.txHash = resp.TxHash +// qTx.txBytes = txBytes +// qTx.sequence = sequence +// qTx.shouldResign = false +// } + +func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { + fmt.Println("Resubmitting evicted tx") + q.mu.RLock() + txBytes := qTx.txBytes + q.mu.RUnlock() + + // check if the tx needs to be resubmitted + resubmitResp, err := q.client.ResubmitTxBytes(q.ctx, txBytes) + if err != nil || resubmitResp.Code != 0 { + // send error and remove from queue select { - case <-ticker.C: - q.checkPendingTransactions() + case qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("evicted and failed to resubmit: %w", err), + }: case <-q.ctx.Done(): - return } + q.removeFromQueue(qTx) + return } } -// checkPendingTransactions checks status of all pending transactions -func (q *sequentialQueue) checkPendingTransactions() { - // todo; possibly error prone - txClient := tx.NewTxClient(q.client.GetConn()) +// checkBroadcastTransactions checks status of all broadcast transactions +func (q *sequentialQueue) checkBroadcastTransactions() { + fmt.Println("Checking broadcast transactions") + q.mu.RLock() + // Collect all broadcast transactions (those with non-empty txHash) + var broadcastTxs []*queuedTx // TODO: cap the size + for _, tx := range q.queue { + if tx.txHash != "" { + broadcastTxs = append(broadcastTxs, tx) + } + } + q.mu.RUnlock() + + if len(broadcastTxs) == 0 { + return + } - // Check all pending transactions - for _, pending := range q.pendingTxs { - seq := pending.sequence + // Create tx client for status queries + txClient := tx.NewTxClient(q.client.GetGRPCConnection()) - // Check transaction status - statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: pending.txHash}) + for _, qTx := range broadcastTxs { + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) if err != nil { - // Network error, skip this check - continue + continue // Network error, try again later } switch statusResp.Status { - case core.TxStatusPending: - // Still pending, continue monitoring - continue - case core.TxStatusCommitted: - // Check execution code - if statusResp.ExecutionCode != abci.CodeTypeOK { - // Execution failed - treat like rejection, roll back signer sequence - // should prolly use execution error type same as in tx client - pending.resultsC <- SequentialSubmissionResult{ - Error: fmt.Errorf("tx execution failed with code %d: %s", statusResp.ExecutionCode, statusResp.Error), - } - continue - } - // use populate tx response from tx client TODO: reuse the function that populates - pending.resultsC <- SequentialSubmissionResult{ - TxResponse: &sdktypes.TxResponse{ - Height: statusResp.Height, - TxHash: pending.txHash, - Code: statusResp.ExecutionCode, - Codespace: statusResp.Codespace, - GasWanted: statusResp.GasWanted, - GasUsed: statusResp.GasUsed, - Signers: statusResp.Signers, - }, - Error: nil, - } - + q.handleCommitted(qTx, statusResp) case core.TxStatusEvicted: - // Transaction evicted - put back in queue for resubmission WITHOUT resigning - // Add back to queuedTxs with same sequence - q.mu.Lock() - q.queuedTxs[seq] = &queuedTx{ - sequence: seq, - blobs: pending.blobs, - options: pending.options, - resultsC: pending.resultsC, - needsResign: false, // Keep existing tx bytes, no resigning - } - delete(q.pendingTxs, seq) - q.mu.Unlock() + fmt.Println("Handling evicted tx") + q.ResubmitChan <- qTx case core.TxStatusRejected: - // tx rejected - roll back the signer sequence - // check if the tx before was also rejected, if so, no need to roll back the signer sequence - // because it would have been rolled back already - prevTx, exists := q.pendingTxs[seq-1] // TODO: we need accessors with mutexes for this - if !exists || prevTx.status != core.TxStatusRejected { - // no previous tx, so we can roll back the signer sequence - q.client.TxClient.Signer().SetSequence(q.accountName, seq) - continue - } - pending.status = core.TxStatusRejected - // TODO: how do i do this? maybe i should keep track of tx hashes and just tx status them - pending.resultsC <- SequentialSubmissionResult{ - Error: fmt.Errorf("tx rejected: %s", statusResp.Error), - } - // TODO: remove the bytes from the tx tracker and just keep the sequence, hash and status - // this is to save storage space + q.handleRejected(qTx, statusResp, txClient) + } + } +} - // Mark subsequent transactions for resigning - q.markForResign(seq) +// handleCommitted processes a confirmed transaction +func (q *sequentialQueue) handleCommitted(qTx *queuedTx, statusResp *tx.TxStatusResponse) { + fmt.Println("Handling confirmed tx") + // Check execution code + if statusResp.ExecutionCode != abci.CodeTypeOK { + // Execution failed + select { + case <-q.ctx.Done(): + case qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("tx execution failed with code %d: %s", statusResp.ExecutionCode, statusResp.Error), + }: } + q.removeFromQueue(qTx) + return + } + + // Success - send result + select { + case <-q.ctx.Done(): + return + case qTx.resultsC <- SequentialSubmissionResult{ + TxResponse: &sdktypes.TxResponse{ + Height: statusResp.Height, + TxHash: qTx.txHash, + Code: statusResp.ExecutionCode, + Codespace: statusResp.Codespace, + GasWanted: statusResp.GasWanted, + GasUsed: statusResp.GasUsed, + Signers: statusResp.Signers, + }, + Error: nil, + }: } + // Update last confirmed sequence + q.setLastConfirmedSeq(qTx.sequence) + q.removeFromQueue(qTx) } -// markForResign marks all subsequent queued transactions for resigning -// idk if this is the best way, we shall see -func (q *sequentialQueue) markForResign(fromSeq uint64) { - for seq, queued := range q.queuedTxs { - if seq > fromSeq { - queued.needsResign = true +func (q *sequentialQueue) setLastConfirmedSeq(seq uint64) { + q.mu.Lock() + defer q.mu.Unlock() + q.lastConfirmedSeq = seq +} + +// handleRejected processes a rejected transaction +func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { + fmt.Println("Handling rejected tx") + // Step 1: Roll back sequence if previous tx was confirmed + if q.isPreviousTxConfirmed(qTx.sequence) { + q.mu.Lock() + q.client.Signer().SetSequence(q.accountName, qTx.sequence) + q.mu.Unlock() + } + + isNonceMismatch := isSequenceMismatchRejection(statusResp.Error) + if isNonceMismatch { + q.mu.Lock() + qTx.shouldResign = true + q.mu.Unlock() + } + + // Step 2: Collect subsequent transactions to check + q.mu.RLock() + var subsequentTxs []*queuedTx + for _, subTx := range q.queue { + if subTx.sequence > qTx.sequence && subTx.txHash != "" { + subsequentTxs = append(subsequentTxs, subTx) + } + } + q.mu.RUnlock() + + // Step 3: Batch query subsequent transactions to see if they were also rejected // TODO: in future this should be handled by batch txstatus request + for _, subTx := range subsequentTxs { + if subTx.sequence > qTx.sequence && subTx.txHash != "" { + // TODO: this should also be rejected for sequence mismatch + resp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: subTx.txHash}) + if err == nil && resp.Status == core.TxStatusRejected && resp.ExecutionCode == 32 { + fmt.Println("Sequence mismatch error: ReCheck()") + q.mu.Lock() + subTx.shouldResign = true + q.mu.Unlock() + } + } + } + // Q: should we wait till all txs are marked for resign before sending to resign channel? + q.ResignChan <- qTx + + if !isNonceMismatch { + // Non-nonce error remove from queue and return error back to user + select { + case <-q.ctx.Done(): + case qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("tx rejected: %s", statusResp.Error), + }: } + q.removeFromQueue(qTx) + return } + } -// purgeOldTxs removes rejected/confirmed transactions that are older than 10 minutes (possible refactor this to height since time is the worst in blockchain env -// but here it feels not too bad since its only for cleanup) -// TODO: find out how much storage is required for this to work. -// we could possibly remove everything but sequence, status and hash for confirmed/rejected txs for lightweight storage. -func (q *sequentialQueue) purgeOldTxs() { +// removeFromQueue removes a transaction from the queue +func (q *sequentialQueue) removeFromQueue(qTx *queuedTx) { q.mu.Lock() defer q.mu.Unlock() - // remove all transactions that are older than 10 minutes - for seq, pending := range q.pendingTxs { - if time.Since(pending.submittedAt) > 10*time.Minute { - delete(q.pendingTxs, seq) - delete(q.queuedTxs, seq) + for i, tx := range q.queue { + if tx == qTx { + q.queue = append(q.queue[:i], q.queue[i+1:]...) + return } } } -// GetQueueSize returns the number of pending transactions -func (q *sequentialQueue) GetQueueSize() int { +// isPreviousTxConfirmed checks if the previous transaction was confirmed +func (q *sequentialQueue) isPreviousTxConfirmed(seq uint64) bool { q.mu.RLock() defer q.mu.RUnlock() - return len(q.queuedTxs) + len(q.pendingTxs) + if seq == 0 { + return true + } + return q.lastConfirmedSeq >= seq-1 +} + +// isSequenceMismatchRejection checks if an error message indicates sequence mismatch +func isSequenceMismatchRejection(errMsg string) bool { + return strings.Contains(errMsg, "account sequence mismatch") || + strings.Contains(errMsg, "incorrect account sequence") } diff --git a/pkg/user/v2/tx_client.go b/pkg/user/v2/tx_client.go index 52336cc7aa..06e1dac9c3 100644 --- a/pkg/user/v2/tx_client.go +++ b/pkg/user/v2/tx_client.go @@ -7,12 +7,14 @@ import ( "time" "github.com/celestiaorg/celestia-app/v6/app/encoding" + apperrors "github.com/celestiaorg/celestia-app/v6/app/errors" "github.com/celestiaorg/celestia-app/v6/pkg/user" "github.com/celestiaorg/go-square/v3/share" "github.com/cosmos/cosmos-sdk/codec" codectypes "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/crypto/keyring" sdktypes "github.com/cosmos/cosmos-sdk/types" + sdktx "github.com/cosmos/cosmos-sdk/types/tx" "google.golang.org/grpc" ) @@ -25,6 +27,9 @@ type TxClient struct { // Sequential queues per account sequentialQueues map[string]*sequentialQueue queueMu sync.RWMutex + + // Primary GRPC connection (stored separately for access) + conn *grpc.ClientConn } // NewTxClient creates a new v2 TxClient by wrapping the original NewTxClient function. @@ -42,6 +47,7 @@ func NewTxClient( v2Client := &TxClient{ TxClient: v1Client, sequentialQueues: make(map[string]*sequentialQueue), + conn: conn, } if err := v2Client.StartSequentialQueue(context.Background(), v1Client.DefaultAccountName()); err != nil { return nil, err @@ -64,6 +70,7 @@ func SetupTxClient( v2Client := &TxClient{ TxClient: v1Client, sequentialQueues: make(map[string]*sequentialQueue), + conn: conn, } if err := v2Client.StartSequentialQueue(ctx, v1Client.DefaultAccountName()); err != nil { return nil, err @@ -72,10 +79,12 @@ func SetupTxClient( } // Wrapv1TxClient wraps a v1 TxClient and returns a v2 TxClient. +// Note: connection will be nil, so sequential queue features requiring direct connection access won't work func Wrapv1TxClient(v1Client *user.TxClient) *TxClient { return &TxClient{ TxClient: v1Client, sequentialQueues: make(map[string]*sequentialQueue), + conn: nil, } } @@ -158,9 +167,7 @@ func (c *TxClient) StartSequentialQueueWithPollTime(ctx context.Context, account } queue := newSequentialQueue(c, accountName, pollTime) - if err := queue.start(ctx); err != nil { - return fmt.Errorf("failed to start sequential queue: %w", err) - } + queue.start(); c.sequentialQueues[accountName] = queue return nil @@ -172,7 +179,7 @@ func (c *TxClient) StopSequentialQueue(accountName string) { defer c.queueMu.Unlock() if queue, exists := c.sequentialQueues[accountName]; exists { - queue.stop() + queue.cancel() delete(c.sequentialQueues, accountName) } } @@ -183,7 +190,7 @@ func (c *TxClient) StopAllSequentialQueues() { defer c.queueMu.Unlock() for accountName, queue := range c.sequentialQueues { - queue.stop() + queue.cancel() delete(c.sequentialQueues, accountName) } } @@ -204,7 +211,8 @@ func (c *TxClient) SubmitPFBToSequentialQueueWithAccount(ctx context.Context, ac } resultsC := make(chan SequentialSubmissionResult, 1) - defer close(resultsC) + // Note: We don't close this channel because the queue worker goroutines + // may still send on it after ctx.Done(). The channel will be GC'd naturally. job := &SequentialSubmissionJob{ Blobs: blobs, @@ -212,7 +220,7 @@ func (c *TxClient) SubmitPFBToSequentialQueueWithAccount(ctx context.Context, ac Ctx: ctx, ResultsC: resultsC, } - + fmt.Println("Submitting job to sequential queue") queue.submitJob(job) // Wait for result @@ -241,49 +249,66 @@ func (c *TxClient) GetSequentialQueueSize(accountName string) (int, error) { } // BroadcastPayForBlobWithoutRetry broadcasts a PayForBlob transaction without automatic retry logic. -// TTODO: this will do for now, but i should refactor tx client functions without breaking the API to make it more usable func (c *TxClient) BroadcastPayForBlobWithoutRetry(ctx context.Context, accountName string, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, error) { - // Lock the client for the duration of build+sign+send - c.TxClient.mtx.Lock() - defer c.TxClient.mtx.Unlock() - - // Check account is loaded - if err := c.TxClient.CheckAccountLoadedExported(ctx, accountName); err != nil { - return nil, err - } - - acc := c.Signer().Account(accountName) - if acc == nil { - return nil, fmt.Errorf("account %s not found", accountName) - } - - // Build the transaction bytes - txBytes, _, err := c.Signer().CreatePayForBlobs(accountName, blobs, opts...) + // Use BroadcastPayForBlobWithAccount but without confirmation + resp, err := c.TxClient.BroadcastPayForBlobWithAccount(ctx, accountName, blobs, opts...) if err != nil { return nil, err } - // Get the sequence before sending - sequence := c.Signer().Account(accountName).Sequence() + return &sdktypes.TxResponse{ + Height: resp.Height, + TxHash: resp.TxHash, + Code: resp.Code, + Codespace: resp.Codespace, + GasWanted: resp.GasWanted, + GasUsed: resp.GasUsed, + Signers: resp.Signers, + }, nil +} - // Send directly without retry logic - conn := c.GetConn() +// ResubmitTxBytes resubmits a transaction using pre-signed bytes without retry logic +func (c *TxClient) ResubmitTxBytes(ctx context.Context, txBytes []byte) (*sdktypes.TxResponse, error) { + // Get the connection for broadcasting + conn := c.GetGRPCConnection() if conn == nil { return nil, fmt.Errorf("no connection available") } - resp, err := c.TxClient.SendTxToConnectionExported(ctx, conn, txBytes) + // Use the SDK tx service client to broadcast + sdktxClient := sdktx.NewServiceClient(conn) + resp, err := sdktxClient.BroadcastTx(ctx, &sdktx.BroadcastTxRequest{ + Mode: sdktx.BroadcastMode_BROADCAST_MODE_SYNC, + TxBytes: txBytes, + }) if err != nil { return nil, err } - // Track the transaction - c.TxClient.GetTxTrackerExported().TrackTransaction(accountName, sequence, resp.TxHash, txBytes) + // Check if broadcast was successful + if resp.TxResponse.Code != 0 { + return nil, fmt.Errorf("broadcast failed with code %d: %s", resp.TxResponse.Code, resp.TxResponse.RawLog) + } + + return resp.TxResponse, nil +} + +// GetGRPCConnection returns the primary GRPC connection for creating tx status clients +func (c *TxClient) GetGRPCConnection() *grpc.ClientConn { + return c.conn +} + +// IsSequenceMismatchError checks if an error is a sequence mismatch (nonce mismatch) +func IsSequenceMismatchError(err error) bool { + if err == nil { + return false + } - // Increment sequence after successful broadcast - if err := c.Signer().IncrementSequence(accountName); err != nil { - return nil, fmt.Errorf("error incrementing sequence: %w", err) + // Check if it's a BroadcastTxError with sequence mismatch code + broadcastTxErr, ok := err.(*user.BroadcastTxError) + if !ok { + return false } - return resp, nil + return apperrors.IsNonceMismatchCode(broadcastTxErr.Code) } diff --git a/scripts/single-node-all-upgrades.sh b/scripts/single-node-all-upgrades.sh index 769c5d4aaf..e7b4bd2d24 100755 --- a/scripts/single-node-all-upgrades.sh +++ b/scripts/single-node-all-upgrades.sh @@ -101,7 +101,7 @@ startCelestiaApp() { --api.enable \ --grpc.enable \ --grpc-web.enable \ - --delayed-precommit-timeout 1s + --delayedt-precommit-timeout 1s } # Function to perform upgrade to a specific version diff --git a/tools/spam_txclient/main.go b/tools/spam_txclient/main.go index 85ea6c9029..e3689d9dde 100644 --- a/tools/spam_txclient/main.go +++ b/tools/spam_txclient/main.go @@ -14,9 +14,11 @@ import ( "github.com/celestiaorg/celestia-app/v6/app" "github.com/celestiaorg/celestia-app/v6/app/encoding" "github.com/celestiaorg/celestia-app/v6/pkg/user" + v2 "github.com/celestiaorg/celestia-app/v6/pkg/user/v2" "github.com/celestiaorg/go-square/v3/share" "github.com/cosmos/cosmos-sdk/client/grpc/cmtservice" "github.com/cosmos/cosmos-sdk/crypto/keyring" + sdktypes "github.com/cosmos/cosmos-sdk/types" "golang.org/x/sync/errgroup" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" @@ -34,7 +36,7 @@ func main() { MochaEndpoint: "rpc-mocha.pops.one:9090", BlobSizeKB: 300, // 300 KiB blobs IntervalMs: 1000, // Submit every 1 second - TestDurationSec: 240, // Run for 240 seconds + TestDurationSec: 0, // Run forever (0 = no timeout) } err := RunLoadTest(cfg) @@ -45,16 +47,25 @@ func main() { // RunLoadTest sets up the tx client, runs submissions, and reports results. func RunLoadTest(cfg Config) error { - log.Println("Setting up tx client and connecting to a mocha node") - - // Global test context with configured timeout - ctx, cancel := context.WithTimeout(context.Background(), time.Duration(cfg.TestDurationSec)*time.Second) + log.Println("Setting up tx client v2 with sequential queue and connecting to a mocha node") + + // Global test context with optional timeout + var ctx context.Context + var cancel context.CancelFunc + if cfg.TestDurationSec > 0 { + ctx, cancel = context.WithTimeout(context.Background(), time.Duration(cfg.TestDurationSec)*time.Second) + log.Printf("Running for %d seconds", cfg.TestDurationSec) + } else { + ctx, cancel = context.WithCancel(context.Background()) + log.Println("Running forever (Ctrl+C to stop)") + } defer cancel() - txClient, grpcConn, _, err := NewMochaTxClient(ctx, cfg) + txClient, grpcConn, cleanupFunc, err := NewMochaTxClientV2(ctx, cfg) if err != nil { - return fmt.Errorf("failed to set up tx client: %w", err) + return fmt.Errorf("failed to set up tx client v2: %w", err) } + defer cleanupFunc() var ( txCounter int64 @@ -78,83 +89,92 @@ func RunLoadTest(cfg Config) error { case <-ctx.Done(): return ctx.Err() case <-ticker.C: - if time.Since(time.Unix(0, lastSuccess.Load())) > 10*time.Second { - return fmt.Errorf("TxClient appears halted: no successful submission recently") + if time.Since(time.Unix(0, lastSuccess.Load())) > 10*time.Minute { + return fmt.Errorf("TxClient appears halted: no successful submission in 10 minutes") } id := atomic.AddInt64(&txCounter, 1) - // Separate goroutine for broadcasting and confirming txs + // Separate goroutine for submitting txs via sequential queue g.Go(func() error { - hash, err := BroadcastPayForBlob(ctx, txClient, grpcConn, cfg.BlobSizeKB, id) - if err != nil || hash == "" { - fmt.Printf("\nTX-%d: Broadcast failed: %v\n", id, err) + // Use background context so transaction never times out + txCtx := context.Background() + + resp, err := SubmitPayForBlobSequential(txCtx, txClient, grpcConn, cfg.BlobSizeKB, id) + if err != nil { + fmt.Printf("\nTX-%d: Sequential submission failed: %v\n", id, err) failedBroadcasts.Add(1) return nil } lastSuccess.Store(time.Now().UnixNano()) - fmt.Printf("\nTX-%d: Broadcast success (hash %s)\n", id, hash) + fmt.Printf("\nTX-%d: Sequential submission success (hash %s)\n", id, resp.TxHash) successfulBroadcasts.Add(1) - resp, err := txClient.ConfirmTx(ctx, hash) - if err != nil { - fmt.Printf("\nTX-%d: Confirm failed: %v\n", id, err) + // Sequential queue handles confirmation internally + if resp.Code == 0 { + fmt.Printf("\nTX-%d: Confirmed successfully: %s\n", id, resp.TxHash) + successfulConfirms.Add(1) + } else { + fmt.Printf("\nTX-%d: Execution failed with code %d\n", id, resp.Code) failedConfirms.Add(1) - return nil } - - fmt.Printf("\nTX-%d: Confirm success for %s: %d\n", id, hash, resp.Code) - successfulConfirms.Add(1) return nil }) } } }) - // This should only fail if the client halts + // This should only fail if the client halts unexpectedly err = g.Wait() - if err != nil && !errors.Is(err, context.DeadlineExceeded) { - return err + if err != nil && !errors.Is(err, context.DeadlineExceeded) && !errors.Is(err, context.Canceled) { + return fmt.Errorf("tx client v2 halted unexpectedly: %w", err) } - fmt.Println("\nScript completed successfully!!") - fmt.Printf("Successful broadcasts: %d\n", successfulBroadcasts.Load()) - fmt.Printf("Successful confirms: %d\n", successfulConfirms.Load()) - fmt.Printf("Failed confirms: %d\n", failedConfirms.Load()) - fmt.Printf("Failed broadcasts: %d\n", failedBroadcasts.Load()) + fmt.Println("\n=== TxClient V2 Sequential Queue Test Results ===") + fmt.Printf("Successful sequential submissions: %d\n", successfulBroadcasts.Load()) + fmt.Printf("Successful confirmations: %d\n", successfulConfirms.Load()) + fmt.Printf("Failed confirmations: %d\n", failedConfirms.Load()) + fmt.Printf("Failed submissions: %d\n", failedBroadcasts.Load()) + + queueSize, err := txClient.GetSequentialQueueSize(txClient.DefaultAccountName()) + if err == nil { + fmt.Printf("Final queue size: %d\n", queueSize) + } cancel() // Clean up context before exit return nil } -func BroadcastPayForBlob(ctx context.Context, txClient *user.TxClient, grpcConn *grpc.ClientConn, blobSizeKB int, txID int64) (string, error) { +func SubmitPayForBlobSequential(ctx context.Context, txClient *v2.TxClient, grpcConn *grpc.ClientConn, blobSizeKB int, txID int64) (*sdktypes.TxResponse, error) { // Create random blob data of the given size blobData := make([]byte, blobSizeKB*1024) if _, err := cryptorand.Read(blobData); err != nil { - return "", err + return nil, err } namespace := share.RandomBlobNamespace() blob, err := share.NewV0Blob(namespace, blobData) if err != nil { - return "", err + return nil, err } currentHeight, err := getCurrentBlockHeight(ctx, grpcConn) if err != nil { - return "", err + return nil, err } - // Set timeout height to be between 1 and 5 blocks from the current height - timeoutHeight := currentHeight + int64(rand.Intn(5)) - resp, err := txClient.BroadcastPayForBlob(ctx, []*share.Blob{blob}, user.SetTimeoutHeight(uint64(timeoutHeight))) + // Set timeout height to be between 3 and 10 blocks from the current height + timeoutHeight := currentHeight + int64(rand.Intn(8)+3) + + // Submit via sequential queue - this handles signing, broadcasting, and confirmation + resp, err := txClient.SubmitPFBToSequentialQueue(ctx, []*share.Blob{blob}, user.SetTimeoutHeight(uint64(timeoutHeight))) if err != nil { - return "", err + return nil, err } - return resp.TxHash, nil + return resp, nil } // getCurrentBlockHeight gets the current block height from the chain @@ -176,7 +196,7 @@ func getCurrentBlockHeight(ctx context.Context, grpcConn *grpc.ClientConn) (int6 return resp.SdkBlock.Header.Height, nil } -func NewMochaTxClient(ctx context.Context, cfg Config) (*user.TxClient, *grpc.ClientConn, context.CancelFunc, error) { +func NewMochaTxClientV2(ctx context.Context, cfg Config) (*v2.TxClient, *grpc.ClientConn, func(), error) { encCfg := encoding.MakeConfig(app.ModuleEncodingRegisters...) kr, err := keyring.New(app.Name, keyring.BackendTest, "~/.celestia-app", nil, encCfg.Codec) if err != nil { @@ -196,12 +216,22 @@ func NewMochaTxClient(ctx context.Context, cfg Config) (*user.TxClient, *grpc.Cl } clientCtx, cancel := context.WithCancel(ctx) - txClient, err := user.SetupTxClient(clientCtx, kr, grpcConn, encCfg) + + // Create v2 TxClient with sequential queue + txClient, err := v2.SetupTxClient(clientCtx, kr, grpcConn, encCfg) if err != nil { grpcConn.Close() cancel() - return nil, nil, nil, fmt.Errorf("failed to create tx client: %w", err) + return nil, nil, nil, fmt.Errorf("failed to create tx client v2: %w", err) + } + + cleanup := func() { + txClient.StopAllSequentialQueues() + grpcConn.Close() + cancel() } - return txClient, grpcConn, cancel, nil + log.Printf("Sequential queue started for account: %s", txClient.DefaultAccountName()) + + return txClient, grpcConn, cleanup, nil } From ff6327fb23dd4f47dea650a0432754cc40543d3b Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Mon, 24 Nov 2025 22:35:46 -0500 Subject: [PATCH 04/12] refactor: consider txs being evicted in sets --- latency_results.csv | 86 ++++++++++++ pkg/user/tx_client.go | 102 +++++++++----- pkg/user/v2/sequential_queue.go | 235 +++++++++++++++++++++++++++----- pkg/user/v2/tx_client.go | 20 +-- tools/latency-monitor/main.go | 81 +++++------ 5 files changed, 399 insertions(+), 125 deletions(-) create mode 100644 latency_results.csv diff --git a/latency_results.csv b/latency_results.csv new file mode 100644 index 0000000000..e0824d44c5 --- /dev/null +++ b/latency_results.csv @@ -0,0 +1,86 @@ +Submit Time,Commit Time,Latency (ms),Tx Hash,Height,Code,Failed,Error +2025-11-24T22:21:23.771767-05:00,2025-11-24T22:21:35.988516-05:00,12216.00,7A9B74F21EB480ADD5FAB98867C2D294F603282D533714086C56372A120AB76B,8955106,0,false, +2025-11-24T22:21:24.270797-05:00,2025-11-24T22:21:41.538142-05:00,17267.00,003DE58F51AC96CCAAD8A3C91D2D339C792FA32ADDDD18D796E2D3EFD92CA069,8955107,0,false, +2025-11-24T22:21:24.77405-05:00,2025-11-24T22:21:41.850616-05:00,17076.00,5F538FAD784196B377D3E77155495AA303A278AC215EC32D75FF7BE9C29DDBBF,8955107,0,false, +2025-11-24T22:21:25.266592-05:00,2025-11-24T22:21:41.960953-05:00,16694.00,ECEBF9035BE7F10148493B6A74D33DB2AB315D2B835567EDA08C35F406BF5C66,8955107,0,false, +2025-11-24T22:21:25.767846-05:00,2025-11-24T22:21:42.111902-05:00,16343.00,6D7A85F359F3326BD1781872793541503040B66E1D3D1344EF141059D3F5B28B,8955107,0,false, +2025-11-24T22:21:26.26837-05:00,2025-11-24T22:21:42.224576-05:00,15956.00,4447C28DFFCF8F7FCAFE4B233CA475856C50C532D65FFF5BEA9A4582B7604C4B,8955107,0,false, +2025-11-24T22:21:26.777362-05:00,2025-11-24T22:21:53.546686-05:00,26769.00,00353954058BAF2489F5F4033D2A2709FADB03AEDFBD6890C6003791AD621FFF,8955109,0,false, +2025-11-24T22:21:27.267245-05:00,2025-11-24T22:22:00.454188-05:00,33186.00,0C24802E396EA61C99BA62D5D092C9660A967165ABF0EB2CD33DF656B1BE5986,8955110,0,false, +2025-11-24T22:21:27.780155-05:00,2025-11-24T22:22:06.283226-05:00,38502.00,3AF8536E7F827D913EC9A8BFD44665749C674B10A10B81F9A3A90688E5F6D0A0,8955111,0,false, +2025-11-24T22:21:28.274506-05:00,2025-11-24T22:22:06.529445-05:00,38254.00,5F26A3200AC794AEF161D65C5001D8496134D9751FA04917C47D582B7915E028,8955111,0,false, +2025-11-24T22:21:28.773141-05:00,2025-11-24T22:22:06.849261-05:00,38075.00,7F10CCFBDB5263FE9DF2B8B1E47FF16151A424C4576DB6D79D1D9DCE49111607,8955111,0,false, +2025-11-24T22:21:29.269952-05:00,2025-11-24T22:22:07.173533-05:00,37903.00,DB52C95C8027EB7E7B50051463030E7FBF7528D4B3F79A1D27AC393FB15ABA6F,8955111,0,false, +2025-11-24T22:21:29.780953-05:00,2025-11-24T22:22:11.549672-05:00,41768.00,5BE0445105829A173BDE8249E3BCCA4C48B1920F53C3778FE8FD9E554E88E3CB,8955112,0,false, +2025-11-24T22:21:30.279436-05:00,2025-11-24T22:22:23.503275-05:00,53223.00,0A12A2B2B1477D9070D4CE6B391F5CCA9D9CA1AB357319642A8A57D9953D73ED,8955114,0,false, +2025-11-24T22:21:30.768476-05:00,2025-11-24T22:22:23.91719-05:00,53148.00,FCB3D29C7288047B1E1B6CAB68E258CEA7361EF12DE377A4295AEC4D2A922BF3,8955114,0,false, +2025-11-24T22:21:31.273826-05:00,2025-11-24T22:22:24.260286-05:00,52986.00,16658D0C195473BFFB9A5CC6C2FC72D3A926CA92962402823DEF71E63572E37E,8955114,0,false, +2025-11-24T22:21:31.777077-05:00,2025-11-24T22:22:24.579931-05:00,52802.00,684D6D2A7BE8B3A3AE6229CBBCE36C929B549E20A875F96B636D93CBAD68CD94,8955114,0,false, +2025-11-24T22:21:34.26893-05:00,2025-11-24T22:22:44.219936-05:00,69950.00,EBDCB73DDFD0189FE48064BD12B2A2AD67D64BADF0619F05E976B453A1EA8F4A,8955116,0,false, +2025-11-24T22:21:34.782483-05:00,2025-11-24T22:22:45.157658-05:00,70374.00,A6DDBE82D1CEF47ADED3A8E91A3139908BCC0191DDC96594039E98390850E1E5,8955116,0,false, +2025-11-24T22:21:35.275238-05:00,2025-11-24T22:22:45.794181-05:00,70518.00,07C0FF570F98EEC1D6F8256C68B0E374261A82932EAF87B2DEEF2BB61B8948D5,8955116,0,false, +2025-11-24T22:21:32.281308-05:00,2025-11-24T22:22:47.782354-05:00,75500.00,1BA714E90CE90C72988841AC86744F6E691FCEA99DAE71E3960BA1743CD7C335,8955116,0,false, +2025-11-24T22:21:32.776912-05:00,2025-11-24T22:22:48.283223-05:00,75505.00,49A5DF699EA9C89C755BFD0367A155B5B3762740F89E10523ED5B0DCB49DA2A3,8955116,0,false, +2025-11-24T22:21:33.269645-05:00,2025-11-24T22:22:48.6106-05:00,75340.00,AD448DD0E47521242A6A16F2CA9D32D54AC4923284148E58D1F4B0B708670460,8955116,0,false, +2025-11-24T22:21:33.770702-05:00,2025-11-24T22:22:49.020195-05:00,75249.00,C3DC304E49346030929A12E6A310D5577E6445622AF418CBAE1AD51FCE7EB490,8955116,0,false, +2025-11-24T22:21:37.270098-05:00,2025-11-24T22:22:50.398071-05:00,73127.00,853C798B8F8DB046EB35A16D7D7C7FDF2B485010DFC38E5DCB8FFE50EE1D1B87,8955117,0,false, +2025-11-24T22:21:35.770066-05:00,2025-11-24T22:22:50.716655-05:00,74946.00,7F7B1AF4520593C3BC6A80DD75CD911FAB3DC897A3D01F08073B55A843597B88,8955117,0,false, +2025-11-24T22:21:36.267245-05:00,2025-11-24T22:22:51.097941-05:00,74830.00,8B05A1526FFC181B95F69C70451B3CEDDE664DE283E54C252CD93C8C12CCB8FD,8955117,0,false, +2025-11-24T22:21:36.77219-05:00,2025-11-24T22:22:51.417115-05:00,74644.00,9F8350DC9A5D52AB4B804F8B3EE199C5853CD0980844B6E71C62EE1979C60330,8955117,0,false, +2025-11-24T22:21:37.777745-05:00,2025-11-24T22:23:11.495606-05:00,93717.00,D64C31B619020C26864077E6B2C823F6E99FCB555F11D21734BA1B1489847C17,8955120,0,false, +2025-11-24T22:21:38.283-05:00,2025-11-24T22:23:11.803265-05:00,93519.00,96D4D70126595FCA57812F92E54DC21BF7F7FD9032C70EF43365647559BA92DE,8955120,0,false, +2025-11-24T22:21:38.767909-05:00,2025-11-24T22:23:15.226668-05:00,96458.00,70A0214BC77ECD4224B5720B18B9DF891F26FCD1C9937839638D5DF75A66FBB8,8955121,0,false, +2025-11-24T22:21:39.280079-05:00,2025-11-24T22:23:28.192018-05:00,108911.00,8E7D0BE9F8DF07DB4492F54A292108DF834F370971CEB48AD7C6A5857A561845,8955123,0,false, +2025-11-24T22:21:39.766193-05:00,2025-11-24T22:23:28.511461-05:00,108744.00,7F6DE91C2BCCEC533F06AB1D9B144C3AC16FA3348FF6F2033FD807035D2C80A6,8955123,0,false, +2025-11-24T22:21:40.27605-05:00,2025-11-24T22:23:41.529657-05:00,121252.00,D51D283FA9EE7E4B79697DF521EB4E431B3C33C99FB62FEAB4D55E61DF521D4C,8955125,0,false, +2025-11-24T22:21:40.766991-05:00,2025-11-24T22:23:41.813374-05:00,121045.00,64BC35E46898E91B80BE77DF5A87D73F237E5BC8668262E3995473AE7F4A9436,8955125,0,false, +2025-11-24T22:21:41.273143-05:00,2025-11-24T22:23:42.062752-05:00,120788.00,153A2E0C00648F87C2B0C92716A4F02F32FFAAE2C2BC7E4D0A740AE4930472A5,8955125,0,false, +2025-11-24T22:21:41.78573-05:00,2025-11-24T22:23:45.424815-05:00,123638.00,1070E416C0A87B850B97F190973A773B27EA6AA87FB0E313A89C5CCEEA2D297F,8955126,0,false, +2025-11-24T22:21:42.271998-05:00,2025-11-24T22:23:51.883117-05:00,129610.00,4C2E3F9D56C227954389158D6C43B3738E56DF491DC39F15F07BD95A52909AB7,8955127,0,false, +2025-11-24T22:21:42.775749-05:00,2025-11-24T22:24:09.382422-05:00,146605.00,AA0085458A41F405CBA9DD372EBB99B8B6C51BC274BC526BD8357A6319661E0C,8955128,0,false, +2025-11-24T22:21:43.285607-05:00,2025-11-24T22:24:11.356963-05:00,148070.00,165FB8ACA92E48278D30B354469A36D6E893211EDA46A63EB28DFF25524FBE77,8955128,0,false, +2025-11-24T22:21:43.783332-05:00,2025-11-24T22:24:12.291446-05:00,148507.00,8110066814826BC3CFFACFB52A831DD2F5561E5406A69F6AF1FFE295BEE48004,8955128,0,false, +2025-11-24T22:21:44.271932-05:00,2025-11-24T22:24:12.992785-05:00,148719.00,196D18EBCBAB18B757F42C1057D947C8CF72D30CFBC45D6813F2570FFF9F2356,8955129,0,false, +2025-11-24T22:21:44.777277-05:00,2025-11-24T22:24:13.506684-05:00,148728.00,3D19310DB64808BC26DD1F577423FF280BC9A8C3EE1DE5899ABC74BFAB93C9C4,8955129,0,false, +2025-11-24T22:21:45.26798-05:00,2025-11-24T22:24:13.98714-05:00,148718.00,863C3914F9F43E884D2D3CD29F050F95E33A1526A2EA34109CE31692EC73239F,8955129,0,false, +2025-11-24T22:21:45.776116-05:00,2025-11-24T22:24:14.39538-05:00,148618.00,7F155EBA9E8B2099591AF0412FE09A2D3CA6F22462BE4211A5F93882EDC30F36,8955129,0,false, +2025-11-24T22:21:46.282655-05:00,2025-11-24T22:24:32.394574-05:00,166110.00,D87305E2DAB16F1306F454DB90C367010D1BE2DCA3ACE881D377B9575F3B1AFA,8955132,0,false, +2025-11-24T22:21:46.772217-05:00,2025-11-24T22:24:32.694008-05:00,165920.00,F0D66B11E50293F0816F155E3EBBC637FE5A8E29B9A90AF72A27D32C36EDFB2C,8955132,0,false, +2025-11-24T22:21:47.280043-05:00,2025-11-24T22:24:32.949745-05:00,165668.00,8EB0A7A303B5320E896182C2CA59198D8929F03F2026F83C75E8CB342C8BB026,8955132,0,false, +2025-11-24T22:21:47.783358-05:00,2025-11-24T22:24:33.260243-05:00,165475.00,19906B99A42B04641CCA73D211E8A9941522F17D616F9AF02518C8A3A2AC4DCA,8955132,0,false, +2025-11-24T22:21:48.267669-05:00,2025-11-24T22:24:33.559396-05:00,165290.00,20877311D734BD2785C61AEDFE9CC422D263BE171B8238853B1FA655274677AB,8955132,0,false, +2025-11-24T22:21:48.774203-05:00,2025-11-24T22:24:33.855918-05:00,165080.00,520E7CCD899CF3A4E538CD7DA2816189DD1439E71330B1AAD76906E73ED10E6F,8955132,0,false, +2025-11-24T22:21:49.281341-05:00,2025-11-24T22:24:50.612438-05:00,181330.00,A2DAA0BDE252EF5B3613C620EA00EE490801D88190FBDD0BC44A7B8E94FA795A,8955135,0,false, +2025-11-24T22:21:49.782419-05:00,2025-11-24T22:24:50.938513-05:00,181154.00,CEBD6F6B35AACB1E2F8F3608D7113103C766FE751C58BEA815E201F58FB20199,8955135,0,false, +2025-11-24T22:21:50.293966-05:00,2025-11-24T22:24:51.045694-05:00,180750.00,51A2B793F5DA651B3048BE3B46A37FF8CF03AE9F8D946230E3A04C214090A9A2,8955135,0,false, +2025-11-24T22:21:50.775025-05:00,2025-11-24T22:24:51.156326-05:00,180380.00,1B18E52DA41968DE60480DF0E271C269EB18C35B78EBFA18CEB5DE34B1E800F0,8955135,0,false, +2025-11-24T22:21:51.274822-05:00,2025-11-24T22:24:51.43435-05:00,180158.00,C1DFAC681C48B0E7145FCC64252A8F09D805623BE0DF68B9DCF8261F59D1CBF2,8955135,0,false, +2025-11-24T22:21:52.27148-05:00,2025-11-24T22:25:01.713911-05:00,189441.00,EC9E9C3DBE29A79B63CF5F946AC0BDC98318816C6835888C213FC5D3005FEA47,8955137,0,false, +2025-11-24T22:21:51.784505-05:00,2025-11-24T22:25:03.973845-05:00,192188.00,3634C801CD1A721F7A4298EB5146D48F9DF859B213DACC8CF3F3293322785BA9,8955137,0,false, +2025-11-24T22:21:52.780562-05:00,2025-11-24T22:25:14.523499-05:00,201741.00,1944717C6550125EC76AECAFE16E65DC9219214A0374D6539E55F9719CE0C98E,8955139,0,false, +2025-11-24T22:21:53.276391-05:00,2025-11-24T22:25:14.765018-05:00,201487.00,D74E183A811D9C23FF584E5CD089801E30A2D40AF2E1C580790F406D33328DD4,8955139,0,false, +2025-11-24T22:21:53.769558-05:00,2025-11-24T22:25:14.880136-05:00,201109.00,D04427E8C03294D914678986F0853B3EDCEC4201C6C5EF3A430E1F7FC2EE3EE9,8955139,0,false, +2025-11-24T22:21:54.27916-05:00,2025-11-24T22:25:15.94475-05:00,201664.00,C222F78AE0D49B67DC246FDC53AE16FB4CCEDFC13195C9E39D97098EAE01BFA5,8955139,0,false, +2025-11-24T22:21:54.767228-05:00,2025-11-24T22:25:16.277814-05:00,201509.00,F99E6167EA395846B04EE8D947DE66181EA7AD80AACEE5BD35B6DB3738879733,8955139,0,false, +2025-11-24T22:21:55.27339-05:00,2025-11-24T22:25:16.60701-05:00,201332.00,E7AE7DC4E623EEE38EE2A1170E2DE00BA6BBF5E30A42358994533317C558F97F,8955139,0,false, +2025-11-24T22:21:55.77596-05:00,2025-11-24T22:25:16.908814-05:00,201131.00,B341BC2EAFBBF6467F32DA2ABA087C16F767F5A1FFD88CF06F7F61B797AAB680,8955139,0,false, +2025-11-24T22:21:56.276516-05:00,2025-11-24T22:25:48.939431-05:00,232661.00,1533E605B672B4BE63BE1215DDE21E724B2DB155F2311DB059BBA87886227BB3,8955143,0,false, +2025-11-24T22:21:56.776386-05:00,2025-11-24T22:25:49.267828-05:00,232490.00,C1E0662ADB661F18F5410F4D8E475A2CA84E889F2F95A6B4A5A46E43FC9F2719,8955143,0,false, +2025-11-24T22:21:57.279309-05:00,2025-11-24T22:25:49.592874-05:00,232312.00,9584E2DE835C0D2B8F1B01FF7366D161DE6E1B3D517E2D6BC139BAE2CAC26C00,8955143,0,false, +2025-11-24T22:21:57.774804-05:00,2025-11-24T22:25:56.477388-05:00,238701.00,B67D7DDC68F09E96EC21FF265CC3FD938D0EA5E4780758C86CF3665E07048989,8955144,0,false, +2025-11-24T22:21:58.283375-05:00,2025-11-24T22:26:00.093256-05:00,241808.00,19D5F572A3C02DD75FBB6D20604D1EF9653C9C99D540BE7E0A4AB8D92D9D7625,8955145,0,false, +2025-11-24T22:21:58.779236-05:00,2025-11-24T22:26:00.920665-05:00,242139.00,B8C7194367CA2543B1C526B1C26DE6295495F27D718A8297BF07F78A9FC612C1,8955145,0,false, +2025-11-24T22:21:59.28152-05:00,2025-11-24T22:26:01.254284-05:00,241971.00,BB712099F8E01207FE6B7F16C7B463017E2B14B5CB2B738C54035781AE5D4666,8955145,0,false, +2025-11-24T22:21:59.783232-05:00,2025-11-24T22:26:01.588189-05:00,241803.00,BC351B0AC766BF4CEA96D3ACA4EA8C1694FBB9A7F133A339624294F02120F7CA,8955145,0,false, +2025-11-24T22:22:00.297655-05:00,2025-11-24T22:26:05.68611-05:00,245386.00,B1E6038E4753E1415CD8CB18BB01A503069FE90DE5D1783636E67672E172CE21,8955146,0,false, +2025-11-24T22:22:00.766919-05:00,2025-11-24T22:26:06.000592-05:00,245232.00,16875D0B1B2831BC4B6B08867F618E71F81048A1B2EBB49B998A025D74CC8F4D,8955146,0,false, +2025-11-24T22:22:01.283126-05:00,2025-11-24T22:26:06.317163-05:00,245032.00,B724C34E239EA50CE05AF0CBE475B9F7E16C609D17B6ED83065742B65557A233,8955146,0,false, +2025-11-24T22:22:01.774468-05:00,2025-11-24T22:26:06.617086-05:00,244841.00,4788B93AE62464FDBDD5D858ADC15B5AF3386A94EB2B631F3A5C823F42650BC0,8955146,0,false, +2025-11-24T22:22:02.26709-05:00,2025-11-24T22:26:44.536842-05:00,282268.00,771DCB8AF3D54C748A4087E094324FF1D99885306740E6D725F2DDDFFC08DCFB,8955152,0,false, +2025-11-24T22:22:02.7702-05:00,2025-11-24T22:26:50.8384-05:00,288066.00,C9F1D2952402C1A6240A2964922D9651DED712F5D600E0E7DBCF8380D2AF0B73,8955153,0,false, +2025-11-24T22:22:03.269254-05:00,2025-11-24T22:26:56.538174-05:00,293267.00,86E16288EAA1794153FC89B6FFDF644FD97677919B6554F83DB6F1C0580D1F44,8955154,0,false, +2025-11-24T22:22:05.275931-05:00,2025-11-24T22:27:06.338925-05:00,301061.00,F5AD10B0D289FC344E06B19CB7CEEEE1630CFEF367954AEFD0C464A030578778,8955156,0,false, +2025-11-24T22:22:05.771444-05:00,2025-11-24T22:27:06.646675-05:00,300873.00,6F396D422EF2FF9072B45B24D0FD12C9C2F1470E5D928C67E0FCEDC32F84F59B,8955156,0,false, +2025-11-24T22:22:03.779062-05:00,2025-11-24T22:27:08.373405-05:00,304592.00,09F4A203FAD77434475F284F55E5E619C3196B4131F91840C292A9409EA9F115,8955156,0,false, +2025-11-24T22:22:04.271105-05:00,2025-11-24T22:27:08.474921-05:00,304201.00,5485F512F137FFCFD5312EEAB9A3A750458E9027776D6086316D1539DE085746,8955156,0,false, +2025-11-24T22:22:04.773575-05:00,2025-11-24T22:27:09.558038-05:00,304782.00,E010293B821167995455D7A4BEAA624C8AA1C3B525F9A41FF86ABF8DF0D47176,8955156,0,false, diff --git a/pkg/user/tx_client.go b/pkg/user/tx_client.go index 97c3e0d343..fa89bc1481 100644 --- a/pkg/user/tx_client.go +++ b/pkg/user/tx_client.go @@ -31,7 +31,8 @@ import ( codectypes "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/crypto/keyring" sdktypes "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + + // sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" sdktx "github.com/cosmos/cosmos-sdk/types/tx" paramtypes "github.com/cosmos/cosmos-sdk/x/params/types/proposal" "go.opentelemetry.io/otel/attribute" @@ -385,7 +386,7 @@ func (client *TxClient) QueueBlob(ctx context.Context, resultC chan SubmissionRe // SubmitPayForBlobWithAccount forms a transaction from the provided blobs, signs it with the provided account, and submits it to the chain. // TxOptions may be provided to set the fee and gas limit. func (client *TxClient) SubmitPayForBlobWithAccount(ctx context.Context, accountName string, blobs []*share.Blob, opts ...TxOption) (*TxResponse, error) { - resp, err := client.BroadcastPayForBlobWithAccount(ctx, accountName, blobs, opts...) + resp, _, err := client.BroadcastPayForBlobWithAccount(ctx, accountName, blobs, opts...) if err != nil { return nil, err } @@ -404,23 +405,24 @@ func (client *TxClient) SubmitPayForBlobWithAccount(ctx context.Context, account // If no gas or gas price is set, it will estimate the gas and use // the max effective gas price: max(localMinGasPrice, networkMinGasPrice). func (client *TxClient) BroadcastPayForBlob(ctx context.Context, blobs []*share.Blob, opts ...TxOption) (*sdktypes.TxResponse, error) { - return client.BroadcastPayForBlobWithAccount(ctx, client.defaultAccount, blobs, opts...) + resp, _, err := client.BroadcastPayForBlobWithAccount(ctx, client.defaultAccount, blobs, opts...) + return resp, err } -func (client *TxClient) BroadcastPayForBlobWithAccount(ctx context.Context, accountName string, blobs []*share.Blob, opts ...TxOption) (*sdktypes.TxResponse, error) { +func (client *TxClient) BroadcastPayForBlobWithAccount(ctx context.Context, accountName string, blobs []*share.Blob, opts ...TxOption) (*sdktypes.TxResponse, []byte, error) { client.mtx.Lock() defer client.mtx.Unlock() if err := client.checkAccountLoaded(ctx, accountName); err != nil { - return nil, err + return nil, nil, err } acc, exists := client.signer.accounts[accountName] if !exists { - return nil, fmt.Errorf("account %s not found", accountName) + return nil, nil, fmt.Errorf("account %s not found", accountName) } signer := acc.Address().String() msg, err := blobtypes.NewMsgPayForBlobs(signer, 0, blobs...) if err != nil { - return nil, err + return nil, nil, err } gasLimit := blobtypes.DefaultEstimateGas(msg) fee := uint64(math.Ceil(appconsts.DefaultMinGasPrice * float64(gasLimit))) @@ -429,10 +431,15 @@ func (client *TxClient) BroadcastPayForBlobWithAccount(ctx context.Context, acco txBytes, _, err := client.signer.CreatePayForBlobs(accountName, blobs, opts...) if err != nil { - return nil, err + return nil, nil, err } - return client.routeTx(ctx, txBytes, accountName) + resp, err := client.routeTx(ctx, txBytes, accountName) + if err != nil { + return nil, nil, err + } + + return resp, txBytes, nil } // SubmitTx forms a transaction from the provided messages, signs it, and submits it to the chain. TxOptions @@ -485,31 +492,33 @@ func (client *TxClient) BroadcastTx(ctx context.Context, msgs []sdktypes.Msg, op } gasLimit, err = client.estimateGas(ctx, txBuilder) if err != nil { - // If not a sequence mismatch, return the error. - if !strings.Contains(err.Error(), sdkerrors.ErrWrongSequence.Error()) { - return nil, err - } + return nil, err + } + // // If not a sequence mismatch, return the error. + // if !strings.Contains(err.Error(), sdkerrors.ErrWrongSequence.Error()) { + // return nil, err + // } - // Handle the sequence mismatch path by setting the sequence to the expected sequence - // and retrying gas estimation. - parsedErr := extractSequenceError(err.Error()) + // // Handle the sequence mismatch path by setting the sequence to the expected sequence + // // and retrying gas estimation. + // parsedErr := extractSequenceError(err.Error()) - expectedSequence, err := apperrors.ParseExpectedSequence(parsedErr) - if err != nil { - return nil, fmt.Errorf("parsing sequence mismatch: %w. RawLog: %s", err, err) - } + // expectedSequence, err := apperrors.ParseExpectedSequence(parsedErr) + // if err != nil { + // return nil, fmt.Errorf("parsing sequence mismatch: %w. RawLog: %s", err, err) + // } - if err = client.signer.SetSequence(account, expectedSequence); err != nil { - return nil, fmt.Errorf("setting sequence: %w", err) - } + // if err = client.signer.SetSequence(account, expectedSequence); err != nil { + // return nil, fmt.Errorf("setting sequence: %w", err) + // } - // Retry gas estimation with the corrected sequence. - gasLimit, err = client.estimateGas(ctx, txBuilder) - if err != nil { - return nil, fmt.Errorf("retrying gas estimation: %w", err) - } - } - txBuilder.SetGasLimit(gasLimit) + // // Retry gas estimation with the corrected sequence. + // gasLimit, err = client.estimateGas(ctx, txBuilder) + // if err != nil { + // return nil, fmt.Errorf("retrying gas estimation: %w", err) + // } + // } + // txBuilder.SetGasLimit(gasLimit) } if !hasUserSetFee { @@ -541,14 +550,14 @@ func (client *TxClient) routeTx(ctx context.Context, txBytes []byte, signer stri return client.submitToMultipleConnections(ctx, txBytes, signer) } span.AddEvent("txclient: broadcasting PFB to single endpoint") - return client.submitToSingleConnection(ctx, txBytes, signer) + return client.SubmitToSingleConnectionWithoutRetry(ctx, txBytes, signer) } // submitToSingleConnection handles submission to a single connection with retry logic at sequence mismatches and sequence management func (client *TxClient) submitToSingleConnection(ctx context.Context, txBytes []byte, signer string) (*sdktypes.TxResponse, error) { span := trace.SpanFromContext(ctx) - resp, err := client.sendTxToConnection(ctx, client.conns[0], txBytes) + resp, err := client.SendTxToConnection(ctx, client.conns[0], txBytes) if err != nil { broadcastTxErr, ok := err.(*BroadcastTxError) if !ok || !apperrors.IsNonceMismatchCode(broadcastTxErr.Code) { @@ -570,7 +579,7 @@ func (client *TxClient) submitToSingleConnection(ctx context.Context, txBytes [] } span.AddEvent("txclient/submitToSingleConnection: successfully rebroadcasted tx after sequence mismatch") - return client.submitToSingleConnection(ctx, retryTxBytes, signer) + return client.SubmitToSingleConnectionWithoutRetry(ctx, retryTxBytes, signer) } // Save the sequence, signer and txBytes of the in the local txTracker // before the sequence is incremented @@ -585,8 +594,29 @@ func (client *TxClient) submitToSingleConnection(ctx context.Context, txBytes [] return resp, nil } +func (client *TxClient) SubmitToSingleConnectionWithoutRetry(ctx context.Context, txBytes []byte, signer string) (*sdktypes.TxResponse, error) { + // span := trace.SpanFromContext(ctx) + + resp, err := client.SendTxToConnection(ctx, client.conns[0], txBytes) + if err != nil { + return nil, err + } + + // Save the sequence, signer and txBytes of the in the local txTracker + // before the sequence is incremented + sequence := client.signer.Account(signer).Sequence() + client.TxTracker.trackTransaction(signer, sequence, resp.TxHash, txBytes) + + // Increment sequence after successful submission + if err := client.signer.IncrementSequence(signer); err != nil { + return nil, fmt.Errorf("error incrementing sequence: %w", err) + } + + return resp, nil +} + // sendTxToConnection broadcasts a transaction to the chain and returns the response. -func (client *TxClient) sendTxToConnection(ctx context.Context, conn *grpc.ClientConn, txBytes []byte) (*sdktypes.TxResponse, error) { +func (client *TxClient) SendTxToConnection(ctx context.Context, conn *grpc.ClientConn, txBytes []byte) (*sdktypes.TxResponse, error) { span := trace.SpanFromContext(ctx) resp, err := sdktx.NewServiceClient(conn).BroadcastTx( @@ -687,7 +717,7 @@ func (client *TxClient) submitToMultipleConnections(ctx context.Context, txBytes go func(conn *grpc.ClientConn) { defer wg.Done() - resp, err := client.sendTxToConnection(ctx, conn, txBytes) + resp, err := client.SendTxToConnection(ctx, conn, txBytes) if err != nil { errCh <- err return @@ -788,7 +818,7 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon )) // If we're not already tracking eviction timeout, try to resubmit - _, err := client.sendTxToConnection(ctx, client.conns[0], txBytes) + _, err := client.SendTxToConnection(ctx, client.conns[0], txBytes) if err != nil { // Check if the error is a broadcast tx error _, ok := err.(*BroadcastTxError) diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index 4ecda959e6..22334e19dd 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -3,6 +3,8 @@ package v2 import ( "context" "fmt" + "regexp" + "strconv" "strings" "sync" "time" @@ -47,6 +49,13 @@ type sequentialQueue struct { // Track last confirmed sequence for rollback logic lastConfirmedSeq uint64 + + // Submission tracking metrics + newBroadcastCount uint64 // Count of new transaction broadcasts + resubmitCount uint64 // Count of resubmissions (evicted txs) + resignCount uint64 // Count of resignations (rejected txs) + lastMetricsLog time.Time // Last time we logged metrics + metricsStartTime time.Time // Start time for rate calculation } // queuedTx represents a transaction in the queue (from submission to confirmation) @@ -57,11 +66,12 @@ type queuedTx struct { resultsC chan SequentialSubmissionResult // Set after broadcast - txHash string // Empty until broadcast - txBytes []byte // Set after broadcast, used for eviction resubmission - sequence uint64 // Set after broadcast - submittedAt time.Time // Set after broadcast - shouldResign bool // Set after broadcast + txHash string // Empty until broadcast + txBytes []byte // Set after broadcast, used for eviction resubmission + sequence uint64 // Set after broadcast + submittedAt time.Time // Set after broadcast + shouldResign bool // Set after broadcast + isResubmitting bool // True if transaction is currently being resubmitted (prevents duplicates) } const ( @@ -74,15 +84,18 @@ func newSequentialQueue(client *TxClient, accountName string, pollTime time.Dura } ctx, cancel := context.WithCancel(context.Background()) + now := time.Now() q := &sequentialQueue{ - client: client, - accountName: accountName, - pollTime: pollTime, - ctx: ctx, - cancel: cancel, - queue: make([]*queuedTx, 0, defaultSequentialQueueSize), - ResignChan: make(chan *queuedTx, 10), // Buffered channel for resign requests - ResubmitChan: make(chan *queuedTx, 10), // Buffered channel for resubmit requests + client: client, + accountName: accountName, + pollTime: pollTime, + ctx: ctx, + cancel: cancel, + queue: make([]*queuedTx, 0, defaultSequentialQueueSize), + ResignChan: make(chan *queuedTx, 50), // Buffered channel for resign requests + ResubmitChan: make(chan *queuedTx, 200), // Buffered channel for resubmit requests (large to prevent blocking) + lastMetricsLog: now, + metricsStartTime: now, } return q } @@ -123,6 +136,7 @@ func (q *sequentialQueue) GetQueueSize() int { // processNextTx signs and broadcasts the next unbroadcast transaction in queue func (q *sequentialQueue) processNextTx() { + // Find first unbroadcast transaction (txHash is empty) fmt.Println("Processing next tx") var qTx *queuedTx @@ -139,7 +153,7 @@ func (q *sequentialQueue) processNextTx() { return } - resp, err := q.client.BroadcastPayForBlobWithoutRetry( + resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( q.ctx, q.accountName, qTx.blobs, @@ -173,18 +187,22 @@ func (q *sequentialQueue) processNextTx() { defer q.mu.Unlock() // Broadcast successful - mark as broadcast in queue sequence := q.client.Signer().Account(q.accountName).Sequence() - txBytes := q.client.TxClient.TxTracker.GetTxBytes(q.accountName, sequence) qTx.txHash = resp.TxHash qTx.txBytes = txBytes qTx.sequence = sequence qTx.submittedAt = time.Now() + + // Track submission metrics + q.newBroadcastCount++ + q.logSubmissionMetrics() + fmt.Println("Broadcast successful - marking as broadcast in queue") } // monitorLoop periodically checks the status of broadcast transactions func (q *sequentialQueue) monitorLoop() { - ticker := time.NewTicker(1 * time.Second) + ticker := time.NewTicker(3 * time.Second) defer ticker.Stop() for { @@ -199,8 +217,8 @@ func (q *sequentialQueue) monitorLoop() { // coordinate coordinates transaction submission with confirmation func (q *sequentialQueue) coordinate() { - ticker := time.NewTicker(time.Second) - defer ticker.Stop() + // ticker := time.NewTicker(time.Millisecond * 500) //TODO: understand if this acceptable cadence + // defer ticker.Stop() for { select { @@ -210,9 +228,8 @@ func (q *sequentialQueue) coordinate() { fmt.Println("Resigning rejected tx") q.ResignRejected() case qTx := <-q.ResubmitChan: - fmt.Println("Resubmitting evicted tx") q.ResubmitEvicted(qTx) - case <-ticker.C: + default: q.processNextTx() } } @@ -233,7 +250,7 @@ func (q *sequentialQueue) ResignRejected() { for _, qTx := range txsToResign { if qTx.shouldResign { // resign the tx - resp, err := q.client.BroadcastPayForBlobWithoutRetry( + resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( q.ctx, q.accountName, qTx.blobs, @@ -250,13 +267,15 @@ func (q *sequentialQueue) ResignRejected() { q.removeFromQueue(qTx) return } + q.mu.Lock() sequence := q.client.Signer().Account(q.accountName).Sequence() - txBytes := q.client.TxClient.TxTracker.GetTxBytes(q.accountName, sequence) - qTx.txHash = resp.TxHash qTx.txBytes = txBytes qTx.sequence = sequence qTx.shouldResign = false + q.resignCount++ + q.logSubmissionMetrics() + q.mu.Unlock() fmt.Printf("Resigned and submitted tx successfully: %s\n", resp.TxHash) } } @@ -264,34 +283,85 @@ func (q *sequentialQueue) ResignRejected() { // TODO: come back to this and see if it makes sense // func (q *sequentialQueue) setTxInfo(qTx *queuedTx, resp *sdktypes.TxResponse, txBytes []byte, sequence uint64) { -// q.mu.Lock() -// defer q.mu.Unlock() +// q.mu.Lock() +// defer q.mu.Unlock() -// qTx.txHash = resp.TxHash -// qTx.txBytes = txBytes -// qTx.sequence = sequence -// qTx.shouldResign = false +// qTx.txHash = resp.TxHash +// qTx.txBytes = txBytes +// qTx.sequence = sequence +// qTx.shouldResign = false // } func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { - fmt.Println("Resubmitting evicted tx") + fmt.Printf("Resubmitting evicted tx with hash %s and sequence %d\n", qTx.txHash[:16], qTx.sequence) q.mu.RLock() txBytes := qTx.txBytes q.mu.RUnlock() // check if the tx needs to be resubmitted - resubmitResp, err := q.client.ResubmitTxBytes(q.ctx, txBytes) + resubmitResp, err := q.client.SendTxToConnection(q.ctx, q.client.GetGRPCConnection(), txBytes) if err != nil || resubmitResp.Code != 0 { - // send error and remove from queue + // Check if this is a sequence mismatch + // if IsSequenceMismatchError(err) { + // // Sequence mismatch means blockchain is at earlier sequence than this tx + // // All txs from blockchain sequence onwards are stale - remove them all at once + // expectedSeq := parseExpectedSequence(err.Error()) + // fmt.Printf("Sequence mismatch: blockchain at %d but tx at %d. Removing all stale txs >= %d\n", + // expectedSeq, qTx.sequence, expectedSeq) + + // // Collect all transactions with sequence >= expectedSeq + // q.mu.RLock() + // var staleTxs []*queuedTx + // for _, tx := range q.queue { + // if tx.sequence >= expectedSeq { + // staleTxs = append(staleTxs, tx) + // } + // } + // q.mu.RUnlock() + + // // check the first tx to see if it was evicted then we can be sure that all txs are evicted + // TxClient := tx.NewTxClient(q.client.GetGRPCConnection()) + // statusResp, err := TxClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) + // if err != nil { + // fmt.Printf("Failed to check status of expected sequence tx: %v\n", err) + // // Reset flag and return - let next poll cycle handle it + // q.mu.Lock() + // qTx.isResubmitting = false + // q.mu.Unlock() + // return + // } + // // lets just log for now + // fmt.Printf("TX STATUS OF EXPECTED SEQUENCE %d: %s\n", expectedSeq, statusResp.Status) + + // if statusResp.Status == core.TxStatusEvicted { + // // All stale txs are evicted. Reset current tx flag and return. + // // Next poll cycle will scan from beginning and handle all evicted txs properly. + // fmt.Printf("Confirmed: all txs from seq %d onwards are evicted. Resetting flag for next poll cycle.\n", expectedSeq) + // q.mu.Lock() + // qTx.isResubmitting = false + // q.mu.Unlock() + // return + // } + // } + select { case qTx.resultsC <- SequentialSubmissionResult{ - Error: fmt.Errorf("evicted and failed to resubmit: %w", err), + Error: fmt.Errorf("evicted and failed to resubmit with hash %s: %w", qTx.txHash[:16], err), }: case <-q.ctx.Done(): } + // send error and remove from queue q.removeFromQueue(qTx) return } + + // Successful resubmission - reset flag and track metrics + q.mu.Lock() + qTx.isResubmitting = false + q.resubmitCount++ + q.logSubmissionMetrics() + q.mu.Unlock() + fmt.Printf("Successfully resubmitted tx %s\n", qTx.txHash[:16]) } // checkBroadcastTransactions checks status of all broadcast transactions @@ -305,8 +375,11 @@ func (q *sequentialQueue) checkBroadcastTransactions() { broadcastTxs = append(broadcastTxs, tx) } } + totalQueueSize := len(q.queue) q.mu.RUnlock() + fmt.Printf("Total queue size: %d, Broadcast txs: %d\n", totalQueueSize, len(broadcastTxs)) + if len(broadcastTxs) == 0 { return } @@ -317,21 +390,74 @@ func (q *sequentialQueue) checkBroadcastTransactions() { for _, qTx := range broadcastTxs { statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) if err != nil { - continue // Network error, try again later + qTx.resultsC <- SequentialSubmissionResult{ + Error: fmt.Errorf("tx status check failed: %w", err), + } } + fmt.Printf("Tx %s status: %s\n", qTx.txHash[:16], statusResp.Status) + switch statusResp.Status { case core.TxStatusCommitted: q.handleCommitted(qTx, statusResp) case core.TxStatusEvicted: - fmt.Println("Handling evicted tx") - q.ResubmitChan <- qTx + // Found an evicted tx - scan entire queue from beginning to find all evicted txs + fmt.Printf("Detected evicted tx with sequence %d - scanning queue for all evictions", qTx.sequence) + // check if the tx is already being resubmitted + q.mu.RLock() + alreadyResubmitting := qTx.isResubmitting + q.mu.RUnlock() + if alreadyResubmitting { + fmt.Printf("Tx %s is already being resubmitted - skipping\n", qTx.txHash[:16]) + continue + } + q.mu.RLock() + var potentialEvictions []*queuedTx + for _, tx := range q.queue { + if tx.txHash != "" && !tx.isResubmitting { + potentialEvictions = append(potentialEvictions, tx) + } + } + q.mu.RUnlock() + + // Check status of each transaction in order to find first evicted one since we might have received evictions while + // already processing the queue + // Collect ALL evicted transactions first + var evictedTxs []*queuedTx + for _, evictedTx := range potentialEvictions { + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: evictedTx.txHash}) + if err != nil { + continue + } + if statusResp.Status == core.TxStatusEvicted { + evictedTxs = append(evictedTxs, evictedTx) + } + } + + // Now send them in order with proper locking + for _, evictedTx := range evictedTxs { + q.mu.Lock() + if !evictedTx.isResubmitting { + evictedTx.isResubmitting = true + q.mu.Unlock() + fmt.Printf("Sending evicted tx (seq %d) to resubmit channel\n", evictedTx.sequence) + q.ResubmitChan <- evictedTx + } else { + q.mu.Unlock() + fmt.Printf("Skipping evicted tx (seq %d) - already being resubmitted\n", evictedTx.sequence) + } + } + return // Skip processing remaining txs in this poll cycle case core.TxStatusRejected: q.handleRejected(qTx, statusResp, txClient) } } } +func (q *sequentialQueue) handleEvicted(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { + // TODO: move evicted logic here +} + // handleCommitted processes a confirmed transaction func (q *sequentialQueue) handleCommitted(qTx *queuedTx, statusResp *tx.TxStatusResponse) { fmt.Println("Handling confirmed tx") @@ -462,3 +588,40 @@ func isSequenceMismatchRejection(errMsg string) bool { return strings.Contains(errMsg, "account sequence mismatch") || strings.Contains(errMsg, "incorrect account sequence") } + +// parseExpectedSequence extracts the expected sequence number from error message +// e.g., "account sequence mismatch, expected 9727, got 9811" -> returns 9727 +func parseExpectedSequence(errMsg string) uint64 { + // Look for "expected " + re := regexp.MustCompile(`expected (\d+)`) + matches := re.FindStringSubmatch(errMsg) + if len(matches) >= 2 { + if seq, err := strconv.ParseUint(matches[1], 10, 64); err == nil { + return seq + } + } + return 0 +} + +// logSubmissionMetrics logs submission statistics every 30 seconds +// Note: Caller must hold q.mu lock +func (q *sequentialQueue) logSubmissionMetrics() { + now := time.Now() + if now.Sub(q.lastMetricsLog) < 30*time.Second { + return + } + + elapsed := now.Sub(q.metricsStartTime).Seconds() + if elapsed < 1 { + return + } + + totalSubmissions := q.newBroadcastCount + q.resubmitCount + q.resignCount + submissionsPerSec := float64(totalSubmissions) / elapsed + + fmt.Printf("[METRICS] Total submissions: %d (new: %d, resubmit: %d, resign: %d) | Rate: %.2f tx/sec | Queue size: %d\n", + totalSubmissions, q.newBroadcastCount, q.resubmitCount, q.resignCount, + submissionsPerSec, len(q.queue)) + + q.lastMetricsLog = now +} diff --git a/pkg/user/v2/tx_client.go b/pkg/user/v2/tx_client.go index 06e1dac9c3..a99a15d219 100644 --- a/pkg/user/v2/tx_client.go +++ b/pkg/user/v2/tx_client.go @@ -167,7 +167,7 @@ func (c *TxClient) StartSequentialQueueWithPollTime(ctx context.Context, account } queue := newSequentialQueue(c, accountName, pollTime) - queue.start(); + queue.start() c.sequentialQueues[accountName] = queue return nil @@ -220,7 +220,6 @@ func (c *TxClient) SubmitPFBToSequentialQueueWithAccount(ctx context.Context, ac Ctx: ctx, ResultsC: resultsC, } - fmt.Println("Submitting job to sequential queue") queue.submitJob(job) // Wait for result @@ -249,22 +248,15 @@ func (c *TxClient) GetSequentialQueueSize(accountName string) (int, error) { } // BroadcastPayForBlobWithoutRetry broadcasts a PayForBlob transaction without automatic retry logic. -func (c *TxClient) BroadcastPayForBlobWithoutRetry(ctx context.Context, accountName string, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, error) { +// Returns the transaction response and the raw transaction bytes. +func (c *TxClient) BroadcastPayForBlobWithoutRetry(ctx context.Context, accountName string, blobs []*share.Blob, opts ...user.TxOption) (*sdktypes.TxResponse, []byte, error) { // Use BroadcastPayForBlobWithAccount but without confirmation - resp, err := c.TxClient.BroadcastPayForBlobWithAccount(ctx, accountName, blobs, opts...) + resp, txBytes, err := c.TxClient.BroadcastPayForBlobWithAccount(ctx, accountName, blobs, opts...) if err != nil { - return nil, err + return nil, nil, err } - return &sdktypes.TxResponse{ - Height: resp.Height, - TxHash: resp.TxHash, - Code: resp.Code, - Codespace: resp.Codespace, - GasWanted: resp.GasWanted, - GasUsed: resp.GasUsed, - Signers: resp.Signers, - }, nil + return resp, txBytes, nil } // ResubmitTxBytes resubmits a transaction using pre-signed bytes without retry logic diff --git a/tools/latency-monitor/main.go b/tools/latency-monitor/main.go index d7296b0d0e..302033b5c3 100644 --- a/tools/latency-monitor/main.go +++ b/tools/latency-monitor/main.go @@ -16,6 +16,7 @@ import ( "github.com/celestiaorg/celestia-app/v6/app" "github.com/celestiaorg/celestia-app/v6/app/encoding" "github.com/celestiaorg/celestia-app/v6/pkg/user" + v2 "github.com/celestiaorg/celestia-app/v6/pkg/user/v2" "github.com/celestiaorg/go-square/v3/share" "github.com/cosmos/cosmos-sdk/crypto/keyring" "github.com/spf13/cobra" @@ -157,12 +158,19 @@ func monitorLatency( if accountName != "" { opts = append(opts, user.WithDefaultAccount(accountName)) } - txClient, err := user.SetupTxClient(ctx, kr, grpcConn, encCfg, opts...) + txClient, err := v2.SetupTxClient(ctx, kr, grpcConn, encCfg, opts...) if err != nil { return fmt.Errorf("failed to create tx client: %w", err) } fmt.Printf("Using account: %s\n", txClient.DefaultAccountName()) + fmt.Println("Sequential queue started for transaction submission") + + // Ensure sequential queue is stopped on exit + defer func() { + fmt.Println("Stopping sequential queue...") + txClient.StopAllSequentialQueues() + }() fmt.Println("Submitting transactions...") @@ -207,63 +215,58 @@ func monitorLatency( submitTime := time.Now() - // Broadcast transaction without waiting for confirmation - resp, err := txClient.BroadcastPayForBlob(ctx, []*share.Blob{blob}) - if err != nil { - fmt.Printf("Failed to broadcast tx: %v\n", err) - continue - } - - fmt.Printf("[SUBMIT] tx=%s size=%d bytes time=%s\n", - resp.TxHash[:16], randomSize, submitTime.Format("15:04:05.000")) - - if disableMetrics { - continue - } - - // Launch background goroutine to confirm the transaction - go func(txHash string, submitTime time.Time) { - confirmed, err := txClient.ConfirmTx(ctx, txHash) + // Submit to sequential queue (handles both broadcast and confirmation) + go func(submitTime time.Time, blobData []*share.Blob, size int) { + resp, err := txClient.SubmitPFBToSequentialQueue(ctx, blobData) if err != nil { if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) { - fmt.Printf("[CANCELLED] tx=%s context closed before confirmation\n", txHash[:16]) + fmt.Printf("[CANCELLED] context closed before submission\n") return } - resultsMux.Lock() - // Track failed confirmation - fmt.Printf("[FAILED] tx=%s error=%v\n", txHash[:16], err) - results = append(results, txResult{ - submitTime: submitTime, - commitTime: time.Now(), - latency: 0, - txHash: txHash, - code: 0, - height: 0, - failed: true, - errorMsg: err.Error(), - }) - resultsMux.Unlock() + if !disableMetrics { + resultsMux.Lock() + // Track failed submission/confirmation + fmt.Printf("[FAILED] error=%v\n", err) + results = append(results, txResult{ + submitTime: submitTime, + commitTime: time.Now(), + latency: 0, + txHash: "", + code: 0, + height: 0, + failed: true, + errorMsg: err.Error(), + }) + resultsMux.Unlock() + } + return + } + + fmt.Printf("[SUBMIT] tx=%s size=%d bytes time=%s\n", + resp.TxHash[:16], size, submitTime.Format("15:04:05.000")) + + if disableMetrics { return } - resultsMux.Lock() // Track successful confirmation commitTime := time.Now() latency := commitTime.Sub(submitTime) + resultsMux.Lock() fmt.Printf("[CONFIRM] tx=%s height=%d latency=%dms code=%d time=%s\n", - confirmed.TxHash[:16], confirmed.Height, latency.Milliseconds(), confirmed.Code, commitTime.Format("15:04:05.000")) + resp.TxHash[:16], resp.Height, latency.Milliseconds(), resp.Code, commitTime.Format("15:04:05.000")) results = append(results, txResult{ submitTime: submitTime, commitTime: commitTime, latency: latency, - txHash: confirmed.TxHash, - code: confirmed.Code, - height: confirmed.Height, + txHash: resp.TxHash, + code: resp.Code, + height: resp.Height, failed: false, errorMsg: "", }) resultsMux.Unlock() - }(resp.TxHash, submitTime) + }(submitTime, []*share.Blob{blob}, randomSize) } } } From df62e6410fd541d028e19e5993a511c279c44652 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Mon, 24 Nov 2025 23:31:06 -0500 Subject: [PATCH 05/12] chore: add time tracking --- latency_results.csv | 97 ++++----------------------------- pkg/user/v2/sequential_queue.go | 67 ++++++++++++----------- 2 files changed, 48 insertions(+), 116 deletions(-) diff --git a/latency_results.csv b/latency_results.csv index e0824d44c5..370da70a6c 100644 --- a/latency_results.csv +++ b/latency_results.csv @@ -1,86 +1,13 @@ Submit Time,Commit Time,Latency (ms),Tx Hash,Height,Code,Failed,Error -2025-11-24T22:21:23.771767-05:00,2025-11-24T22:21:35.988516-05:00,12216.00,7A9B74F21EB480ADD5FAB98867C2D294F603282D533714086C56372A120AB76B,8955106,0,false, -2025-11-24T22:21:24.270797-05:00,2025-11-24T22:21:41.538142-05:00,17267.00,003DE58F51AC96CCAAD8A3C91D2D339C792FA32ADDDD18D796E2D3EFD92CA069,8955107,0,false, -2025-11-24T22:21:24.77405-05:00,2025-11-24T22:21:41.850616-05:00,17076.00,5F538FAD784196B377D3E77155495AA303A278AC215EC32D75FF7BE9C29DDBBF,8955107,0,false, -2025-11-24T22:21:25.266592-05:00,2025-11-24T22:21:41.960953-05:00,16694.00,ECEBF9035BE7F10148493B6A74D33DB2AB315D2B835567EDA08C35F406BF5C66,8955107,0,false, -2025-11-24T22:21:25.767846-05:00,2025-11-24T22:21:42.111902-05:00,16343.00,6D7A85F359F3326BD1781872793541503040B66E1D3D1344EF141059D3F5B28B,8955107,0,false, -2025-11-24T22:21:26.26837-05:00,2025-11-24T22:21:42.224576-05:00,15956.00,4447C28DFFCF8F7FCAFE4B233CA475856C50C532D65FFF5BEA9A4582B7604C4B,8955107,0,false, -2025-11-24T22:21:26.777362-05:00,2025-11-24T22:21:53.546686-05:00,26769.00,00353954058BAF2489F5F4033D2A2709FADB03AEDFBD6890C6003791AD621FFF,8955109,0,false, -2025-11-24T22:21:27.267245-05:00,2025-11-24T22:22:00.454188-05:00,33186.00,0C24802E396EA61C99BA62D5D092C9660A967165ABF0EB2CD33DF656B1BE5986,8955110,0,false, -2025-11-24T22:21:27.780155-05:00,2025-11-24T22:22:06.283226-05:00,38502.00,3AF8536E7F827D913EC9A8BFD44665749C674B10A10B81F9A3A90688E5F6D0A0,8955111,0,false, -2025-11-24T22:21:28.274506-05:00,2025-11-24T22:22:06.529445-05:00,38254.00,5F26A3200AC794AEF161D65C5001D8496134D9751FA04917C47D582B7915E028,8955111,0,false, -2025-11-24T22:21:28.773141-05:00,2025-11-24T22:22:06.849261-05:00,38075.00,7F10CCFBDB5263FE9DF2B8B1E47FF16151A424C4576DB6D79D1D9DCE49111607,8955111,0,false, -2025-11-24T22:21:29.269952-05:00,2025-11-24T22:22:07.173533-05:00,37903.00,DB52C95C8027EB7E7B50051463030E7FBF7528D4B3F79A1D27AC393FB15ABA6F,8955111,0,false, -2025-11-24T22:21:29.780953-05:00,2025-11-24T22:22:11.549672-05:00,41768.00,5BE0445105829A173BDE8249E3BCCA4C48B1920F53C3778FE8FD9E554E88E3CB,8955112,0,false, -2025-11-24T22:21:30.279436-05:00,2025-11-24T22:22:23.503275-05:00,53223.00,0A12A2B2B1477D9070D4CE6B391F5CCA9D9CA1AB357319642A8A57D9953D73ED,8955114,0,false, -2025-11-24T22:21:30.768476-05:00,2025-11-24T22:22:23.91719-05:00,53148.00,FCB3D29C7288047B1E1B6CAB68E258CEA7361EF12DE377A4295AEC4D2A922BF3,8955114,0,false, -2025-11-24T22:21:31.273826-05:00,2025-11-24T22:22:24.260286-05:00,52986.00,16658D0C195473BFFB9A5CC6C2FC72D3A926CA92962402823DEF71E63572E37E,8955114,0,false, -2025-11-24T22:21:31.777077-05:00,2025-11-24T22:22:24.579931-05:00,52802.00,684D6D2A7BE8B3A3AE6229CBBCE36C929B549E20A875F96B636D93CBAD68CD94,8955114,0,false, -2025-11-24T22:21:34.26893-05:00,2025-11-24T22:22:44.219936-05:00,69950.00,EBDCB73DDFD0189FE48064BD12B2A2AD67D64BADF0619F05E976B453A1EA8F4A,8955116,0,false, -2025-11-24T22:21:34.782483-05:00,2025-11-24T22:22:45.157658-05:00,70374.00,A6DDBE82D1CEF47ADED3A8E91A3139908BCC0191DDC96594039E98390850E1E5,8955116,0,false, -2025-11-24T22:21:35.275238-05:00,2025-11-24T22:22:45.794181-05:00,70518.00,07C0FF570F98EEC1D6F8256C68B0E374261A82932EAF87B2DEEF2BB61B8948D5,8955116,0,false, -2025-11-24T22:21:32.281308-05:00,2025-11-24T22:22:47.782354-05:00,75500.00,1BA714E90CE90C72988841AC86744F6E691FCEA99DAE71E3960BA1743CD7C335,8955116,0,false, -2025-11-24T22:21:32.776912-05:00,2025-11-24T22:22:48.283223-05:00,75505.00,49A5DF699EA9C89C755BFD0367A155B5B3762740F89E10523ED5B0DCB49DA2A3,8955116,0,false, -2025-11-24T22:21:33.269645-05:00,2025-11-24T22:22:48.6106-05:00,75340.00,AD448DD0E47521242A6A16F2CA9D32D54AC4923284148E58D1F4B0B708670460,8955116,0,false, -2025-11-24T22:21:33.770702-05:00,2025-11-24T22:22:49.020195-05:00,75249.00,C3DC304E49346030929A12E6A310D5577E6445622AF418CBAE1AD51FCE7EB490,8955116,0,false, -2025-11-24T22:21:37.270098-05:00,2025-11-24T22:22:50.398071-05:00,73127.00,853C798B8F8DB046EB35A16D7D7C7FDF2B485010DFC38E5DCB8FFE50EE1D1B87,8955117,0,false, -2025-11-24T22:21:35.770066-05:00,2025-11-24T22:22:50.716655-05:00,74946.00,7F7B1AF4520593C3BC6A80DD75CD911FAB3DC897A3D01F08073B55A843597B88,8955117,0,false, -2025-11-24T22:21:36.267245-05:00,2025-11-24T22:22:51.097941-05:00,74830.00,8B05A1526FFC181B95F69C70451B3CEDDE664DE283E54C252CD93C8C12CCB8FD,8955117,0,false, -2025-11-24T22:21:36.77219-05:00,2025-11-24T22:22:51.417115-05:00,74644.00,9F8350DC9A5D52AB4B804F8B3EE199C5853CD0980844B6E71C62EE1979C60330,8955117,0,false, -2025-11-24T22:21:37.777745-05:00,2025-11-24T22:23:11.495606-05:00,93717.00,D64C31B619020C26864077E6B2C823F6E99FCB555F11D21734BA1B1489847C17,8955120,0,false, -2025-11-24T22:21:38.283-05:00,2025-11-24T22:23:11.803265-05:00,93519.00,96D4D70126595FCA57812F92E54DC21BF7F7FD9032C70EF43365647559BA92DE,8955120,0,false, -2025-11-24T22:21:38.767909-05:00,2025-11-24T22:23:15.226668-05:00,96458.00,70A0214BC77ECD4224B5720B18B9DF891F26FCD1C9937839638D5DF75A66FBB8,8955121,0,false, -2025-11-24T22:21:39.280079-05:00,2025-11-24T22:23:28.192018-05:00,108911.00,8E7D0BE9F8DF07DB4492F54A292108DF834F370971CEB48AD7C6A5857A561845,8955123,0,false, -2025-11-24T22:21:39.766193-05:00,2025-11-24T22:23:28.511461-05:00,108744.00,7F6DE91C2BCCEC533F06AB1D9B144C3AC16FA3348FF6F2033FD807035D2C80A6,8955123,0,false, -2025-11-24T22:21:40.27605-05:00,2025-11-24T22:23:41.529657-05:00,121252.00,D51D283FA9EE7E4B79697DF521EB4E431B3C33C99FB62FEAB4D55E61DF521D4C,8955125,0,false, -2025-11-24T22:21:40.766991-05:00,2025-11-24T22:23:41.813374-05:00,121045.00,64BC35E46898E91B80BE77DF5A87D73F237E5BC8668262E3995473AE7F4A9436,8955125,0,false, -2025-11-24T22:21:41.273143-05:00,2025-11-24T22:23:42.062752-05:00,120788.00,153A2E0C00648F87C2B0C92716A4F02F32FFAAE2C2BC7E4D0A740AE4930472A5,8955125,0,false, -2025-11-24T22:21:41.78573-05:00,2025-11-24T22:23:45.424815-05:00,123638.00,1070E416C0A87B850B97F190973A773B27EA6AA87FB0E313A89C5CCEEA2D297F,8955126,0,false, -2025-11-24T22:21:42.271998-05:00,2025-11-24T22:23:51.883117-05:00,129610.00,4C2E3F9D56C227954389158D6C43B3738E56DF491DC39F15F07BD95A52909AB7,8955127,0,false, -2025-11-24T22:21:42.775749-05:00,2025-11-24T22:24:09.382422-05:00,146605.00,AA0085458A41F405CBA9DD372EBB99B8B6C51BC274BC526BD8357A6319661E0C,8955128,0,false, -2025-11-24T22:21:43.285607-05:00,2025-11-24T22:24:11.356963-05:00,148070.00,165FB8ACA92E48278D30B354469A36D6E893211EDA46A63EB28DFF25524FBE77,8955128,0,false, -2025-11-24T22:21:43.783332-05:00,2025-11-24T22:24:12.291446-05:00,148507.00,8110066814826BC3CFFACFB52A831DD2F5561E5406A69F6AF1FFE295BEE48004,8955128,0,false, -2025-11-24T22:21:44.271932-05:00,2025-11-24T22:24:12.992785-05:00,148719.00,196D18EBCBAB18B757F42C1057D947C8CF72D30CFBC45D6813F2570FFF9F2356,8955129,0,false, -2025-11-24T22:21:44.777277-05:00,2025-11-24T22:24:13.506684-05:00,148728.00,3D19310DB64808BC26DD1F577423FF280BC9A8C3EE1DE5899ABC74BFAB93C9C4,8955129,0,false, -2025-11-24T22:21:45.26798-05:00,2025-11-24T22:24:13.98714-05:00,148718.00,863C3914F9F43E884D2D3CD29F050F95E33A1526A2EA34109CE31692EC73239F,8955129,0,false, -2025-11-24T22:21:45.776116-05:00,2025-11-24T22:24:14.39538-05:00,148618.00,7F155EBA9E8B2099591AF0412FE09A2D3CA6F22462BE4211A5F93882EDC30F36,8955129,0,false, -2025-11-24T22:21:46.282655-05:00,2025-11-24T22:24:32.394574-05:00,166110.00,D87305E2DAB16F1306F454DB90C367010D1BE2DCA3ACE881D377B9575F3B1AFA,8955132,0,false, -2025-11-24T22:21:46.772217-05:00,2025-11-24T22:24:32.694008-05:00,165920.00,F0D66B11E50293F0816F155E3EBBC637FE5A8E29B9A90AF72A27D32C36EDFB2C,8955132,0,false, -2025-11-24T22:21:47.280043-05:00,2025-11-24T22:24:32.949745-05:00,165668.00,8EB0A7A303B5320E896182C2CA59198D8929F03F2026F83C75E8CB342C8BB026,8955132,0,false, -2025-11-24T22:21:47.783358-05:00,2025-11-24T22:24:33.260243-05:00,165475.00,19906B99A42B04641CCA73D211E8A9941522F17D616F9AF02518C8A3A2AC4DCA,8955132,0,false, -2025-11-24T22:21:48.267669-05:00,2025-11-24T22:24:33.559396-05:00,165290.00,20877311D734BD2785C61AEDFE9CC422D263BE171B8238853B1FA655274677AB,8955132,0,false, -2025-11-24T22:21:48.774203-05:00,2025-11-24T22:24:33.855918-05:00,165080.00,520E7CCD899CF3A4E538CD7DA2816189DD1439E71330B1AAD76906E73ED10E6F,8955132,0,false, -2025-11-24T22:21:49.281341-05:00,2025-11-24T22:24:50.612438-05:00,181330.00,A2DAA0BDE252EF5B3613C620EA00EE490801D88190FBDD0BC44A7B8E94FA795A,8955135,0,false, -2025-11-24T22:21:49.782419-05:00,2025-11-24T22:24:50.938513-05:00,181154.00,CEBD6F6B35AACB1E2F8F3608D7113103C766FE751C58BEA815E201F58FB20199,8955135,0,false, -2025-11-24T22:21:50.293966-05:00,2025-11-24T22:24:51.045694-05:00,180750.00,51A2B793F5DA651B3048BE3B46A37FF8CF03AE9F8D946230E3A04C214090A9A2,8955135,0,false, -2025-11-24T22:21:50.775025-05:00,2025-11-24T22:24:51.156326-05:00,180380.00,1B18E52DA41968DE60480DF0E271C269EB18C35B78EBFA18CEB5DE34B1E800F0,8955135,0,false, -2025-11-24T22:21:51.274822-05:00,2025-11-24T22:24:51.43435-05:00,180158.00,C1DFAC681C48B0E7145FCC64252A8F09D805623BE0DF68B9DCF8261F59D1CBF2,8955135,0,false, -2025-11-24T22:21:52.27148-05:00,2025-11-24T22:25:01.713911-05:00,189441.00,EC9E9C3DBE29A79B63CF5F946AC0BDC98318816C6835888C213FC5D3005FEA47,8955137,0,false, -2025-11-24T22:21:51.784505-05:00,2025-11-24T22:25:03.973845-05:00,192188.00,3634C801CD1A721F7A4298EB5146D48F9DF859B213DACC8CF3F3293322785BA9,8955137,0,false, -2025-11-24T22:21:52.780562-05:00,2025-11-24T22:25:14.523499-05:00,201741.00,1944717C6550125EC76AECAFE16E65DC9219214A0374D6539E55F9719CE0C98E,8955139,0,false, -2025-11-24T22:21:53.276391-05:00,2025-11-24T22:25:14.765018-05:00,201487.00,D74E183A811D9C23FF584E5CD089801E30A2D40AF2E1C580790F406D33328DD4,8955139,0,false, -2025-11-24T22:21:53.769558-05:00,2025-11-24T22:25:14.880136-05:00,201109.00,D04427E8C03294D914678986F0853B3EDCEC4201C6C5EF3A430E1F7FC2EE3EE9,8955139,0,false, -2025-11-24T22:21:54.27916-05:00,2025-11-24T22:25:15.94475-05:00,201664.00,C222F78AE0D49B67DC246FDC53AE16FB4CCEDFC13195C9E39D97098EAE01BFA5,8955139,0,false, -2025-11-24T22:21:54.767228-05:00,2025-11-24T22:25:16.277814-05:00,201509.00,F99E6167EA395846B04EE8D947DE66181EA7AD80AACEE5BD35B6DB3738879733,8955139,0,false, -2025-11-24T22:21:55.27339-05:00,2025-11-24T22:25:16.60701-05:00,201332.00,E7AE7DC4E623EEE38EE2A1170E2DE00BA6BBF5E30A42358994533317C558F97F,8955139,0,false, -2025-11-24T22:21:55.77596-05:00,2025-11-24T22:25:16.908814-05:00,201131.00,B341BC2EAFBBF6467F32DA2ABA087C16F767F5A1FFD88CF06F7F61B797AAB680,8955139,0,false, -2025-11-24T22:21:56.276516-05:00,2025-11-24T22:25:48.939431-05:00,232661.00,1533E605B672B4BE63BE1215DDE21E724B2DB155F2311DB059BBA87886227BB3,8955143,0,false, -2025-11-24T22:21:56.776386-05:00,2025-11-24T22:25:49.267828-05:00,232490.00,C1E0662ADB661F18F5410F4D8E475A2CA84E889F2F95A6B4A5A46E43FC9F2719,8955143,0,false, -2025-11-24T22:21:57.279309-05:00,2025-11-24T22:25:49.592874-05:00,232312.00,9584E2DE835C0D2B8F1B01FF7366D161DE6E1B3D517E2D6BC139BAE2CAC26C00,8955143,0,false, -2025-11-24T22:21:57.774804-05:00,2025-11-24T22:25:56.477388-05:00,238701.00,B67D7DDC68F09E96EC21FF265CC3FD938D0EA5E4780758C86CF3665E07048989,8955144,0,false, -2025-11-24T22:21:58.283375-05:00,2025-11-24T22:26:00.093256-05:00,241808.00,19D5F572A3C02DD75FBB6D20604D1EF9653C9C99D540BE7E0A4AB8D92D9D7625,8955145,0,false, -2025-11-24T22:21:58.779236-05:00,2025-11-24T22:26:00.920665-05:00,242139.00,B8C7194367CA2543B1C526B1C26DE6295495F27D718A8297BF07F78A9FC612C1,8955145,0,false, -2025-11-24T22:21:59.28152-05:00,2025-11-24T22:26:01.254284-05:00,241971.00,BB712099F8E01207FE6B7F16C7B463017E2B14B5CB2B738C54035781AE5D4666,8955145,0,false, -2025-11-24T22:21:59.783232-05:00,2025-11-24T22:26:01.588189-05:00,241803.00,BC351B0AC766BF4CEA96D3ACA4EA8C1694FBB9A7F133A339624294F02120F7CA,8955145,0,false, -2025-11-24T22:22:00.297655-05:00,2025-11-24T22:26:05.68611-05:00,245386.00,B1E6038E4753E1415CD8CB18BB01A503069FE90DE5D1783636E67672E172CE21,8955146,0,false, -2025-11-24T22:22:00.766919-05:00,2025-11-24T22:26:06.000592-05:00,245232.00,16875D0B1B2831BC4B6B08867F618E71F81048A1B2EBB49B998A025D74CC8F4D,8955146,0,false, -2025-11-24T22:22:01.283126-05:00,2025-11-24T22:26:06.317163-05:00,245032.00,B724C34E239EA50CE05AF0CBE475B9F7E16C609D17B6ED83065742B65557A233,8955146,0,false, -2025-11-24T22:22:01.774468-05:00,2025-11-24T22:26:06.617086-05:00,244841.00,4788B93AE62464FDBDD5D858ADC15B5AF3386A94EB2B631F3A5C823F42650BC0,8955146,0,false, -2025-11-24T22:22:02.26709-05:00,2025-11-24T22:26:44.536842-05:00,282268.00,771DCB8AF3D54C748A4087E094324FF1D99885306740E6D725F2DDDFFC08DCFB,8955152,0,false, -2025-11-24T22:22:02.7702-05:00,2025-11-24T22:26:50.8384-05:00,288066.00,C9F1D2952402C1A6240A2964922D9651DED712F5D600E0E7DBCF8380D2AF0B73,8955153,0,false, -2025-11-24T22:22:03.269254-05:00,2025-11-24T22:26:56.538174-05:00,293267.00,86E16288EAA1794153FC89B6FFDF644FD97677919B6554F83DB6F1C0580D1F44,8955154,0,false, -2025-11-24T22:22:05.275931-05:00,2025-11-24T22:27:06.338925-05:00,301061.00,F5AD10B0D289FC344E06B19CB7CEEEE1630CFEF367954AEFD0C464A030578778,8955156,0,false, -2025-11-24T22:22:05.771444-05:00,2025-11-24T22:27:06.646675-05:00,300873.00,6F396D422EF2FF9072B45B24D0FD12C9C2F1470E5D928C67E0FCEDC32F84F59B,8955156,0,false, -2025-11-24T22:22:03.779062-05:00,2025-11-24T22:27:08.373405-05:00,304592.00,09F4A203FAD77434475F284F55E5E619C3196B4131F91840C292A9409EA9F115,8955156,0,false, -2025-11-24T22:22:04.271105-05:00,2025-11-24T22:27:08.474921-05:00,304201.00,5485F512F137FFCFD5312EEAB9A3A750458E9027776D6086316D1539DE085746,8955156,0,false, -2025-11-24T22:22:04.773575-05:00,2025-11-24T22:27:09.558038-05:00,304782.00,E010293B821167995455D7A4BEAA624C8AA1C3B525F9A41FF86ABF8DF0D47176,8955156,0,false, +2025-11-24T23:29:32.021369-05:00,2025-11-24T23:29:46.365573-05:00,14344.00,F7A5D76DAAC76779BC6A9C618EB2113795E1005C4416A0ABF0E1F9D9E7DE5F8A,8955716,0,false, +2025-11-24T23:29:33.013307-05:00,2025-11-24T23:29:52.386416-05:00,19372.00,8B0EB88ACCA13DAAAB9C7D6339C723D8CFA570DDC7E3EBDA9C54779D7E515EFB,8955717,0,false, +2025-11-24T23:29:34.01281-05:00,2025-11-24T23:29:52.714499-05:00,18701.00,558F84A38D134C414B2589D32B30082078C3285E1E05D83406BE648A923CCBFF,8955717,0,false, +2025-11-24T23:29:35.010764-05:00,2025-11-24T23:29:52.820569-05:00,17809.00,538DA9A5F156748E567C430E6B50E65217501AAA72E401D17B997D487F9B4073,8955717,0,false, +2025-11-24T23:29:36.024857-05:00,2025-11-24T23:29:58.251224-05:00,22226.00,0014B732ED282C5D69E9550522AF23E05B7201D2BD36608B57B45816C55FAA14,8955718,0,false, +2025-11-24T23:29:37.011508-05:00,2025-11-24T23:29:58.55681-05:00,21545.00,972F39B103F124D74CA1CA0652707D5FA90C027D98AFC9F47A6EDE18C9D138D8,8955718,0,false, +2025-11-24T23:29:38.008865-05:00,2025-11-24T23:29:58.760735-05:00,20751.00,B769B415D90BF59A0ED171858B4F3C74F325D535E048355D7C2EBAF7CAFA7B4F,8955718,0,false, +2025-11-24T23:29:39.01153-05:00,2025-11-24T23:29:58.867871-05:00,19856.00,19A7FB4B91FE279F3C799DA26B327FA4B1583303EC7E53AAE586C5BF4875A4DB,8955718,0,false, +2025-11-24T23:29:40.013085-05:00,2025-11-24T23:30:04.376415-05:00,24363.00,3F202ACD6B31183EAE6D894BAC39DD78DA241C72E949045998C28D10676D5986,8955719,0,false, +2025-11-24T23:29:41.01202-05:00,2025-11-24T23:30:04.870542-05:00,23858.00,578CFF44505CAF743F3133815D55ED3FC03C7779C85DDAA69CD8B08C14C311DE,8955719,0,false, +2025-11-24T23:29:42.014416-05:00,2025-11-24T23:30:29.102982-05:00,47087.00,EB8AE4277E6B4E40B838DAC1D7E940E3FB3DE9D6E17255FB8FA55B2FFF3902BC,8955721,0,false, +2025-11-24T23:29:43.009887-05:00,2025-11-24T23:30:29.414156-05:00,46403.00,5424A9DAB82931E91AC61C7DD399EC877FABB5560717F906E562F980AA141E90,8955721,0,false, diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index 22334e19dd..83126fb7a6 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -136,9 +136,12 @@ func (q *sequentialQueue) GetQueueSize() int { // processNextTx signs and broadcasts the next unbroadcast transaction in queue func (q *sequentialQueue) processNextTx() { + startTime := time.Now() // Find first unbroadcast transaction (txHash is empty) fmt.Println("Processing next tx") + + scanStart := time.Now() var qTx *queuedTx q.mu.RLock() for _, tx := range q.queue { @@ -147,18 +150,25 @@ func (q *sequentialQueue) processNextTx() { break } } + queueSize := len(q.queue) q.mu.RUnlock() + scanDuration := time.Since(scanStart) if qTx == nil { return } + fmt.Printf("[TIMING] Queue scan took %v (queue size: %d)\n", scanDuration, queueSize) + + broadcastStart := time.Now() resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( q.ctx, q.accountName, qTx.blobs, qTx.options..., ) + broadcastDuration := time.Since(broadcastStart) + fmt.Printf("[TIMING] Broadcast call took %v\n", broadcastDuration) if err != nil || resp.Code != 0 { // Check if this is a sequence mismatch AND we're blocked @@ -193,11 +203,8 @@ func (q *sequentialQueue) processNextTx() { qTx.sequence = sequence qTx.submittedAt = time.Now() - // Track submission metrics - q.newBroadcastCount++ - q.logSubmissionMetrics() - - fmt.Println("Broadcast successful - marking as broadcast in queue") + fmt.Printf("Broadcast successful for tx %s - marking as broadcast in queue\n", qTx.txHash[:16]) + fmt.Printf("[TIMING] Total processNextTx took %v\n", time.Since(startTime)) } // monitorLoop periodically checks the status of broadcast transactions @@ -237,6 +244,7 @@ func (q *sequentialQueue) coordinate() { // ResignRejected resigns a rejected transaction func (q *sequentialQueue) ResignRejected() { + startTime := time.Now() fmt.Println("Resigning rejected tx") q.mu.RLock() var txsToResign []*queuedTx @@ -250,6 +258,7 @@ func (q *sequentialQueue) ResignRejected() { for _, qTx := range txsToResign { if qTx.shouldResign { // resign the tx + resignStart := time.Now() resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( q.ctx, q.accountName, @@ -274,11 +283,12 @@ func (q *sequentialQueue) ResignRejected() { qTx.sequence = sequence qTx.shouldResign = false q.resignCount++ - q.logSubmissionMetrics() q.mu.Unlock() - fmt.Printf("Resigned and submitted tx successfully: %s\n", resp.TxHash) + resignDuration := time.Since(resignStart) + fmt.Printf("Resigned and submitted tx successfully: %s (took %v)\n", resp.TxHash, resignDuration) } } + fmt.Printf("[TIMING] Total ResignRejected took %v\n", time.Since(startTime)) } // TODO: come back to this and see if it makes sense @@ -293,13 +303,17 @@ func (q *sequentialQueue) ResignRejected() { // } func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { + startTime := time.Now() fmt.Printf("Resubmitting evicted tx with hash %s and sequence %d\n", qTx.txHash[:16], qTx.sequence) q.mu.RLock() txBytes := qTx.txBytes q.mu.RUnlock() // check if the tx needs to be resubmitted + resubmitStart := time.Now() resubmitResp, err := q.client.SendTxToConnection(q.ctx, q.client.GetGRPCConnection(), txBytes) + resubmitDuration := time.Since(resubmitStart) + fmt.Printf("[TIMING] Resubmit network call took %v\n", resubmitDuration) if err != nil || resubmitResp.Code != 0 { // Check if this is a sequence mismatch // if IsSequenceMismatchError(err) { @@ -359,14 +373,17 @@ func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { q.mu.Lock() qTx.isResubmitting = false q.resubmitCount++ - q.logSubmissionMetrics() q.mu.Unlock() fmt.Printf("Successfully resubmitted tx %s\n", qTx.txHash[:16]) + fmt.Printf("[TIMING] Total ResubmitEvicted took %v\n", time.Since(startTime)) } // checkBroadcastTransactions checks status of all broadcast transactions func (q *sequentialQueue) checkBroadcastTransactions() { + startTime := time.Now() fmt.Println("Checking broadcast transactions") + + scanStart := time.Now() q.mu.RLock() // Collect all broadcast transactions (those with non-empty txHash) var broadcastTxs []*queuedTx // TODO: cap the size @@ -375,10 +392,13 @@ func (q *sequentialQueue) checkBroadcastTransactions() { broadcastTxs = append(broadcastTxs, tx) } } + fmt.Printf("Broadcast txs: %d\n", len(broadcastTxs)) totalQueueSize := len(q.queue) q.mu.RUnlock() + scanDuration := time.Since(scanStart) fmt.Printf("Total queue size: %d, Broadcast txs: %d\n", totalQueueSize, len(broadcastTxs)) + fmt.Printf("[TIMING] Collecting broadcast txs scan took %v\n", scanDuration) if len(broadcastTxs) == 0 { return @@ -387,7 +407,10 @@ func (q *sequentialQueue) checkBroadcastTransactions() { // Create tx client for status queries txClient := tx.NewTxClient(q.client.GetGRPCConnection()) + statusCheckStart := time.Now() + statusCheckCount := 0 for _, qTx := range broadcastTxs { + statusCheckCount++ statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) if err != nil { qTx.resultsC <- SequentialSubmissionResult{ @@ -452,6 +475,11 @@ func (q *sequentialQueue) checkBroadcastTransactions() { q.handleRejected(qTx, statusResp, txClient) } } + + statusCheckDuration := time.Since(statusCheckStart) + fmt.Printf("[TIMING] Status checks took %v for %d txs (avg: %v per tx)\n", + statusCheckDuration, statusCheckCount, statusCheckDuration/time.Duration(statusCheckCount)) + fmt.Printf("[TIMING] Total checkBroadcastTransactions took %v\n", time.Since(startTime)) } func (q *sequentialQueue) handleEvicted(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { @@ -602,26 +630,3 @@ func parseExpectedSequence(errMsg string) uint64 { } return 0 } - -// logSubmissionMetrics logs submission statistics every 30 seconds -// Note: Caller must hold q.mu lock -func (q *sequentialQueue) logSubmissionMetrics() { - now := time.Now() - if now.Sub(q.lastMetricsLog) < 30*time.Second { - return - } - - elapsed := now.Sub(q.metricsStartTime).Seconds() - if elapsed < 1 { - return - } - - totalSubmissions := q.newBroadcastCount + q.resubmitCount + q.resignCount - submissionsPerSec := float64(totalSubmissions) / elapsed - - fmt.Printf("[METRICS] Total submissions: %d (new: %d, resubmit: %d, resign: %d) | Rate: %.2f tx/sec | Queue size: %d\n", - totalSubmissions, q.newBroadcastCount, q.resubmitCount, q.resignCount, - submissionsPerSec, len(q.queue)) - - q.lastMetricsLog = now -} From fa006a37a03232afe14966043195b87ab0471fb3 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Wed, 26 Nov 2025 13:15:10 -0500 Subject: [PATCH 06/12] chore: prototype is accounting for rejections but its not necessary --- go.mod | 63 +++--- go.sum | 132 ++++++------ latency_results.csv | 366 ++++++++++++++++++++++++++++++-- pkg/user/v2/sequential_queue.go | 249 +++++++++++++++++----- 4 files changed, 641 insertions(+), 169 deletions(-) diff --git a/go.mod b/go.mod index 3984991917..b59aa05b44 100644 --- a/go.mod +++ b/go.mod @@ -17,11 +17,11 @@ require ( cosmossdk.io/x/feegrant v0.1.1 cosmossdk.io/x/tx v0.13.8 cosmossdk.io/x/upgrade v0.1.4 - github.com/aws/aws-sdk-go-v2 v1.39.6 - github.com/aws/aws-sdk-go-v2/config v1.31.17 - github.com/aws/aws-sdk-go-v2/credentials v1.18.21 + github.com/aws/aws-sdk-go-v2 v1.40.0 + github.com/aws/aws-sdk-go-v2/config v1.31.20 + github.com/aws/aws-sdk-go-v2/credentials v1.18.24 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11 - github.com/aws/aws-sdk-go-v2/service/s3 v1.90.0 + github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0 github.com/bcp-innovations/hyperlane-cosmos v1.0.1 github.com/celestiaorg/go-square/v2 v2.3.3 github.com/celestiaorg/go-square/v3 v3.0.2 @@ -54,8 +54,8 @@ require ( golang.org/x/oauth2 v0.33.0 golang.org/x/sync v0.18.0 google.golang.org/api v0.247.0 - google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c - google.golang.org/grpc v1.76.0 + google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8 + google.golang.org/grpc v1.77.0 google.golang.org/protobuf v1.36.10 gopkg.in/yaml.v2 v2.4.0 ) @@ -65,7 +65,7 @@ require ( cloud.google.com/go v0.121.6 // indirect cloud.google.com/go/auth v0.16.4 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.8 // indirect - cloud.google.com/go/compute/metadata v0.8.0 // indirect + cloud.google.com/go/compute/metadata v0.9.0 // indirect cloud.google.com/go/iam v1.5.2 // indirect cloud.google.com/go/monitoring v1.24.2 // indirect cloud.google.com/go/storage v1.56.0 // indirect @@ -76,7 +76,7 @@ require ( github.com/99designs/keyring v1.2.2 // indirect github.com/DataDog/datadog-go v4.8.3+incompatible // indirect github.com/DataDog/zstd v1.5.6 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.53.0 // indirect github.com/Microsoft/go-winio v0.6.2 // indirect @@ -84,17 +84,17 @@ require ( github.com/aws/aws-sdk-go v1.55.5 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.30.1 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.39.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.30.3 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.40.2 // indirect github.com/aws/smithy-go v1.23.2 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bgentry/go-netrc v0.0.0-20140422174119-9fd32a8b3d3d // indirect @@ -110,7 +110,7 @@ require ( github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/chzyer/readline v1.5.1 // indirect github.com/cloudwego/base64x v0.1.6 // indirect - github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 // indirect + github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f // indirect github.com/cockroachdb/apd/v2 v2.0.2 // indirect github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect @@ -136,14 +136,14 @@ require ( github.com/dustin/go-humanize v1.0.1 // indirect github.com/dvsekhvalnov/jose2go v1.7.0 // indirect github.com/emicklei/dot v1.6.2 // indirect - github.com/envoyproxy/go-control-plane/envoy v1.32.4 // indirect + github.com/envoyproxy/go-control-plane/envoy v1.35.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.2.1 // indirect github.com/ethereum/go-ethereum v1.15.8 // indirect github.com/fatih/color v1.18.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/fsnotify/fsnotify v1.9.0 // indirect github.com/getsentry/sentry-go v0.35.0 // indirect - github.com/go-jose/go-jose/v4 v4.1.2 // indirect + github.com/go-jose/go-jose/v4 v4.1.3 // indirect github.com/go-kit/kit v0.13.0 // indirect github.com/go-logfmt/logfmt v0.6.1 // indirect github.com/go-logr/logr v1.4.3 // indirect @@ -223,7 +223,7 @@ require ( github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_golang v1.23.2 // indirect github.com/prometheus/client_model v0.6.2 // indirect - github.com/prometheus/common v0.67.1 // indirect + github.com/prometheus/common v0.67.4 // indirect github.com/prometheus/procfs v0.16.1 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/rogpeppe/go-internal v1.14.1 // indirect @@ -232,7 +232,7 @@ require ( github.com/sasha-s/go-deadlock v0.3.6 // indirect github.com/sourcegraph/conc v0.3.1-0.20240121214520-5f936abd7ae8 // indirect github.com/spf13/afero v1.15.0 // indirect - github.com/spiffe/go-spiffe/v2 v2.5.0 // indirect + github.com/spiffe/go-spiffe/v2 v2.6.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/subosito/gotenv v1.6.0 // indirect github.com/syndtr/goleveldb v1.0.1-0.20220721030215-126854af5e6d // indirect @@ -240,14 +240,13 @@ require ( github.com/tidwall/btree v1.7.0 // indirect github.com/twitchyliquid64/golang-asm v0.15.1 // indirect github.com/ulikunitz/xz v0.5.14 // indirect - github.com/zeebo/errs v1.4.0 // indirect github.com/zondax/golem v0.27.0 // indirect github.com/zondax/hid v0.9.2 // indirect github.com/zondax/ledger-go v1.0.1 // indirect go.etcd.io/bbolt v1.4.0 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/auto/sdk v1.1.0 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.36.0 // indirect + go.opentelemetry.io/auto/sdk v1.2.1 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.38.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.62.0 // indirect go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.38.0 // indirect @@ -259,15 +258,15 @@ require ( go.yaml.in/yaml/v2 v2.4.3 // indirect go.yaml.in/yaml/v3 v3.0.4 // indirect golang.org/x/arch v0.15.0 // indirect - golang.org/x/crypto v0.43.0 // indirect + golang.org/x/crypto v0.45.0 // indirect golang.org/x/exp v0.0.0-20250911091902-df9299821621 // indirect - golang.org/x/net v0.46.0 // indirect - golang.org/x/sys v0.37.0 // indirect - golang.org/x/term v0.36.0 // indirect - golang.org/x/text v0.30.0 // indirect + golang.org/x/net v0.47.0 // indirect + golang.org/x/sys v0.38.0 // indirect + golang.org/x/term v0.37.0 // indirect + golang.org/x/text v0.31.0 // indirect golang.org/x/time v0.12.0 // indirect google.golang.org/genproto v0.0.0-20250603155806-513f23925822 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect gotest.tools/v3 v3.5.2 // indirect nhooyr.io/websocket v1.8.17 // indirect @@ -278,7 +277,7 @@ require ( replace ( cosmossdk.io/api => github.com/celestiaorg/cosmos-sdk/api v0.7.6 cosmossdk.io/x/upgrade => github.com/celestiaorg/cosmos-sdk/x/upgrade v0.2.0 - github.com/cometbft/cometbft => github.com/celestiaorg/celestia-core v0.39.15 + github.com/cometbft/cometbft => /Users/ninabarbakadze/Desktop/celestia/celestia-core github.com/cosmos/cosmos-sdk => github.com/celestiaorg/cosmos-sdk v0.51.7 github.com/cosmos/ibc-go/v8 => github.com/celestiaorg/ibc-go/v8 v8.7.2 // Use ledger-cosmos-go v0.16.0 because v0.15.0 causes "hidapi: unknown failure" diff --git a/go.sum b/go.sum index 8e4330463c..9188db4822 100644 --- a/go.sum +++ b/go.sum @@ -186,8 +186,8 @@ cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/compute/metadata v0.8.0 h1:HxMRIbao8w17ZX6wBnjhcDkW6lTFpgcaobyVfZWqRLA= -cloud.google.com/go/compute/metadata v0.8.0/go.mod h1:sYOGTp851OV9bOFJ9CH7elVvyzopvWQFNNghtDQ/Biw= +cloud.google.com/go/compute/metadata v0.9.0 h1:pDUj4QMoPejqq20dK0Pg2N4yG9zIkYGdBtwLoEkH9Zs= +cloud.google.com/go/compute/metadata v0.9.0/go.mod h1:E0bWwX5wTnLPedCKqk3pJmVgCBSM6qQI1yTBdEb3C10= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= @@ -662,8 +662,8 @@ github.com/DataDog/datadog-go v4.8.3+incompatible h1:fNGaYSuObuQb5nzeTQqowRAd9bp github.com/DataDog/datadog-go v4.8.3+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/DataDog/zstd v1.5.6 h1:LbEglqepa/ipmmQJUDnSsfvA8e8IStVcGaFWDuxvGOY= github.com/DataDog/zstd v1.5.6/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 h1:UQUsRi8WTzhZntp5313l+CHIAT95ojUI2lpP/ExlZa4= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0/go.mod h1:Cz6ft6Dkn3Et6l2v2a9/RpN7epQ1GtDlO6lj8bEcOvw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0 h1:sBEjpZlNHzK1voKq9695PJSX2o5NEXl7/OL3coiIY0c= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0/go.mod h1:P4WPRUkOhJC13W//jWpyfJNDAIpvRbAUIYLX/4jtlE0= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 h1:owcC2UnmsZycprQ5RfRgjydWhuoxg71LUfyiQdijZuM= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0/go.mod h1:ZPpqegjbE99EPKsu3iUWV22A04wzGPcAY/ziSIQEEgs= github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.53.0 h1:4LP6hvB4I5ouTbGgWtixJhgED6xdf67twf9PoY96Tbg= @@ -714,42 +714,42 @@ github.com/aws/aws-sdk-go v1.44.122/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU= github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= -github.com/aws/aws-sdk-go-v2 v1.39.6 h1:2JrPCVgWJm7bm83BDwY5z8ietmeJUbh3O2ACnn+Xsqk= -github.com/aws/aws-sdk-go-v2 v1.39.6/go.mod h1:c9pm7VwuW0UPxAEYGyTmyurVcNrbF6Rt/wixFqDhcjE= +github.com/aws/aws-sdk-go-v2 v1.40.0 h1:/WMUA0kjhZExjOQN2z3oLALDREea1A7TobfuiBrKlwc= +github.com/aws/aws-sdk-go-v2 v1.40.0/go.mod h1:c9pm7VwuW0UPxAEYGyTmyurVcNrbF6Rt/wixFqDhcjE= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3 h1:DHctwEM8P8iTXFxC/QK0MRjwEpWQeM9yzidCRjldUz0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3/go.mod h1:xdCzcZEtnSTKVDOmUZs4l/j3pSV6rpo1WXl5ugNsL8Y= -github.com/aws/aws-sdk-go-v2/config v1.31.17 h1:QFl8lL6RgakNK86vusim14P2k8BFSxjvUkcWLDjgz9Y= -github.com/aws/aws-sdk-go-v2/config v1.31.17/go.mod h1:V8P7ILjp/Uef/aX8TjGk6OHZN6IKPM5YW6S78QnRD5c= -github.com/aws/aws-sdk-go-v2/credentials v1.18.21 h1:56HGpsgnmD+2/KpG0ikvvR8+3v3COCwaF4r+oWwOeNA= -github.com/aws/aws-sdk-go-v2/credentials v1.18.21/go.mod h1:3YELwedmQbw7cXNaII2Wywd+YY58AmLPwX4LzARgmmA= +github.com/aws/aws-sdk-go-v2/config v1.31.20 h1:/jWF4Wu90EhKCgjTdy1DGxcbcbNrjfBHvksEL79tfQc= +github.com/aws/aws-sdk-go-v2/config v1.31.20/go.mod h1:95Hh1Tc5VYKL9NJ7tAkDcqeKt+MCXQB1hQZaRdJIZE0= +github.com/aws/aws-sdk-go-v2/credentials v1.18.24 h1:iJ2FmPT35EaIB0+kMa6TnQ+PwG5A1prEdAw+PsMzfHg= +github.com/aws/aws-sdk-go-v2/credentials v1.18.24/go.mod h1:U91+DrfjAiXPDEGYhh/x29o4p0qHX5HDqG7y5VViv64= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13 h1:T1brd5dR3/fzNFAQch/iBKeX07/ffu/cLu+q+RuzEWk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13/go.mod h1:Peg/GBAQ6JDt+RoBf4meB1wylmAipb7Kg2ZFakZTlwk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11 h1:w4GjasReY0m9vZA/3YhoBUBi1ZIWUHYQRm61v0BKcZg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11/go.mod h1:IPS1CSYQ8lfLYGytpMEPW4erZmVFUdxLpC0RCI/RCn8= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13 h1:a+8/MLcWlIxo1lF9xaGt3J/u3yOZx+CdSveSNwjhD40= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13/go.mod h1:oGnKwIYZ4XttyU2JWxFrwvhF6YKiK/9/wmE3v3Iu9K8= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13 h1:HBSI2kDkMdWz4ZM7FjwE7e/pWDEZ+nR95x8Ztet1ooY= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13/go.mod h1:YE94ZoDArI7awZqJzBAZ3PDD2zSfuP7w6P2knOzIn8M= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14 h1:PZHqQACxYb8mYgms4RZbhZG0a7dPW06xOjmaH0EJC/I= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14/go.mod h1:VymhrMJUWs69D8u0/lZ7jSB6WgaG/NqHi3gX0aYf6U0= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14 h1:bOS19y6zlJwagBfHxs0ESzr1XCOU2KXJCWcq3E2vfjY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14/go.mod h1:1ipeGBMAxZ0xcTm6y6paC2C/J6f6OO7LBODV9afuAyM= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4 h1:WKuaxf++XKWlHWu9ECbMlha8WOEGm0OUEZqm4K/Gcfk= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4/go.mod h1:ZWy7j6v1vWGmPReu0iSGvRiise4YI5SkR3OHKTZ6Wuc= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13 h1:eg/WYAa12vqTphzIdWMzqYRVKKnCboVPRlvaybNCqPA= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13/go.mod h1:/FDdxWhz1486obGrKKC1HONd7krpk38LBt+dutLcN9k= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14 h1:ITi7qiDSv/mSGDSWNpZ4k4Ve0DQR6Ug2SJQ8zEHoDXg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14/go.mod h1:k1xtME53H1b6YpZt74YmwlONMWf4ecM+lut1WQLAF/U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3 h1:x2Ibm/Af8Fi+BH+Hsn9TXGdT+hKbDd5XOTZxTMxDk7o= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3/go.mod h1:IW1jwyrQgMdhisceG8fQLmQIydcT/jWY21rFhzgaKwo= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4 h1:NvMjwvv8hpGUILarKw7Z4Q0w1H9anXKsesMxtw++MA4= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4/go.mod h1:455WPHSwaGj2waRSpQp7TsnpOnBfw8iDfPfbwl7KPJE= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13 h1:kDqdFvMY4AtKoACfzIGD8A0+hbT41KTKF//gq7jITfM= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13/go.mod h1:lmKuogqSU3HzQCwZ9ZtcqOc5XGMqtDK7OIc2+DxiUEg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13 h1:zhBJXdhWIFZ1acfDYIhu4+LCzdUS2Vbcum7D01dXlHQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13/go.mod h1:JaaOeCE368qn2Hzi3sEzY6FgAZVCIYcC2nwbro2QCh8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.90.0 h1:ef6gIJR+xv/JQWwpa5FYirzoQctfSJm7tuDe3SZsUf8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.90.0/go.mod h1:+wArOOrcHUevqdto9k1tKOF5++YTe9JEcPSc9Tx2ZSw= -github.com/aws/aws-sdk-go-v2/service/sso v1.30.1 h1:0JPwLz1J+5lEOfy/g0SURC9cxhbQ1lIMHMa+AHZSzz0= -github.com/aws/aws-sdk-go-v2/service/sso v1.30.1/go.mod h1:fKvyjJcz63iL/ftA6RaM8sRCtN4r4zl4tjL3qw5ec7k= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5 h1:OWs0/j2UYR5LOGi88sD5/lhN6TDLG6SfA7CqsQO9zF0= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5/go.mod h1:klO+ejMvYsB4QATfEOIXk8WAEwN4N0aBfJpvC+5SZBo= -github.com/aws/aws-sdk-go-v2/service/sts v1.39.1 h1:mLlUgHn02ue8whiR4BmxxGJLR2gwU6s6ZzJ5wDamBUs= -github.com/aws/aws-sdk-go-v2/service/sts v1.39.1/go.mod h1:E19xDjpzPZC7LS2knI9E6BaRFDK43Eul7vd6rSq2HWk= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5 h1:Hjkh7kE6D81PgrHlE/m9gx+4TyyeLHuY8xJs7yXN5C4= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5/go.mod h1:nPRXgyCfAurhyaTMoBMwRBYBhaHI4lNPAnJmjM0Tslc= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14 h1:FIouAnCE46kyYqyhs0XEBDFFSREtdnr8HQuLPQPLCrY= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14/go.mod h1:UTwDc5COa5+guonQU8qBikJo1ZJ4ln2r1MkF7Dqag1E= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14 h1:FzQE21lNtUor0Fb7QNgnEyiRCBlolLTX/Z1j65S7teM= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14/go.mod h1:s1ydyWG9pm3ZwmmYN21HKyG9WzAZhYVW85wMHs5FV6w= +github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0 h1:8FshVvnV2sr9kOSAbOnc/vwVmmAwMjOedKH6JW2ddPM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0/go.mod h1:wYNqY3L02Z3IgRYxOBPH9I1zD9Cjh9hI5QOy/eOjQvw= +github.com/aws/aws-sdk-go-v2/service/sso v1.30.3 h1:NjShtS1t8r5LUfFVtFeI8xLAHQNTa7UI0VawXlrBMFQ= +github.com/aws/aws-sdk-go-v2/service/sso v1.30.3/go.mod h1:fKvyjJcz63iL/ftA6RaM8sRCtN4r4zl4tjL3qw5ec7k= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7 h1:gTsnx0xXNQ6SBbymoDvcoRHL+q4l/dAFsQuKfDWSaGc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7/go.mod h1:klO+ejMvYsB4QATfEOIXk8WAEwN4N0aBfJpvC+5SZBo= +github.com/aws/aws-sdk-go-v2/service/sts v1.40.2 h1:HK5ON3KmQV2HcAunnx4sKLB9aPf3gKGwVAf7xnx0QT0= +github.com/aws/aws-sdk-go-v2/service/sts v1.40.2/go.mod h1:E19xDjpzPZC7LS2knI9E6BaRFDK43Eul7vd6rSq2HWk= github.com/aws/smithy-go v1.23.2 h1:Crv0eatJUQhaManss33hS5r40CG3ZFH+21XSkqMrIUM= github.com/aws/smithy-go v1.23.2/go.mod h1:LEj2LM3rBRQJxPZTB4KuzZkaZYnZPnvgIhb4pu07mx0= github.com/bcp-innovations/hyperlane-cosmos v1.0.1 h1:gT8OqyJ866Q6AHOlIXKxSdLjd0p8crKG9XXERIWoh4c= @@ -783,8 +783,6 @@ github.com/bytedance/sonic v1.14.1/go.mod h1:gi6uhQLMbTdeP0muCnrjHLeCUPyb70ujhnN github.com/bytedance/sonic/loader v0.3.0 h1:dskwH8edlzNMctoruo8FPTJDF3vLtDT0sXZwvZJyqeA= github.com/bytedance/sonic/loader v0.3.0/go.mod h1:N8A3vUdtUebEY2/VQC0MyhYeKUFosQU6FxH2JmUe6VI= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= -github.com/celestiaorg/celestia-core v0.39.15 h1:Wpmu0nAPzhEiK2TbSGeTus4c0kikHQGs0Y3Q1q9eo1o= -github.com/celestiaorg/celestia-core v0.39.15/go.mod h1:98r/9+ol56uhrtLQPmTpgGto2DJ4n8ga45T0xdSg4C0= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35 h1:FREwqZwPvYsodr1AqqEIyW+VsBnwTzJNtC6NFdZX8rs= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35/go.mod h1:SI38xqZZ4ccoAxszUJqsJ/a5rOkzQRijzHQQlLKkyUc= github.com/celestiaorg/cosmos-sdk v0.51.7 h1:OSHrWinwqPXU78rpP+hJ8QJP4nx/8pa1JZgTt4bw+ZU= @@ -847,8 +845,8 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 h1:aQ3y1lwWyqYPiWZThqv1aFbZMiM9vblcSArJRf2Irls= -github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= +github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f h1:Y8xYupdHxryycyPlc9Y+bSQAYZnetRJ70VMVKm5CKI0= +github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f/go.mod h1:HlzOvOjVBOfTGSRXRyY0OiCS/3J1akRGQQpRO/7zyF4= github.com/cockroachdb/apd/v2 v2.0.2 h1:weh8u7Cneje73dDh+2tEVLUvyBc89iwepWCD8b8034E= github.com/cockroachdb/apd/v2 v2.0.2/go.mod h1:DDxRlzC2lo3/vSlmSoS7JkqbbrARPuFOGr0B9pvN3Gw= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= @@ -957,10 +955,10 @@ github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go. github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJJM//w9BV6Fxbg2LuVd34= github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q= -github.com/envoyproxy/go-control-plane v0.13.4 h1:zEqyPVyku6IvWCFwux4x9RxkLOMUL+1vC9xUFv5l2/M= -github.com/envoyproxy/go-control-plane v0.13.4/go.mod h1:kDfuBlDVsSj2MjrLEtRWtHlsWIFcGyB2RMO44Dc5GZA= -github.com/envoyproxy/go-control-plane/envoy v1.32.4 h1:jb83lalDRZSpPWW2Z7Mck/8kXZ5CQAFYVjQcdVIr83A= -github.com/envoyproxy/go-control-plane/envoy v1.32.4/go.mod h1:Gzjc5k8JcJswLjAx1Zm+wSYE20UrLtt7JZMWiWQXQEw= +github.com/envoyproxy/go-control-plane v0.13.5-0.20251024222203-75eaa193e329 h1:K+fnvUM0VZ7ZFJf0n4L/BRlnsb9pL/GuDG6FqaH+PwM= +github.com/envoyproxy/go-control-plane v0.13.5-0.20251024222203-75eaa193e329/go.mod h1:Alz8LEClvR7xKsrq3qzoc4N0guvVNSS8KmSChGYr9hs= +github.com/envoyproxy/go-control-plane/envoy v1.35.0 h1:ixjkELDE+ru6idPxcHLj8LBVc2bFP7iBytj353BoHUo= +github.com/envoyproxy/go-control-plane/envoy v1.35.0/go.mod h1:09qwbGVuSWWAyN5t/b3iyVfz5+z8QWGrzkoqm/8SbEs= github.com/envoyproxy/go-control-plane/ratelimit v0.1.0 h1:/G9QYbddjL25KvtKTv3an9lx6VBE2cnb8wp1vEGNYGI= github.com/envoyproxy/go-control-plane/ratelimit v0.1.0/go.mod h1:Wk+tMFAFbCXaJPzVVHnPgRKdUdwW/KdbRt94AzgRee4= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= @@ -1004,8 +1002,8 @@ github.com/go-fonts/stix v0.1.0/go.mod h1:w/c1f0ldAUlJmLBvlbkvVXLAD+tAMqobIIQpmn github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-jose/go-jose/v4 v4.1.2 h1:TK/7NqRQZfgAh+Td8AlsrvtPoUyiHh0LqVvokh+1vHI= -github.com/go-jose/go-jose/v4 v4.1.2/go.mod h1:22cg9HWM1pOlnRiY+9cQYJ9XHmya1bYW8OeDM6Ku6Oo= +github.com/go-jose/go-jose/v4 v4.1.3 h1:CVLmWDhDVRa6Mi/IgCgaopNosCaHz7zrMeF9MlZRkrs= +github.com/go-jose/go-jose/v4 v4.1.3/go.mod h1:x4oUasVrzR7071A4TnHLGSPpNOm2a21K9Kf04k1rs08= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= @@ -1527,8 +1525,8 @@ github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8b github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.67.1 h1:OTSON1P4DNxzTg4hmKCc37o4ZAZDv0cfXLkOt0oEowI= -github.com/prometheus/common v0.67.1/go.mod h1:RpmT9v35q2Y+lsieQsdOh5sXZ6ajUGC8NjZAmr8vb0Q= +github.com/prometheus/common v0.67.4 h1:yR3NqWO1/UyO1w2PhUvXlGQs/PtFmoveVO0KZ4+Lvsc= +github.com/prometheus/common v0.67.4/go.mod h1:gP0fq6YjjNCLssJCQp0yk4M8W6ikLURwkdd/YKtTbyI= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= @@ -1598,8 +1596,8 @@ github.com/spf13/pflag v1.0.10 h1:4EBh2KAYBwaONj6b2Ye1GiHfwjqyROoF4RwYO+vPwFk= github.com/spf13/pflag v1.0.10/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.21.0 h1:x5S+0EU27Lbphp4UKm1C+1oQO+rKx36vfCoaVebLFSU= github.com/spf13/viper v1.21.0/go.mod h1:P0lhsswPGWD/1lZJ9ny3fYnVqxiegrlNrEmgLjbTCAY= -github.com/spiffe/go-spiffe/v2 v2.5.0 h1:N2I01KCUkv1FAjZXJMwh95KK1ZIQLYbPfhaxw8WS0hE= -github.com/spiffe/go-spiffe/v2 v2.5.0/go.mod h1:P+NxobPc6wXhVtINNtFjNWGBTreew1GBUCwT2wPmb7g= +github.com/spiffe/go-spiffe/v2 v2.6.0 h1:l+DolpxNWYgruGQVV0xsfeya3CsC7m8iBzDnMpsbLuo= +github.com/spiffe/go-spiffe/v2 v2.6.0/go.mod h1:gm2SeUoMZEtpnzPNs2Csc0D/gX33k1xIx7lEzqblHEs= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= @@ -1659,8 +1657,6 @@ github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1 github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= -github.com/zeebo/errs v1.4.0 h1:XNdoD/RRMKP7HD0UhJnIzUy74ISdGGxURlYG8HSWSfM= -github.com/zeebo/errs v1.4.0/go.mod h1:sgbWHsvVuTPHcqJJGQ1WhI5KbWlHYz+2+2C/LSEtCw4= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= github.com/zondax/golem v0.27.0 h1:IbBjGIXF3SoGOZHsILJvIM/F/ylwJzMcHAcggiqniPw= github.com/zondax/golem v0.27.0/go.mod h1:AmorCgJPt00L8xN1VrMBe13PSifoZksnQ1Ge906bu4A= @@ -1688,10 +1684,10 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= -go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= -go.opentelemetry.io/contrib/detectors/gcp v1.36.0 h1:F7q2tNlCaHY9nMKHR6XH9/qkp8FktLnIcy6jJNyOCQw= -go.opentelemetry.io/contrib/detectors/gcp v1.36.0/go.mod h1:IbBN8uAIIx734PTonTPxAxnjc2pQTxWNkwfstZ+6H2k= +go.opentelemetry.io/auto/sdk v1.2.1 h1:jXsnJ4Lmnqd11kwkBV2LgLoFMZKizbCi5fNZ/ipaZ64= +go.opentelemetry.io/auto/sdk v1.2.1/go.mod h1:KRTj+aOaElaLi+wW1kO/DZRXwkF4C5xPbEe3ZiIhN7Y= +go.opentelemetry.io/contrib/detectors/gcp v1.38.0 h1:ZoYbqX7OaA/TAikspPl3ozPI6iY6LiIY9I8cUfm+pJs= +go.opentelemetry.io/contrib/detectors/gcp v1.38.0/go.mod h1:SU+iU7nu5ud4oCb3LQOhIZ3nRLj6FNVrKgtflbaf2ts= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0 h1:q4XOmH/0opmeuJtPsbFNivyl7bCt7yRBbeEm2sC/XtQ= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0/go.mod h1:snMWehoOh2wsEwnvvwtDyFCxVeDAODenXHtn5vzrKjo= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.62.0 h1:Hf9xI/XLML9ElpiHVDNwvqI0hIFlzV8dgIr35kV1kRU= @@ -1761,8 +1757,8 @@ golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliY golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= -golang.org/x/crypto v0.43.0 h1:dduJYIi3A3KOfdGOHX8AVZ/jGiyPa3IbBozJ5kNuE04= -golang.org/x/crypto v0.43.0/go.mod h1:BFbav4mRNlXJL4wNeejLpWxB7wMbc79PdRGhWKncxR0= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1900,8 +1896,8 @@ golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.34.0/go.mod h1:di0qlW3YNM5oh6GqDGQr92MyTozJPmybPK4Ev/Gm31k= -golang.org/x/net v0.46.0 h1:giFlY12I07fugqwPuWJi68oOnpfqFnJIJzaIIm2JVV4= -golang.org/x/net v0.46.0/go.mod h1:Q9BGdFy1y4nkUwiLvT5qtyhAnEHgnQ/zd8PfU6nc210= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -2067,8 +2063,8 @@ golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.37.0 h1:fdNQudmxPjkdUTPnLn5mdQv7Zwvbvpaxqs831goi9kQ= -golang.org/x/sys v0.37.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/sys v0.38.0 h1:3yZWxaJjBmCWXqhN1qh02AkOnCQ1poK6oF+a7xWL6Gc= +golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -2083,8 +2079,8 @@ golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.28.0/go.mod h1:Sw/lC2IAUZ92udQNf3WodGtn4k/XoLyZoh8v/8uiwek= -golang.org/x/term v0.36.0 h1:zMPR+aF8gfksFprF/Nc/rd1wRS1EI6nDBGyWAvDzx2Q= -golang.org/x/term v0.36.0/go.mod h1:Qu394IJq6V6dCBRgwqshf3mPF85AqzYEzofzRdZkWss= +golang.org/x/term v0.37.0 h1:8EGAD0qCmHYZg6J17DvsMy9/wJ7/D/4pV/wfnld5lTU= +golang.org/x/term v0.37.0/go.mod h1:5pB4lxRNYYVZuTLmy8oR2BH8dflOR+IbTYFD8fi3254= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2105,8 +2101,8 @@ golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= -golang.org/x/text v0.30.0 h1:yznKA/E9zq54KzlzBEAWn1NXSQ8DIp/NYMy88xJjl4k= -golang.org/x/text v0.30.0/go.mod h1:yDdHFIX9t+tORqspjENWgzaCVXgk0yYnYuSZ8UzzBVM= +golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= +golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2191,8 +2187,8 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= -golang.org/x/tools v0.37.0 h1:DVSRzp7FwePZW356yEAChSdNcQo6Nsp+fex1SUW09lE= -golang.org/x/tools v0.37.0/go.mod h1:MBN5QPQtLMHVdvsbtarmTNukZDdgwdwlO5qGacAzF0w= +golang.org/x/tools v0.38.0 h1:Hx2Xv8hISq8Lm16jvBZ2VQf+RLmbd7wVUsALibYI/IQ= +golang.org/x/tools v0.38.0/go.mod h1:yEsQ/d/YK8cjh0L6rZlY8tgtlKiBNTL14pGDJPJpYQs= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -2418,10 +2414,10 @@ google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20250603155806-513f23925822 h1:rHWScKit0gvAPuOnu87KpaYtjK5zBMLcULh7gxkCXu4= google.golang.org/genproto v0.0.0-20250603155806-513f23925822/go.mod h1:HubltRL7rMh0LfnQPkMH4NPDFEWp0jw3vixw7jEM53s= -google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c h1:AtEkQdl5b6zsybXcbz00j1LwNodDuH6hVifIaNqk7NQ= -google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c/go.mod h1:ea2MjsO70ssTfCjiwHgI0ZFqcw45Ksuk2ckf9G468GA= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c h1:qXWI/sQtv5UKboZ/zUk7h+mrf/lXORyI+n9DKDAusdg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c/go.mod h1:gw1tLEfykwDz2ET4a12jcXt4couGAm7IwsVaTy0Sflo= +google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8 h1:mepRgnBZa07I4TRuomDE4sTIYieg/osKmzIf4USdWS4= +google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8/go.mod h1:fDMmzKV90WSg1NbozdqrE64fkuTv6mlq2zxo9ad+3yo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8 h1:M1rk8KBnUsBDg1oPGHNCxG4vc1f49epmTO7xscSajMk= +google.golang.org/genproto/googleapis/rpc v0.0.0-20251022142026-3a174f9686a8/go.mod h1:7i2o+ce6H/6BluujYR+kqX3GKH+dChPTQU19wjRPiGk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.0/go.mod h1:chYK+tFQF0nDUGJgXMSgLCQk3phJEuONr2DCgLDdAQM= @@ -2469,8 +2465,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.76.0 h1:UnVkv1+uMLYXoIz6o7chp59WfQUYA2ex/BXQ9rHZu7A= -google.golang.org/grpc v1.76.0/go.mod h1:Ju12QI8M6iQJtbcsV+awF5a4hfJMLi4X0JLo94ULZ6c= +google.golang.org/grpc v1.77.0 h1:wVVY6/8cGA6vvffn+wWK5ToddbgdU3d8MNENr4evgXM= +google.golang.org/grpc v1.77.0/go.mod h1:z0BY1iVj0q8E1uSQCjL9cppRj+gnZjzDnzV0dHhrNig= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= diff --git a/latency_results.csv b/latency_results.csv index 370da70a6c..8a8a2588d8 100644 --- a/latency_results.csv +++ b/latency_results.csv @@ -1,13 +1,355 @@ Submit Time,Commit Time,Latency (ms),Tx Hash,Height,Code,Failed,Error -2025-11-24T23:29:32.021369-05:00,2025-11-24T23:29:46.365573-05:00,14344.00,F7A5D76DAAC76779BC6A9C618EB2113795E1005C4416A0ABF0E1F9D9E7DE5F8A,8955716,0,false, -2025-11-24T23:29:33.013307-05:00,2025-11-24T23:29:52.386416-05:00,19372.00,8B0EB88ACCA13DAAAB9C7D6339C723D8CFA570DDC7E3EBDA9C54779D7E515EFB,8955717,0,false, -2025-11-24T23:29:34.01281-05:00,2025-11-24T23:29:52.714499-05:00,18701.00,558F84A38D134C414B2589D32B30082078C3285E1E05D83406BE648A923CCBFF,8955717,0,false, -2025-11-24T23:29:35.010764-05:00,2025-11-24T23:29:52.820569-05:00,17809.00,538DA9A5F156748E567C430E6B50E65217501AAA72E401D17B997D487F9B4073,8955717,0,false, -2025-11-24T23:29:36.024857-05:00,2025-11-24T23:29:58.251224-05:00,22226.00,0014B732ED282C5D69E9550522AF23E05B7201D2BD36608B57B45816C55FAA14,8955718,0,false, -2025-11-24T23:29:37.011508-05:00,2025-11-24T23:29:58.55681-05:00,21545.00,972F39B103F124D74CA1CA0652707D5FA90C027D98AFC9F47A6EDE18C9D138D8,8955718,0,false, -2025-11-24T23:29:38.008865-05:00,2025-11-24T23:29:58.760735-05:00,20751.00,B769B415D90BF59A0ED171858B4F3C74F325D535E048355D7C2EBAF7CAFA7B4F,8955718,0,false, -2025-11-24T23:29:39.01153-05:00,2025-11-24T23:29:58.867871-05:00,19856.00,19A7FB4B91FE279F3C799DA26B327FA4B1583303EC7E53AAE586C5BF4875A4DB,8955718,0,false, -2025-11-24T23:29:40.013085-05:00,2025-11-24T23:30:04.376415-05:00,24363.00,3F202ACD6B31183EAE6D894BAC39DD78DA241C72E949045998C28D10676D5986,8955719,0,false, -2025-11-24T23:29:41.01202-05:00,2025-11-24T23:30:04.870542-05:00,23858.00,578CFF44505CAF743F3133815D55ED3FC03C7779C85DDAA69CD8B08C14C311DE,8955719,0,false, -2025-11-24T23:29:42.014416-05:00,2025-11-24T23:30:29.102982-05:00,47087.00,EB8AE4277E6B4E40B838DAC1D7E940E3FB3DE9D6E17255FB8FA55B2FFF3902BC,8955721,0,false, -2025-11-24T23:29:43.009887-05:00,2025-11-24T23:30:29.414156-05:00,46403.00,5424A9DAB82931E91AC61C7DD399EC877FABB5560717F906E562F980AA141E90,8955721,0,false, +2025-11-25T13:48:10.469587-05:00,2025-11-25T13:48:34.514825-05:00,24045.00,1417D63DBB18D9C274142CEA69E72F2307BBE7B8896C5B514A300D043DC1FDB6,8964073,0,false, +2025-11-25T13:48:10.990319-05:00,2025-11-25T13:48:40.216853-05:00,29226.00,4FDA253CAFFB1C53A3B4EA0F10D8F286AD5E0BBAF16CDBD0A421B4E6E2536208,8964074,0,false, +2025-11-25T13:48:11.461864-05:00,2025-11-25T13:48:48.426009-05:00,36963.00,5452F2DEF0DAD0A011A612856BD26E8081187040200F67423741E5B57BAE65EC,8964075,0,false, +2025-11-25T13:48:11.955052-05:00,2025-11-25T13:48:49.864709-05:00,37909.00,09B1808929BB74E1895EF218AA706021823CD3F37AAC22AC77E4C23DFAE0E04D,8964075,0,false, +2025-11-25T13:48:12.457466-05:00,2025-11-25T13:48:52.667233-05:00,40209.00,2A24A1156818A4E201F384F349ECBF22E409E5ED59C6344370158ADDCE2D6AFD,8964076,0,false, +2025-11-25T13:48:12.957885-05:00,2025-11-25T13:49:10.249057-05:00,57290.00,85167AF5965F5F1F489B05F21ECF1EB4A70313FC5A37921A5F8D5991D295B155,8964079,0,false, +2025-11-25T13:48:13.451362-05:00,2025-11-25T13:49:10.686423-05:00,57234.00,45D7020E98D37764F590D67629DA9DF38A6AFE726337414CB2692F9F92CEE172,8964079,0,false, +2025-11-25T13:48:14.229954-05:00,2025-11-25T13:49:11.596121-05:00,57365.00,E5B0F04CFD3FD588CF921EDE73AFC252D5C2DF2E435C9F940BE1796CF9627A6F,8964079,0,false, +2025-11-25T13:48:14.51819-05:00,2025-11-25T13:49:12.04369-05:00,57525.00,E0692F181FBE1C3E947CAD719BE041B5BCED4C43F5E153FC8D481D6BF848851A,8964079,0,false, +2025-11-25T13:48:15.044397-05:00,2025-11-25T13:49:16.235402-05:00,61190.00,F2173EEE37EC668A750981359E2EF1EDEBA31B92CBCF4334D72F70D499236165,8964080,0,false, +2025-11-25T13:48:15.455617-05:00,2025-11-25T13:49:23.574788-05:00,68118.00,19A8F06F3D7677376D4B86255603D5A890F347913D928991514849A811BB6BF1,8964081,0,false, +2025-11-25T13:48:15.955139-05:00,2025-11-25T13:49:37.24723-05:00,81291.00,DC9E4355F5C30788B7614B558461F591EA284B1FAF9C9FC3BA3A576E3CEA90AF,8964083,0,false, +2025-11-25T13:48:16.503404-05:00,2025-11-25T13:49:37.678959-05:00,81175.00,0D52064CECC5CC5D5E52CBAD23F0CF003F95816FF2ACEC26D0D5AFBCE385A716,8964083,0,false, +2025-11-25T13:48:16.953787-05:00,2025-11-25T13:50:00.437231-05:00,103483.00,D6F84D40EF0112D9DD9961F1AB7AA54DA739E0E983C1B5F99BC1BE2DE9711EC6,8964087,0,false, +2025-11-25T13:48:17.452143-05:00,2025-11-25T13:50:01.465036-05:00,104012.00,B83479B3C5F07169ABA3CC1FDB3B92213F197079B203AB34257A484DE8970FF7,8964087,0,false, +2025-11-25T13:48:17.964815-05:00,2025-11-25T13:50:01.915616-05:00,103950.00,F600403624BE504E6F0FF618BA3A9B74E2FFC1282F224AD0E5722C956F3FACDC,8964087,0,false, +2025-11-25T13:48:18.454473-05:00,2025-11-25T13:50:02.365556-05:00,103910.00,382AAC3D79AB63A75C6BB49B669F1E962F88FF1F90AA20D3681B466183398E13,8964087,0,false, +2025-11-25T13:48:19.059393-05:00,2025-11-25T13:50:02.824532-05:00,103764.00,EE67271A0C19AE1B290F559B6514A3877526BB2918627AC1604CD76A537695C2,8964087,0,false, +2025-11-25T13:48:19.471271-05:00,2025-11-25T13:50:03.265233-05:00,103793.00,822970732195D4D374096F0D344DED3BBB6E58B9ED391F903BA771ABC9ABF204,8964087,0,false, +2025-11-25T13:48:19.958298-05:00,2025-11-25T13:50:26.197531-05:00,126238.00,A2104FD5B93BBA6905A31B15B04031B80B59EBECABE778148CB2EAF3E567D97F,8964091,0,false, +2025-11-25T13:48:25.506018-05:00,2025-11-25T13:50:31.247606-05:00,125741.00,C346DB22E483C231CD13A9B51EBEBC89AC700AAF01AFDF9622EC854CA48E936E,8964092,0,false, +2025-11-25T13:48:30.955432-05:00,2025-11-25T13:50:43.468189-05:00,132512.00,0042E8AB7A4F17895B3293F1162D0FCB9EE081FDC03760FB98943917AB5C9C17,8964094,0,false, +2025-11-25T13:48:32.501139-05:00,2025-11-25T13:51:01.898965-05:00,149397.00,F8B9D6B4F6EE140BCDF617490808BE7D2AA207312E58C038402E409865409316,8964097,0,false, +2025-11-25T13:48:47.454571-05:00,2025-11-25T13:51:07.206247-05:00,139751.00,A4543018D3B09D526372BE69ABF3B97AD9AB847245C03490DEC209D06612ED1E,8964098,0,false, +2025-11-25T13:48:39.962429-05:00,2025-11-25T13:51:13.216949-05:00,153253.00,E0DFE3FEAE3CAD17C0E77C1237EB5C7ED9E4ABCA6EC5D735DA9B7AD22C9DF4A3,8964099,0,false, +2025-11-25T13:48:34.97279-05:00,2025-11-25T13:51:13.555864-05:00,158582.00,9C455B17C54F35640E9BF8E1479A03A74C5C6E8A8B026E09D8A495047DBC35DB,8964099,0,false, +2025-11-25T13:49:04.952729-05:00,2025-11-25T13:51:13.895879-05:00,128942.00,1C0FF8D075474C668119A22D056975186E34748B09E3E098CE7D364D30365EB0,8964099,0,false, +2025-11-25T13:49:05.457962-05:00,2025-11-25T13:51:14.100253-05:00,128641.00,8247F9F412EBFB4BABC8DA798639A1CE9B7023C5AF8FF53E04773C4C734E02FE,8964099,0,false, +2025-11-25T13:49:11.953559-05:00,2025-11-25T13:51:27.080069-05:00,135125.00,A2F3E04A48AB9CE7DBA4C108C2D301FC70BF83CF0E01517DD5D5612A490DF89F,8964101,0,false, +2025-11-25T13:49:05.960184-05:00,2025-11-25T13:51:27.853787-05:00,141893.00,3C4A7FD331CBC33C8F7098D4D5F0AFF941D9AE750C8722B5EC612A4DD37330D7,8964101,0,false, +2025-11-25T13:48:35.951218-05:00,2025-11-25T13:51:31.582743-05:00,175630.00,AE17EC48AF208A3A54DE649AB6E06A75AA27E7E88DAB234A9BFDF08B112488CC,8964102,0,false, +2025-11-25T13:48:22.460613-05:00,2025-11-25T13:51:31.91296-05:00,189451.00,770662F787FB1D5ABAD6A030C312D8EB31CE26EFC51D7BB049F2CC28A2386FC6,8964102,0,false, +2025-11-25T13:49:36.44966-05:00,2025-11-25T13:51:32.025144-05:00,115575.00,12E862D45E48B1E2F53C2055012B1E447BEEDC88CC4A90BED2E1C6088B07AFFA,8964102,0,false, +2025-11-25T13:49:36.955763-05:00,2025-11-25T13:51:43.664514-05:00,126708.00,D4A9131897CA1DC1E32BC229F10794F6F1AB0C963FD2F376783B6BC10222F48B,8964104,0,false, +2025-11-25T13:49:50.9625-05:00,2025-11-25T13:51:46.191092-05:00,115228.00,1E242DC687E039CFA8295E09E1D902B8A23721746939EBAB4DC2C71C71275CF6,8964104,0,false, +2025-11-25T13:49:42.455665-05:00,2025-11-25T13:51:49.767462-05:00,127311.00,B6D16AD4FC589A3BE676CC25ADB182F813482367B45324EA6F26B504E59DED3C,8964105,0,false, +2025-11-25T13:50:00.454877-05:00,2025-11-25T13:51:50.169968-05:00,109714.00,3B9640F84031D61BE44817ED6577DEA351D47D4F8A59C973B15A3F498AB9328B,8964105,0,false, +2025-11-25T13:49:53.454262-05:00,2025-11-25T13:51:52.183497-05:00,118728.00,4A60639462E3E6776EA482190AAC370E06949C65DFCBC84D6410975A763A26F7,8964105,0,false, +2025-11-25T13:49:59.451595-05:00,2025-11-25T13:51:54.085983-05:00,114633.00,65BC5DFC003C4867D8A0214E34C0E9E61943A4E0219A0D39FFE748D2E0745FB2,8964105,0,false, +2025-11-25T13:49:27.959797-05:00,2025-11-25T13:52:04.284336-05:00,156323.00,FA9DDCF78DC8C675633DB0D35F612CBDF41E0C5354D57F3106B14D5BF45D95F5,8964107,0,false, +2025-11-25T13:49:16.957181-05:00,2025-11-25T13:52:10.190262-05:00,173232.00,6C8989FFA61BF777595015CF3E76FD8B3D7147DCF61AC1A037CB9313EC2876D5,8964108,0,false, +2025-11-25T13:50:37.949278-05:00,2025-11-25T13:52:16.295085-05:00,98345.00,C710A0ED7E65B7A9650F82C72138A5BA16122BAF17ADEDA25419228AB70F1A9C,8964109,0,false, +2025-11-25T13:50:54.956819-05:00,2025-11-25T13:52:16.732426-05:00,81775.00,AC30567AE3CAEE928C0ABA78B47E8647CDF30D07B145AAC593B9DDAD3356863C,8964109,0,false, +2025-11-25T13:49:43.451022-05:00,2025-11-25T13:52:17.062678-05:00,153611.00,73CFA77591D1889FCC8B0539862AE1DBC51ED79EFD5468C675455C785539B695,8964109,0,false, +2025-11-25T13:50:09.45423-05:00,2025-11-25T13:52:28.209857-05:00,138755.00,01E940C6E39D344FEBCC0E106D6345E8675764391CBD30C8CF099CF5603471DF,8964111,0,false, +2025-11-25T13:51:12.955409-05:00,2025-11-25T13:52:28.543264-05:00,75587.00,B26C426DB3AF0A2E0642B6FB17F29C6AE786BED009E6B60B1380EAD6FB8C29E3,8964111,0,false, +2025-11-25T13:51:09.463767-05:00,2025-11-25T13:52:34.494653-05:00,85030.00,D9C64DE0BEC71CA02131128A6140B49E0E71226A6B9E0C2ECC04D7AB7ADBA9ED,8964112,0,false, +2025-11-25T13:51:07.955161-05:00,2025-11-25T13:52:34.942129-05:00,86986.00,0D3B86FDBAFADF2F0B2F372A94395ABAD0BC989628B1E7EBBE53702912013CF2,8964112,0,false, +2025-11-25T13:51:23.954697-05:00,2025-11-25T13:52:46.29119-05:00,82336.00,3F55807E33819FDD822CB7279654F96CB112C25230177D00F33F3010B59FC9B9,8964114,0,false, +2025-11-25T13:51:06.955651-05:00,2025-11-25T13:52:46.724854-05:00,99768.00,54CD729E86CEA30B0FD577C5F7A72BDCE185D400F76E4A9DD2D4C5CA4CA14E3A,8964114,0,false, +2025-11-25T13:50:58.454139-05:00,2025-11-25T13:52:52.541816-05:00,114087.00,17C8B1C87D1A4575CC729A82FD2CE14EE530FCFC150034EC3AD449CE8B351920,8964115,0,false, +2025-11-25T13:49:20.523337-05:00,2025-11-25T13:52:53.089297-05:00,212565.00,B0A04D51E7E06F665F00422589DAE7593B893F75A4F8ACF72EB8296A659C83DB,8964115,0,false, +2025-11-25T13:49:58.953747-05:00,2025-11-25T13:52:58.209962-05:00,179255.00,458F02D9C9105934947BFA43F2BAA410DE9DEBE75A431B92309E2EB144EB3B31,8964116,0,false, +2025-11-25T13:51:22.459464-05:00,2025-11-25T13:52:58.433587-05:00,95973.00,F2D9D193E41EC897417C9F58EA49FD4097DEA89ADBEE3FD7519521DE19F3ECDC,8964116,0,false, +2025-11-25T13:51:47.953963-05:00,2025-11-25T13:53:06.770149-05:00,78815.00,4CF362B3D3622442F564E49E457B9AB02548DCF6DD1958264076B31C1AE9B74E,8964117,0,false, +2025-11-25T13:51:49.967863-05:00,2025-11-25T13:53:07.324242-05:00,77356.00,55EB31947A30D22B442D4441D5064F959D8A07C32DD2D5D3E2232C233E0623F5,8964117,0,false, +2025-11-25T13:51:47.45262-05:00,2025-11-25T13:53:07.873986-05:00,80421.00,0BD6B9606D11AA77E6A5E6FE1ADED47F8E67979B3C379827EB7F9822F23086C3,8964117,0,false, +2025-11-25T13:51:37.951332-05:00,2025-11-25T13:53:08.869704-05:00,90917.00,BD1B68EA8E2143837ECB079D074EA1E39C386F846C7D1B43D5580DF8F8F0FD23,8964117,0,false, +2025-11-25T13:51:36.453015-05:00,2025-11-25T13:53:09.31193-05:00,92858.00,E88D94897653BBA68D235814AF3BF0E94B6E40BD557DE08B93E05CE689A96D0A,8964117,0,false, +2025-11-25T13:52:00.96091-05:00,2025-11-25T13:53:16.365024-05:00,75403.00,8912F2D64B89751D0C8C7F449CC1DE89F31C48C503AAAAA3B11D04F743148FB1,8964119,0,false, +2025-11-25T13:51:16.455909-05:00,2025-11-25T13:53:28.841556-05:00,132385.00,A53B57C1F69DE2D9B3613730D8B6BA6E41CEF26D3C89D4DD035A85F29F76F557,8964121,0,false, +2025-11-25T13:50:51.459696-05:00,2025-11-25T13:53:34.266857-05:00,162806.00,5183E2198FA7B54EAFEF5B8CDF6D1BE8883E26C0CFA252101190C1FF4487C276,8964122,0,false, +2025-11-25T13:52:10.963258-05:00,2025-11-25T13:53:46.850359-05:00,95886.00,48989A4401EE3749770F30247C079E2CEDF5ABB74124B94ACF257831E3DA6792,8964124,0,false, +2025-11-25T13:52:09.462673-05:00,2025-11-25T13:53:47.096964-05:00,97633.00,E3594231C96FDA382106A4218A250F6733B1E9E1B13570C0038271C2B80A6B6F,8964124,0,false, +2025-11-25T13:51:15.460722-05:00,2025-11-25T13:53:47.420617-05:00,151959.00,FAA72793C733D96F7EE248313A6844B96E864F49343101D4C83A07DC439C1B33,8964124,0,false, +2025-11-25T13:50:06.458426-05:00,2025-11-25T13:53:53.265231-05:00,226805.00,4E2362455FC30CA32ED8F3FC5C6CBA5CE125BF9607F85DF08730707032A678E5,8964125,0,false, +2025-11-25T13:52:34.9536-05:00,2025-11-25T13:53:54.176634-05:00,79222.00,7CA21DEB978262498A107A88061645E1C6FA0AF41918F66F2571F68AD5C8126B,8964125,0,false, +2025-11-25T13:52:28.458847-05:00,2025-11-25T13:53:57.6149-05:00,89155.00,3BA057CA26FF070BC17136D626B80313C5DB2AB9FB06F3EDF712CD6E25CA0990,8964125,0,false, +2025-11-25T13:52:33.46421-05:00,2025-11-25T13:54:10.924027-05:00,97459.00,4258E8C9E212E592B687851543A9519FD2AD8344D681BA99E031EABBFD72A6A3,8964128,0,false, +2025-11-25T13:52:14.451943-05:00,2025-11-25T13:54:11.240559-05:00,116788.00,6F2A60D7AD93198216D5C1BB56B6064E913BB763470000278D6474EAD7AEC66E,8964128,0,false, +2025-11-25T13:52:30.960166-05:00,2025-11-25T13:54:11.58349-05:00,100622.00,9FD1A75A662A3B0E86A9E5A956340CE45633A05B57B342C684C729FEB940F16E,8964128,0,false, +2025-11-25T13:52:42.458842-05:00,2025-11-25T13:54:13.2365-05:00,90777.00,4F3E3B885126441921A0B6C415309CB54217E52A5C3BD278BF774A8A9F3F4540,8964128,0,false, +2025-11-25T13:49:35.976522-05:00,2025-11-25T13:54:13.570487-05:00,277592.00,6A9DE737788154DC6B37216B13F91688468C49977F9F8ADC75823548D3DE8DA0,8964128,0,false, +2025-11-25T13:48:28.954543-05:00,2025-11-25T13:54:19.185576-05:00,350229.00,14402C582F8F4F7B4BC632DFD7912652C3518C99423263AB3149E3E3E4ED25B4,8964129,0,false, +2025-11-25T13:53:04.460096-05:00,2025-11-25T13:54:19.506956-05:00,75046.00,4089B6323334CB28DE0BDC66AD0B4D28AAD77D5BEDF1135BDCD7D26601CC2EAB,8964129,0,false, +2025-11-25T13:53:00.451394-05:00,2025-11-25T13:54:45.854022-05:00,105402.00,AE931D6FCA5FBC16A9F1A31B3388E856DD42D20AAA522DA0AA09F661EE13B568,8964132,0,false, +2025-11-25T13:50:21.464743-05:00,2025-11-25T13:54:50.75106-05:00,269285.00,F814EE519BF5D3FFA98DD4133920C0E4555FE53FA916251E97954529059ADE4F,8964132,0,false, +2025-11-25T13:52:46.963175-05:00,2025-11-25T13:54:52.249188-05:00,125285.00,4BD5F6F9A626BD78B8F42352EDD30A41DA2EDF91DD18C5AC435EDF48BC9158C7,8964132,0,false, +2025-11-25T13:49:21.954812-05:00,2025-11-25T13:54:53.15501-05:00,331198.00,CFC2FE61061B564B3737DCAC50C4EA4EA09BC34B9A2695FE0A89A095B1A64C79,8964132,0,false, +2025-11-25T13:53:14.96158-05:00,2025-11-25T13:54:54.164477-05:00,99202.00,91DA56DADE9E117843544148540097A729D99BAE351110E447A50A29DBB492AB,8964133,0,false, +2025-11-25T13:49:18.452645-05:00,2025-11-25T13:54:54.840692-05:00,336386.00,9E34E67D09815F609880DAC6AA73572CA9A0C768179FCE0DC6A3B541D01F56F1,8964133,0,false, +2025-11-25T13:50:17.462998-05:00,2025-11-25T13:55:06.709371-05:00,289245.00,0E015ADAF7E7A8C9DB8561EF5FEA910AE20CFCE788D4D57AF8D11B40FF15CB55,8964134,0,false, +2025-11-25T13:53:45.95256-05:00,2025-11-25T13:55:07.157246-05:00,81204.00,8454675A1424A6F63F1F5CA07FC72A749B042CEC49F5EBAA85A2BA7B2291D948,8964134,0,false, +2025-11-25T13:53:29.453359-05:00,2025-11-25T13:55:07.710616-05:00,98256.00,DF24532A25DDC2924C05C0959CDAFD88BF6A679D4F84178FC7A7B83AADED2A21,8964135,0,false, +2025-11-25T13:49:37.457772-05:00,2025-11-25T13:55:08.15371-05:00,330694.00,1044EF27C67217A034DE7644E27DAAA1C94AEB6A1CF8C08731D67C09FBF0E27B,8964135,0,false, +2025-11-25T13:53:50.457521-05:00,2025-11-25T13:55:08.60182-05:00,78143.00,912CDF6859E0D3057B095213B0B2B5035AC461FEF632D56E3400D4F280708471,8964135,0,false, +2025-11-25T13:51:44.45451-05:00,2025-11-25T13:55:09.049004-05:00,204593.00,C667182B58AA9D32D2DD2EB915120F7F4102522D081AEF3C677083C527A92C83,8964135,0,false, +2025-11-25T13:53:56.951531-05:00,2025-11-25T13:55:09.275018-05:00,72323.00,08959D91D67B5BC689AEFE59B6E4DB19A25097D582CEA8698C63F9FF33438610,8964135,0,false, +2025-11-25T13:53:32.969805-05:00,2025-11-25T13:55:09.38789-05:00,96417.00,908420F5DB8FF0A2D178B6945049484C6B9EC027C7B2568E9018B7E709306293,8964135,0,false, +2025-11-25T13:51:03.453035-05:00,2025-11-25T13:55:09.777708-05:00,246323.00,DF0B0B142C016A878D561BFFEE77221BA1124E2DBCBEDFE45F65790E18AD19B2,8964135,0,false, +2025-11-25T13:54:09.466415-05:00,2025-11-25T13:55:11.334672-05:00,61868.00,2FEF8FCD677864521EDD2BAB6C38BB12DED2AD6F998CEE963352C695BA6E370A,8964136,0,false, +2025-11-25T13:54:12.952606-05:00,2025-11-25T13:55:11.669875-05:00,58717.00,627DF21382C7B25899242EE98BFBF454CA0870BB3822C3E73742FA08FD756DD4,8964136,0,false, +2025-11-25T13:54:10.957612-05:00,2025-11-25T13:55:25.254859-05:00,74296.00,6E04DFEE7A07848B84370C82B3FB06E9EEA4B9D7101460C48FB7C8C15CCDC125,8964138,0,false, +2025-11-25T13:54:14.963552-05:00,2025-11-25T13:55:25.592574-05:00,70628.00,BB57DC81F2C0D120E98B7FA621B54C39A2E7A2A70E1C6C47BC92D6B56D61CCA3,8964138,0,false, +2025-11-25T13:48:26.979564-05:00,2025-11-25T13:55:26.031003-05:00,419049.00,A9A8259219EB2CF9AFD67B137BEE7CA4CB8A7F17B5D54C8A3BBAC141B0BFB84C,8964138,0,false, +2025-11-25T13:54:45.464977-05:00,2025-11-25T13:55:38.322816-05:00,52857.00,DBFFF7CF5689C5561F270757BA2995BBF1AEC9821F5272B56E31BAEC8119E951,8964140,0,false, +2025-11-25T13:54:49.452458-05:00,2025-11-25T13:55:38.886412-05:00,49433.00,63A8B70B08BBE44CF8916FA622485F737D1A46BB3547B1128204963BB24B7D00,8964140,0,false, +2025-11-25T13:50:18.451875-05:00,2025-11-25T13:55:39.440549-05:00,320987.00,E6BE67C9C3D434261AFE734DF374A91A080342296D4B046FB7DD3BB776C3E21F,8964140,0,false, +2025-11-25T13:48:40.4585-05:00,2025-11-25T13:55:43.191179-05:00,422730.00,93D636F6385871439515DA8459EDFD97D36FACC3DB458E9E3EC5E940DFE691C4,8964141,0,false, +2025-11-25T13:54:38.45572-05:00,2025-11-25T13:55:43.62855-05:00,65172.00,D98475045BEFB92DC683BA6CB00565EFE0480EF4B9FBDF19F551F2910F8047E2,8964141,0,false, +2025-11-25T13:54:54.456495-05:00,2025-11-25T13:55:55.520285-05:00,61063.00,63A922058BAF20CD47B6F41D9FF6E81436BE2C10CB3DEFD9FD1440FA422C91F5,8964143,0,false, +2025-11-25T13:55:00.953232-05:00,2025-11-25T13:55:56.072846-05:00,55119.00,1786B75728D43C264E4BBAEFD65714283C5F8EF7135F6D3ACD1CA25E8D4B8E00,8964143,0,false, +2025-11-25T13:55:05.958605-05:00,2025-11-25T13:56:01.255465-05:00,55296.00,0CBF53F8BCD5D7B333128830B745826B245FA5857E4048E4D4EFA4308E36DD1F,8964144,0,false, +2025-11-25T13:55:01.457741-05:00,2025-11-25T13:56:14.078695-05:00,72620.00,EFB44A8840D29B448803DDDAA5198699207353D46D259ADF3DD871F8C61FF798,8964146,0,false, +2025-11-25T13:55:07.954994-05:00,2025-11-25T13:56:14.522094-05:00,66566.00,FEE7B5FA59DB4BA373E3D052BF6943C041F120A1D4B14C988DEAF92DBB9F4E90,8964146,0,false, +2025-11-25T13:54:37.956723-05:00,2025-11-25T13:56:19.249047-05:00,101291.00,4ED0FCA16FA569C4312CDF3981814D9DCCACD8973DA743C55648856D33D5EAF9,8964147,0,false, +2025-11-25T13:51:38.956961-05:00,2025-11-25T13:56:19.675848-05:00,280717.00,8EBD8ECF5CC19E746AE2EECC6F0179550250F1037132EE4632DE63621D1C415B,8964147,0,false, +2025-11-25T13:55:06.470813-05:00,2025-11-25T13:56:32.84859-05:00,86377.00,0DF3CD68EC5566D41192662E12CEA3A743D70BF5B6BD164B353F31A74F06B01A,8964149,0,false, +2025-11-25T13:55:04.959042-05:00,2025-11-25T13:56:37.294091-05:00,92334.00,5174A380B42DBF0AEE785477B4C51E43D6F64AB9052A81FC0E1E24E86AE7EA20,8964150,0,false, +2025-11-25T13:55:03.461531-05:00,2025-11-25T13:56:37.757953-05:00,94296.00,8FEDCAAB758049FD743C94D105BF16E96C7D3A89A36B9678131313B1A2FC6D7D,8964150,0,false, +2025-11-25T13:55:10.951053-05:00,2025-11-25T13:56:38.113754-05:00,87162.00,BA475ED338691A3230C20E5619F0260D09F491FCB260F272474CFDDE1CCD2E0C,8964150,0,false, +2025-11-25T13:54:44.458426-05:00,2025-11-25T13:56:38.542043-05:00,114083.00,8E610BD9BBE3A14CE9DA8688BF9C84781AA6994BD359B270510D4651F15707AF,8964150,0,false, +2025-11-25T13:51:41.975842-05:00,2025-11-25T13:56:49.838057-05:00,307860.00,2CBD908C0B332F46E8A84324F13EC2EF1769BFE34B4FB3C3D52925EFDBFB682F,8964152,0,false, +2025-11-25T13:55:22.451344-05:00,2025-11-25T13:56:50.302732-05:00,87851.00,E84D57301F3DF05FF5F743CCBDCE80278659EF1923EFC6134DAF775F1E397AD0,8964152,0,false, +2025-11-25T13:55:25.960776-05:00,2025-11-25T13:56:55.227103-05:00,89265.00,1DA5D49596B5C8DB770FCC552101D25AEDF447CAAA704BF87B5B3B4088886EB3,8964153,0,false, +2025-11-25T13:53:01.953212-05:00,2025-11-25T13:56:55.341647-05:00,233387.00,8470738763CF3E7B7C844A4808CA3BD13857A67B55722C3C0DEC48CBAF220510,8964153,0,false, +2025-11-25T13:50:00.961091-05:00,2025-11-25T13:56:55.634207-05:00,414671.00,A0EEA49F46DDBD8C810CE9C0880B7408D7BB511757E8F63481F8436410AD5B8B,8964153,0,false, +2025-11-25T13:53:53.4608-05:00,2025-11-25T13:57:07.073075-05:00,193611.00,B54134DF60C68B55C3DFECB342D309B9CCE9CA709D37188E1CA2175DF8BACD96,8964155,0,false, +2025-11-25T13:55:38.457981-05:00,2025-11-25T13:57:07.198936-05:00,88740.00,7869642CFBCDCA836D7AE2B560B3FE23CD76B370B0C52D8044B26B8D7CA8D68F,8964155,0,false, +2025-11-25T13:50:02.457051-05:00,2025-11-25T13:57:09.041472-05:00,426582.00,87F6EE1EF9EE67DF8FB943BE5065A624539E815E8255C140D3BBF80F98F17888,8964155,0,false, +2025-11-25T13:55:54.450537-05:00,2025-11-25T13:57:09.371532-05:00,74920.00,CC06EFF3BB5EDBE725732CD32E89527D10E7708E5D0F32A404676E340B5DCEB8,8964155,0,false, +2025-11-25T13:54:15.464415-05:00,2025-11-25T13:57:13.270611-05:00,177805.00,35F95B3A29ED87047B8BBB69D757A79F8582AC90B91DB71182A9EB8D61581D41,8964156,0,false, +2025-11-25T13:50:41.450557-05:00,2025-11-25T13:57:13.589793-05:00,392137.00,C7E974673C6C20F34F67451A93714AE5B30995BB6EFACC6465A42D20FF5BBF84,8964156,0,false, +2025-11-25T13:55:56.957303-05:00,2025-11-25T13:57:13.711303-05:00,76753.00,3D715AC6EB2E99BE510A25C7E3B3A84CBEA01F7803F74F2BAF5E9B33E3A223CC,8964156,0,false, +2025-11-25T13:56:06.958755-05:00,2025-11-25T13:57:19.182834-05:00,72223.00,4760F6DA795DC423B32EFB40A2E8EAE69C73CDF10F6E54F74A1CE773D298A53A,8964157,0,false, +2025-11-25T13:54:58.456908-05:00,2025-11-25T13:57:25.208301-05:00,146750.00,123CE73550920B8E73377279DFE7CFE887C9C216089564AED582B88321EFFD7C,8964158,0,false, +2025-11-25T13:51:02.455149-05:00,2025-11-25T13:57:27.193053-05:00,384736.00,33DFD6BCA572E473E290B31B57B6D679B3A88E3EEC78B2F376E5BD45D0680F76,8964158,0,false, +2025-11-25T13:56:16.954027-05:00,2025-11-25T13:57:27.516415-05:00,70562.00,17E84EC899A37DE54D3E88CDE9B262AA0B967F9F989E4EC3D04A131E8D9D39F1,8964158,0,false, +2025-11-25T13:52:08.450862-05:00,2025-11-25T13:57:27.840216-05:00,319388.00,1002583C0484B6B87463B47FF7124B38CB40C0E6C41C5EF4015AAC39A01153C8,8964158,0,false, +2025-11-25T13:50:50.463363-05:00,2025-11-25T13:57:28.072803-05:00,397607.00,286482460A8478E56444C0325BE4081C180C22D26D30BF44EF22369B82C477CA,8964158,0,false, +2025-11-25T13:54:01.454788-05:00,2025-11-25T13:57:28.389765-05:00,206934.00,A39CC8911EABF0688CE1313FA2A4EA196404B2AB10F7DFF9BFEC6352CFAA00B5,8964158,0,false, +2025-11-25T13:51:54.951891-05:00,2025-11-25T13:57:28.499749-05:00,333546.00,A0CF75E6A78846C0ADBB8AF394E542BCD173E1290CC4529AEC9C1C9AB9ACB700,8964158,0,false, +2025-11-25T13:56:38.451358-05:00,2025-11-25T13:57:28.610241-05:00,50158.00,71AD4EFF90019F000C2CA1CD398421C1C823C0FDAA0FBEEF0276EB789DF1B85C,8964158,0,false, +2025-11-25T13:52:21.478174-05:00,2025-11-25T13:57:40.253888-05:00,318774.00,65545DCA7119552C8401E7470A726C9D6E598722E2C74937076AFE259F63F429,8964160,0,false, +2025-11-25T13:55:44.455833-05:00,2025-11-25T13:57:43.505184-05:00,119048.00,993DD56FB839A8A1EA78D0947BC3B0E329CD36EC0431F4434084271EA33B2471,8964161,0,false, +2025-11-25T13:54:17.45362-05:00,2025-11-25T13:57:43.725293-05:00,206270.00,9896E8108036CEA5089413981EE1C822C3512B5CE263AC788CDEDCF175D665C0,8964161,0,false, +2025-11-25T13:52:05.453579-05:00,2025-11-25T13:57:43.843774-05:00,338388.00,BC37E2228FCC112A8CDF9A1258A7036A5D517885F126B9FF52B4F4B6030D278E,8964161,0,false, +2025-11-25T13:57:01.958996-05:00,2025-11-25T13:57:43.957451-05:00,41998.00,169B3F8DDCCF04B8305B98F0B99AD9803CAC7280E54A5A8E5F254B266522DD2B,8964161,0,false, +2025-11-25T13:50:20.951247-05:00,2025-11-25T13:57:44.221383-05:00,443268.00,9C637424D42AF06A96F456AF6501D67CF48E0ED7DA5960B73D7E0A91008F92D3,8964161,0,false, +2025-11-25T13:57:07.459615-05:00,2025-11-25T13:57:44.336215-05:00,36876.00,ADF7A75BF7F51AC33CBCC0A978831C4604E4665E5736BD0885142BFB5EABD037,8964161,0,false, +2025-11-25T13:49:40.950314-05:00,2025-11-25T13:57:46.210291-05:00,485257.00,7F7D5B970834F5A1F9F95ED6E9978AD64F27AD65FB3AFA7C5AC85791ACC39B6F,8964161,0,false, +2025-11-25T13:57:12.971464-05:00,2025-11-25T13:58:08.708368-05:00,55736.00,E9BD5749FC75DB83B3EFA253A5F6416A80D9C62C6DCBAE76FDDF3A445D4D485F,8964165,0,false, +2025-11-25T13:57:05.459586-05:00,2025-11-25T13:58:08.820182-05:00,63360.00,10FBDACA81D0AB608A2BCDF4D23A5A3FF2D6E05C6295DFE9095E1ACD7FF4FB08,8964165,0,false, +2025-11-25T13:53:25.95518-05:00,2025-11-25T13:58:09.14494-05:00,283188.00,CDCA1701C050D4A7E0514B628615ED7938AB37596FEA17E746AD9DDCC904D367,8964165,0,false, +2025-11-25T13:55:43.457621-05:00,2025-11-25T13:58:09.257512-05:00,145799.00,979DD3011154C6540D0A3E7E6CD594C0BDEEAAA6AF263EA31ADA575B2E4F1098,8964165,0,false, +2025-11-25T13:57:15.965412-05:00,2025-11-25T13:58:09.540623-05:00,53574.00,394813CC77B1DE23D16398B7F69B0204CC33798E6F4466AC2F8576D4A3A28755,8964165,0,false, +2025-11-25T13:52:58.952003-05:00,2025-11-25T13:58:23.148442-05:00,324195.00,C19BFD431351F367A86A8236A15F19529BE84E6F67B6850083506485B010A1BA,8964167,0,false, +2025-11-25T13:55:16.47938-05:00,2025-11-25T13:58:23.283599-05:00,186803.00,A7E22F99ACE7526E6D062D903E8D96C024B0C6D8088E8DD6C6DDBD764B4BE7DF,8964167,0,false, +2025-11-25T13:51:33.454211-05:00,2025-11-25T13:58:23.520543-05:00,410064.00,2A2C7C83860F9F0B2A294D3F8719DE8C6F499366F95C649FC862C859BE4ADA9D,8964167,0,false, +2025-11-25T13:48:50.454921-05:00,2025-11-25T13:58:23.632735-05:00,573175.00,2FC43C7F80B95AB31CA83409A8843B5F9CEC093BD5BA82FBAA9C5F633684AC98,8964167,0,false, +2025-11-25T13:57:04.954309-05:00,2025-11-25T13:58:28.004326-05:00,83049.00,B9FD1B9D8C68E4EA26ECC2DE43F971FE6C45BC3ADF5403AACF769311017A684A,8964167,0,false, +2025-11-25T13:51:23.461903-05:00,2025-11-25T13:58:28.122244-05:00,424658.00,DC83A898DCC22937C07D60641748CF37AE046539AEF0E7362CF58E14C0FAD10F,8964167,0,false, +2025-11-25T13:49:58.453007-05:00,2025-11-25T13:58:28.76699-05:00,510311.00,2FBBA863088FACCF6E99F470D37E146C7F06A1F10E104592C09D2EA6D2EE5CA7,8964167,0,false, +2025-11-25T13:57:25.451922-05:00,2025-11-25T13:58:28.886549-05:00,63434.00,C265CE9B4DEC4804BC8C16AA3B2A0EE952D66F652B284713AD65D6054E6E1AFE,8964167,0,false, +2025-11-25T13:53:07.456108-05:00,2025-11-25T13:58:36.374743-05:00,328917.00,C17BB2D0262F350BB2B6552C0E5FF9A4D4FB67774A1D0195771D15591558295E,8964169,0,false, +2025-11-25T13:56:29.465107-05:00,2025-11-25T13:58:39.299411-05:00,129833.00,F42C9CAB4E94904CB2FC7E0D613D5D2331B4876E0158F7DE1AC4BAF0484E37D7,8964170,0,false, +2025-11-25T13:55:43.956171-05:00,2025-11-25T13:58:40.311032-05:00,176354.00,E4BBC6CD5F17B74D3CC5FCDC76165D5652CB8DEEAED26CAEAEA69E9AD65AE911,8964170,0,false, +2025-11-25T13:49:28.954715-05:00,2025-11-25T13:58:40.425106-05:00,551468.00,0FD8FEF0540A3928B760A80DF60F5CB3610B9822818D100858557908D94FFDEF,8964170,0,false, +2025-11-25T13:55:54.975662-05:00,2025-11-25T13:58:48.87314-05:00,173896.00,2F5698424FCD9AA620C630C5F637C728E3989DF4034D61AD2A382FA46C631525,8964170,0,false, +2025-11-25T13:57:43.454912-05:00,2025-11-25T13:58:48.980033-05:00,65524.00,E4FCFE21101B5A14A53A1BEE081F870FC278242674CEA1858508A0EAAAF22706,8964170,0,false, +2025-11-25T13:49:18.952464-05:00,2025-11-25T13:58:49.092474-05:00,570137.00,2E1FC8667A3D7CA9A7856055D822594881948BEA417FC80C096B6ABF74D714DE,8964170,0,false, +2025-11-25T13:55:57.963301-05:00,2025-11-25T13:58:49.317538-05:00,171353.00,136F550A4C245E0A9C021F969DB58568B2EF67189B6DE9CD2980485C00BEB590,8964170,0,false, +2025-11-25T13:54:59.955822-05:00,2025-11-25T13:59:12.839333-05:00,252882.00,41DD19F01FBAF2B007B1651D5CE5DD8ABBDAC37CB22D9F853F1B9AA2FFD445BD,8964172,0,false, +2025-11-25T13:52:36.955431-05:00,2025-11-25T13:59:12.955126-05:00,395998.00,47275F384D9F6B212C2EF1EEA5D1C0F3B94E33E4EEDDD8F7968D7063BA4D3102,8964172,0,false, +2025-11-25T13:50:01.454537-05:00,2025-11-25T13:59:13.065643-05:00,551608.00,1E4D8A87E6A247FE571C759E327C0868F3A0003518E204E695C2408FBEB57083,8964172,0,false, +2025-11-25T13:58:09.452531-05:00,2025-11-25T13:59:13.178636-05:00,63725.00,0BEC27B07456B3A1DE2DA69B170FD82FE5F4CD9FA40FE84C3F8175A817A7569C,8964172,0,false, +2025-11-25T13:53:46.955942-05:00,2025-11-25T13:59:13.290789-05:00,326333.00,794A89C874253FCAFF8E771B89001B0CD1306A167BC82827B69F26B4B277B0AB,8964175,0,false, +2025-11-25T13:56:14.966289-05:00,2025-11-25T13:59:13.40359-05:00,178436.00,5CD818F22F40B631AE3CC231A205F36AABCCF75A85FD1703F22159154F891AF6,8964175,0,false, +2025-11-25T13:55:53.462506-05:00,2025-11-25T13:59:13.516718-05:00,200053.00,39F681465783F219095FE4BF695419811789A463628E4DA9F1866D417BA7DFF6,8964175,0,false, +2025-11-25T13:58:23.454305-05:00,2025-11-25T13:59:13.816221-05:00,50361.00,94FAB09989C25BBFBB8C9C2B8C4E408A4F807BC5CABBC5B0B760D281F80CD807,8964175,0,false, +2025-11-25T13:53:57.454164-05:00,2025-11-25T13:59:13.927329-05:00,316471.00,66A4BFCDA9173C2DC724B682C5118366BDE0A495514B7577EF38DB8CDED2700C,8964175,0,false, +2025-11-25T13:58:25.979583-05:00,2025-11-25T13:59:14.041127-05:00,48061.00,3158B53F46D2414D386EA225915E603ACE1AD048965E74A42DC370CEC0DCD7D2,8964175,0,false, +2025-11-25T13:57:02.471042-05:00,2025-11-25T13:59:15.896071-05:00,133424.00,9F9875B8A7B4E888369C2D3CA807A6A9B5BFAEE7C6DE94DE6A6F9CEE81DEA463,8964176,0,false, +2025-11-25T13:56:55.957606-05:00,2025-11-25T13:59:16.017099-05:00,140058.00,B73B81BA21A81E305142C2BF63E55A5C5FC081ED318F9EEACC41B7C258F95FDB,8964176,0,false, +2025-11-25T13:56:43.965824-05:00,2025-11-25T13:59:16.136661-05:00,152170.00,1EA88022349DFCA659DB045E50F81690243087913B94B9C9A8B943104D8E7B18,8964176,0,false, +2025-11-25T13:50:30.962388-05:00,2025-11-25T13:59:16.254416-05:00,525289.00,515D565DDA4A1006C93FC62A3DC0C04E97E3F6055C056448633C0E24927DDD8A,8964176,0,false, +2025-11-25T13:52:01.949988-05:00,2025-11-25T13:59:16.37126-05:00,434419.00,AECDDF3F24F7588659A2599A263A9246D62B2A4A298CEB619959917D2F459CF2,8964176,0,false, +2025-11-25T13:59:12.959135-05:00,2025-11-25T13:59:33.737754-05:00,20778.00,93CB0E086A1717E3AC4068644DB99B28B327CF3F32EC18673032ED7858517615,8964179,0,false, +2025-11-25T13:57:06.951998-05:00,2025-11-25T13:59:34.564163-05:00,147611.00,927203B61CCC7C04BC506B76575692730F75F5D865754965AA1FC48E7DA840FD,8964179,0,false, +2025-11-25T13:54:01.957394-05:00,2025-11-25T13:59:34.977107-05:00,333018.00,3B218E4E8880B483C80222A14C86E11F1F80D08BFF5841D3B4DEA25CB2BBDB72,8964179,0,false, +2025-11-25T13:56:08.46145-05:00,2025-11-25T13:59:36.086633-05:00,207624.00,A740DF5A9752E037D2E89E0B4010B313EECBA4E39FCC4EA14024277F0358DFE7,8964179,0,false, +2025-11-25T13:57:08.554048-05:00,2025-11-25T13:59:36.420774-05:00,147866.00,29B2A803F0D899A48E37D3D92D0B6F6F98BE230B103F7D6E902C3F1C379F8A9C,8964179,0,false, +2025-11-25T13:55:12.958324-05:00,2025-11-25T13:59:36.649437-05:00,263690.00,4613F7E5E49B29969F0B0E322C426A0B8477549B0ED1CE8A6E8AF62578D0EFDF,8964179,0,false, +2025-11-25T13:56:22.01081-05:00,2025-11-25T13:59:36.761717-05:00,194750.00,73DBEDA08014486770D41C598493D4054213C7650BA8B1DEAC9E1F03F8D73143,8964179,0,false, +2025-11-25T13:52:04.457537-05:00,2025-11-25T13:59:36.872771-05:00,452413.00,316B58D6EC0FC15DD9CD128E1829182DC3D8EEEBB189B4BA66097E99DCFF098D,8964179,0,false, +2025-11-25T13:53:51.45427-05:00,2025-11-25T13:59:36.984763-05:00,345529.00,7495F7E98000FF38543FA7E8E0BF3AA70E38CA14B8D0CB6181F06E5F348B7ED8,8964179,0,false, +2025-11-25T13:52:08.00772-05:00,2025-11-25T13:59:52.067177-05:00,464057.00,5D5E3EC8C710F79E967B651CFBC34FD59F7724DEEFB13DCDCF23B52C357876CA,8964182,0,false, +2025-11-25T13:56:50.954699-05:00,2025-11-25T13:59:52.200528-05:00,181245.00,C2E75444AD065C330D227EAF4725404DCFD0A4A61986F623783F1B896854B214,8964182,0,false, +2025-11-25T13:55:05.484699-05:00,2025-11-25T13:59:52.516862-05:00,287030.00,D29A6415D37DC5EF99FD25A10DFC61F038A12E3E3BFFB0648E152829F2B450FF,8964182,0,false, +2025-11-25T13:55:02.549706-05:00,2025-11-25T13:59:52.75567-05:00,290204.00,38762D70EAD75AA009794110B3D895004F99CBF3D901466911F398409C1E1F93,8964182,0,false, +2025-11-25T13:54:31.955433-05:00,2025-11-25T13:59:54.587673-05:00,322630.00,8C6F2975E22B99775E5D24CCCD9E79F65925AFBC0CEF39270869EAC439C25934,8964182,0,false, +2025-11-25T13:57:39.451404-05:00,2025-11-25T13:59:57.784849-05:00,138332.00,593524BDDC7A93DFA4E9181914713594B1E260600D433DCF9B6106CAFF968DA3,8964183,0,false, +2025-11-25T13:49:17.95946-05:00,2025-11-25T13:59:58.222526-05:00,640260.00,1CC618C02001ED27D48F96E6385E6EE8E8B2EF5E65BBA4AC78BBC95AD572F353,8964183,0,false, +2025-11-25T13:57:26.957384-05:00,2025-11-25T13:59:58.559694-05:00,151601.00,57C1B9646D6650772150D4813D4EBACB0F5DD9AF7E965F04AE915495C85897D2,8964183,0,false, +2025-11-25T13:48:45.952805-05:00,2025-11-25T13:59:58.793431-05:00,672837.00,F39901CE943B04F19AF7C28357494B42B75FD54D7C2E0F4B5101D66C7E76D561,8964183,0,false, +2025-11-25T13:55:42.45467-05:00,2025-11-25T13:59:58.90979-05:00,256454.00,33165443FF451E7869420CCE674319DBD75C87362F7D53AEAF1B036EAE593341,8964183,0,false, +2025-11-25T13:50:32.452922-05:00,2025-11-25T14:00:11.757019-05:00,579301.00,69402CCF7E2CB10DF01FFC131CC7267E01285F87343AA34C35998B26F1E6A849,8964185,0,false, +2025-11-25T13:59:31.031749-05:00,2025-11-25T14:00:22.204267-05:00,51172.00,CB59DB973A91C7B66AEC8FCD0DA74ABC1AD0BE3BCD0BC31693C4E5E6EDC1A539,8964187,0,false, +2025-11-25T13:50:13.457609-05:00,2025-11-25T14:00:34.454223-05:00,620994.00,16C1F2DC8AC44D1408DD655D3FC29B19717B7661DC80C338B965384CB974BFF5,8964189,0,false, +2025-11-25T13:59:36.452006-05:00,2025-11-25T14:00:34.573947-05:00,58121.00,5A9E3AA963B0670E3D4837AD4EDB5CB2C6F2F4C292FE74BED95344054040268A,8964189,0,false, +2025-11-25T13:52:48.463292-05:00,2025-11-25T14:00:34.696294-05:00,466231.00,DB86C8D0E7B759BD9F17CB477816C46BC33416B1DC24938698C0D666304367E3,8964189,0,false, +2025-11-25T13:56:58.451752-05:00,2025-11-25T14:00:34.820747-05:00,216368.00,2F53CB41D0894AA11E2376745CFD2D6BC5781A1154D685105B414C6558E39568,8964189,0,false, +2025-11-25T13:54:23.953915-05:00,2025-11-25T14:00:39.622323-05:00,375666.00,5B13B764391697186F01A37133121D7577AB355AB6828BDFA25948C5F762277A,8964189,0,false, +2025-11-25T13:49:35.558369-05:00,2025-11-25T14:00:40.052591-05:00,664491.00,1CD389C50C0A91EB7B08E2B2A40749E4F4D7B5CB69D09965A45D8B10B3FF6BC1,8964189,0,false, +2025-11-25T13:56:36.956227-05:00,2025-11-25T14:00:40.553188-05:00,243595.00,FAB52D044CD9DFC1B1E6600073746FF9423D9D0B242DC933281896F6DEF8EA23,8964189,0,false, +2025-11-25T13:51:50.457479-05:00,2025-11-25T14:00:49.063303-05:00,538603.00,06E73C2F34C55256F7E819AEE3342D7BE0E034CDAD8AB66065E6144BBB66D239,8964191,0,false, +2025-11-25T13:55:23.45253-05:00,2025-11-25T14:00:49.290612-05:00,325836.00,72AB38DD49A5790F98E700B9B86EECF11BD5638C204506E02C4065F484E95891,8964191,0,false, +2025-11-25T13:48:45.455986-05:00,2025-11-25T14:01:06.787187-05:00,741328.00,5130520712ED03E2CBB1B28E3F58CBD311DFBB3DC9C9611BD3B7320C5C7A6EF0,8964193,0,false, +2025-11-25T13:59:51.813984-05:00,2025-11-25T14:01:06.902401-05:00,75088.00,038360FD71D46E2ACD3765FFE8EBCA67F4ECE313283A0CC19A49E2758DC72174,8964193,0,false, +2025-11-25T13:59:52.462137-05:00,2025-11-25T14:01:07.011364-05:00,74548.00,322FE90F18CEE17D2A777C9C1362F96CD10EFE16BC260E4383490C8E329CF133,8964193,0,false, +2025-11-25T13:55:11.462079-05:00,2025-11-25T14:01:07.121384-05:00,355657.00,BD54A228DCA63793C9128D0F08A978F4FC0464B8326D15E3367A6BA9D5C7A0F9,8964193,0,false, +2025-11-25T13:59:57.485649-05:00,2025-11-25T14:01:21.957663-05:00,84471.00,CEF87492B59CF65A4BAC599499A913095B4753DD7116E946E79076824C846FCB,8964196,0,false, +2025-11-25T13:59:52.98817-05:00,2025-11-25T14:01:22.06776-05:00,89079.00,C089A93552B1046076B9B8533EB1FD374560D12CE35D061B142ACC05255C6156,8964196,0,false, +2025-11-25T13:59:58.059756-05:00,2025-11-25T14:01:45.041882-05:00,106981.00,1DB9C85A720E88744572F7D8F92BBDFF2ADE7C52761D63985AD0BE5A2241DBC6,8964200,0,false, +2025-11-25T13:59:55.974839-05:00,2025-11-25T14:01:45.159926-05:00,109184.00,C5326B73B04F5C1EA8DAB093456E26E111FAA0727D32C855F9229A0F063D6BAA,8964200,0,false, +2025-11-25T13:51:58.453947-05:00,2025-11-25T14:01:45.280901-05:00,586824.00,3C21985BD1CFB3A7293EB09B5E7D51D8288C4B19F1DBF21A739FF1BF16D59BD8,8964200,0,false, +2025-11-25T14:00:11.462056-05:00,2025-11-25T14:01:45.393301-05:00,93930.00,7652D984274810218CECA1D85B0014A23F3BF5F98927FF1B856A77A103B057CB,8964200,0,false, +2025-11-25T14:00:18.452151-05:00,2025-11-25T14:01:45.505234-05:00,87052.00,0ED1C70AFD59506EC238F0B40968F5DEF0EBA050686B6D550BB5CEB3E94951A7,8964200,0,false, +2025-11-25T14:00:34.491201-05:00,2025-11-25T14:01:48.777548-05:00,74286.00,7A8176A6BF29F2A06E10DDEF471D9C19ED0F4B8F13684DEA82E0E4BDBF5B425E,8964201,0,false, +2025-11-25T13:57:36.460824-05:00,2025-11-25T14:01:49.663766-05:00,253201.00,B30C8194150D7C2DE4B59B4223B8636B17EA240EF2A7FA6A7FE3300EEE1DFF42,8964201,0,false, +2025-11-25T13:49:52.464529-05:00,2025-11-25T14:01:50.437737-05:00,717970.00,BD6FCDF267706F568012D205E0441ECCD4982864190F05B5BA1ED9FFBBFBEFD2,8964201,0,false, +2025-11-25T13:58:31.455176-05:00,2025-11-25T14:01:51.218731-05:00,199762.00,8F6A6FDB77265471345C662E1059E642E03D36AE2293A3326BF7A43CC97B9283,8964201,0,false, +2025-11-25T14:00:38.45359-05:00,2025-11-25T14:01:51.775839-05:00,73321.00,3317261B692FB88343D4A5824A3E186CF56B0DB605F850845221888E70EB0BAA,8964201,0,false, +2025-11-25T13:49:07.950093-05:00,2025-11-25T14:01:59.409055-05:00,771455.00,28C2E222F9351B87C63CB258ED474050ECDDFABC2B78076AF163CE58E948CB31,8964201,0,false, +2025-11-25T14:00:48.963781-05:00,2025-11-25T14:02:00.951164-05:00,71987.00,F74A21FCB0A1FDE15BC48012D78466CE78998BBCDC0063E9E336A6D9D7830FFB,8964203,0,false, +2025-11-25T13:54:27.459166-05:00,2025-11-25T14:02:01.611275-05:00,454150.00,1E9E6E01627E64B7401AF44EC96E3BB8366688C261E7522AF29EC505396670D0,8964203,0,false, +2025-11-25T13:53:03.462292-05:00,2025-11-25T14:02:02.293605-05:00,538829.00,6570E934B920A7FF4D9354FB9C01723EDE77C2E3D0BC5B33472BFF6CC4C4A246,8964203,0,false, +2025-11-25T13:48:56.960932-05:00,2025-11-25T14:02:02.739155-05:00,785774.00,6C4FDF4B6282C9E7E3C41B38E9A1839F494C963FD368EF02AC7D3756A616D8DF,8964203,0,false, +2025-11-25T14:01:05.513551-05:00,2025-11-25T14:02:03.181089-05:00,57667.00,999C65AF9E55909F96F246D265A30896203B52E5F5FF783EB2C3C0D26A74E7D4,8964203,0,false, +2025-11-25T13:56:35.958479-05:00,2025-11-25T14:02:04.961087-05:00,329001.00,89B40601253EAD305A7722C08F66F761D41AD1886437009CE90ACC595BA87C9A,8964203,0,false, +2025-11-25T13:56:41.953471-05:00,2025-11-25T14:02:13.291682-05:00,331336.00,AF6530D5177C4E71D6683C69E9E4352A1C2F3613292AD6A3D802F12A706087BC,8964205,0,false, +2025-11-25T13:55:09.453529-05:00,2025-11-25T14:02:13.733442-05:00,424278.00,747D5112098F6AE610DB1CC72FC05F52991D858364DE2EAF09EB096448BB574E,8964205,0,false, +2025-11-25T14:01:21.464904-05:00,2025-11-25T14:02:14.079952-05:00,52614.00,B507BE2920456C10DD44B24607EA0A9D30205DA458F3426EF579F23F39252B2F,8964205,0,false, +2025-11-25T13:53:56.468039-05:00,2025-11-25T14:02:25.904944-05:00,509434.00,DC61A8AF1BE29F6992DDABDB082AE3FF00D2ACB7D57F46205D781770493741D2,8964207,0,false, +2025-11-25T14:00:48.415692-05:00,2025-11-25T14:02:31.65004-05:00,103233.00,5FD1243FB5B46EE2C3C7FFBE3AB0EE741B6815979659CEC7EC8DDBCC68ECE726,8964208,0,false, +2025-11-25T14:01:44.07979-05:00,2025-11-25T14:02:32.003752-05:00,47923.00,07C389C22CC0819055B05FA414CD2028BD76F3727E83B806DED394AC67867EF9,8964208,0,false, +2025-11-25T14:01:44.99661-05:00,2025-11-25T14:02:43.330618-05:00,58333.00,A6A2E2733886FA6EB716087F281EC45081068B5380B4727997506347CCF94343,8964210,0,false, +2025-11-25T13:54:11.957768-05:00,2025-11-25T14:02:43.734215-05:00,511774.00,40903D24E00CB5619FA0D93F0C3B8B4200ED0D4AC089BC6329E178B4D2959CE0,8964210,0,false, +2025-11-25T14:01:47.47304-05:00,2025-11-25T14:02:44.073634-05:00,56600.00,29D92278F0C9835C03C14876C67B2D5CD90C38B63E28C7DEEA1341FBF46CBDA0,8964210,0,false, +2025-11-25T13:59:02.092731-05:00,2025-11-25T14:02:49.256839-05:00,227163.00,BB4D6CBFE49F948D7275C0CB666B6A738D9B4FA652C54364384187ADA140E5DF,8964211,0,false, +2025-11-25T13:59:53.464588-05:00,2025-11-25T14:02:49.612204-05:00,176146.00,DCC6D2143A41671CB238D560CCC0B4280930F00D3B5D60EFB4DC05DB32B41496,8964211,0,false, +2025-11-25T14:00:37.497567-05:00,2025-11-25T14:03:07.217828-05:00,149719.00,289069F39FDC4E9D5069CB846D62FE5538B2701000FA1DE27987AC1FA81873FE,8964214,0,false, +2025-11-25T13:50:14.466627-05:00,2025-11-25T14:03:09.053088-05:00,774583.00,5BE148FC4C210328EDE11C405D7283FF548A95AD06B75756905B65387C1E0CE3,8964214,0,false, +2025-11-25T14:01:51.960268-05:00,2025-11-25T14:03:09.52361-05:00,77563.00,937E431E9A92A5FF0D49E901A3933F47D4C22E0F4CA4C475E6A51A85D00C4D21,8964214,0,false, +2025-11-25T13:53:48.96022-05:00,2025-11-25T14:03:13.271308-05:00,564308.00,C4B0D44E5BE8A2D55351A0D0B79BFEBDE56FD1889E76E2CB46F3462D19975B1F,8964215,0,false, +2025-11-25T13:54:34.453022-05:00,2025-11-25T14:03:25.605681-05:00,531150.00,77E2EC1DC971BB19E9A340C23F5A3DA14CE74574A5C1627A56BEF2E049F4E785,8964217,0,false, +2025-11-25T14:01:24.95403-05:00,2025-11-25T14:03:25.841988-05:00,120887.00,988C6E7C9D63B8BAE62446FAD4333210F43BCE6A33E57699CCB0B7F4AE083450,8964217,0,false, +2025-11-25T13:49:53.954448-05:00,2025-11-25T14:03:26.827951-05:00,812870.00,2E72C91B2663F7F77765E009E71CDFE4AE40EB7DB0B38430E31C0E6FCBD0F823,8964217,0,false, +2025-11-25T14:02:04.961716-05:00,2025-11-25T14:03:27.790565-05:00,82828.00,081B858BAF9EEC1D735A6E8DCD5BFE40565487C2CA92E004A9512853F5641815,8964217,0,false, +2025-11-25T13:57:49.956336-05:00,2025-11-25T14:03:29.713546-05:00,339755.00,BA5013E4CE39CD54D593E3D295B5B15D09ACFADEF0D029293C607852F1D4F08B,8964217,0,false, +2025-11-25T14:02:10.480169-05:00,2025-11-25T14:03:34.241264-05:00,83760.00,D2025DB0B0CB334A9EC5A7ACCF01343D8909B930B7EABA92B9D35124AE1429D8,8964218,0,false, +2025-11-25T13:59:06.481279-05:00,2025-11-25T14:03:34.469936-05:00,267987.00,1919BA34591EC3481C4F0C6CC65B2D1A36476A7342BB33A9B9E759DB0424DB61,8964218,0,false, +2025-11-25T13:49:55.463049-05:00,2025-11-25T14:03:58.17504-05:00,842708.00,F9F67B048487076BB854A1403C4DFEE0F0740D96301036AB145AEDAF91718F52,8964221,0,false, +2025-11-25T13:48:37.963761-05:00,2025-11-25T14:03:58.591213-05:00,920623.00,D2BECC6F4354AA4ABE7A4C3A6BC87544E22B85BAE3662DEBD2A0E57C38CC8428,8964221,0,false, +2025-11-25T14:02:26.01906-05:00,2025-11-25T14:04:00.514465-05:00,94495.00,7E37882C7698687122E9BA008BE7D1D992519F5FE97FE958AA3AE188D8C624D7,8964221,0,false, +2025-11-25T13:48:33.465503-05:00,2025-11-25T14:04:01.291063-05:00,927821.00,D885D27842731458DE4B7DFA97C825D89506F65D746F0F6EA2D254E2B63D6D3E,8964221,0,false, +2025-11-25T14:02:14.962596-05:00,2025-11-25T14:04:02.079789-05:00,107116.00,BF7902C849DEE9A5E6454189B94EEA5AE3DFDA6137C94EAF491A53372BE77BF8,8964222,0,false, +2025-11-25T13:55:49.451736-05:00,2025-11-25T14:04:02.834994-05:00,493381.00,73413038C342CD633036922B78ED958EE94BD3EC3F234C929FD57529CF972466,8964222,0,false, +2025-11-25T13:54:28.456914-05:00,2025-11-25T14:04:03.439993-05:00,574980.00,3B1C9FE04CB2AEBE66B7C6E502C48938122ABCA9D113816AB783D63F4FD36EC8,8964222,0,false, +2025-11-25T14:02:48.960507-05:00,2025-11-25T14:04:04.109216-05:00,75148.00,D5022BC7C95F597D6F3175E40D130FE23510FE0C8D46FC5817F372590D461186,8964222,0,false, +2025-11-25T14:01:20.032923-05:00,2025-11-25T14:04:04.673675-05:00,164640.00,DBC9D3687F6A22BE28B441E3279E4C75CEA55EE3910BCF8FE762D0EFD6CA776B,8964222,0,false, +2025-11-25T13:56:09.965762-05:00,2025-11-25T14:04:05.406953-05:00,475439.00,B95F16E701775ECA2E890870A5229DC50F4EFDBB221FEF59732BE3BFD80DEF42,8964223,0,false, +2025-11-25T14:03:08.458165-05:00,2025-11-25T14:04:05.91439-05:00,57455.00,FF70DB79D7BB62B9D4540714454E7F4BBD31F70F65E0490CB8DC67A186734FCC,8964223,0,false, +2025-11-25T14:03:05.990525-05:00,2025-11-25T14:04:06.3753-05:00,60384.00,CFE5E2BF385326169FADA84BF190CB8BEB56826944F5213D9D8C9D32A8E3CDE3,8964223,0,false, +2025-11-25T14:03:12.459913-05:00,2025-11-25T14:04:22.260123-05:00,69799.00,6400504871522FE2A645C9296731451D292E1CDB2DE9BA660BDAFCFF2EA8F546,8964226,0,false, +2025-11-25T13:56:24.487496-05:00,2025-11-25T14:04:40.28038-05:00,495790.00,3AD7D49F937863452BB0E2EF283F607D74D76E3F095A0A27482BD9477C4CBEFD,8964229,0,false, +2025-11-25T14:03:25.464197-05:00,2025-11-25T14:04:40.727481-05:00,75262.00,58B5F5E9F78734ABA18EC8C5A7AFBC78BFB3A0C077D38C9B426885CCB0857C5B,8964229,0,false, +2025-11-25T14:03:26.455094-05:00,2025-11-25T14:04:53.485811-05:00,87030.00,63852EDE1381A349EA477C43F11893932527A076F12FF017899FA4107EA79910,8964231,0,false, +2025-11-25T13:57:07.954037-05:00,2025-11-25T14:04:57.249109-05:00,469293.00,5451D39BF9C1363C3C74634F32095AB1D64C5F8E5BC1060C1A3CAD802D3FC51E,8964232,0,false, +2025-11-25T13:59:09.55127-05:00,2025-11-25T14:04:58.334946-05:00,348782.00,A0EBA5FE692E307314B0D24DB00375A47825C7B6C485522AFFE15186C7D2BB4A,8964232,0,false, +2025-11-25T13:55:41.950466-05:00,2025-11-25T14:04:58.669137-05:00,556716.00,64422C1B367A738BBA5483EA13E6F26598041D31A98FFD7F33C36B33B40D2C08,8964232,0,false, +2025-11-25T13:49:30.453361-05:00,2025-11-25T14:04:59.099336-05:00,928642.00,CAD0E6F5EA2FFA42404C8A104AE8D629C07BFB472396F5ACE635ABCF3C751FB6,8964232,0,false, +2025-11-25T14:01:13.179348-05:00,2025-11-25T14:05:10.304532-05:00,237124.00,AE111BED127A13933B1E89387431B68705738E5F6D4789508E774303511743CE,8964234,0,false, +2025-11-25T13:57:39.007567-05:00,2025-11-25T14:05:10.65752-05:00,451648.00,03D9E222DEB30D68988387C3C0E2BF068E48D9FCDD8CF416C1DB2BD3AF57F1EA,8964234,0,false, +2025-11-25T14:02:17.962157-05:00,2025-11-25T14:05:16.180014-05:00,178217.00,97D0AD959CFC84E3EE245B3C019B0B6BE97415397EB5AB25E4361FC18EED57AE,8964235,0,false, +2025-11-25T13:59:03.866715-05:00,2025-11-25T14:05:28.309204-05:00,384440.00,D2E1A7DB91A8FB6B5498A28E987898C7CA4B7105E0167BC62801D33DDC81D7C0,8964237,0,false, +2025-11-25T13:50:07.967332-05:00,2025-11-25T14:05:40.946527-05:00,932975.00,76754CF34C9E3346DA17B79320C48B519D7B17EEDFEFAB1CCD12D8B2EB03E654,8964239,0,false, +2025-11-25T14:01:33.959857-05:00,2025-11-25T14:05:46.215813-05:00,252254.00,208B75D97F74CE4FC0DE6B7723D15DD7C6AC930BD8250E262300E6DE1A31BE70,8964240,0,false, +2025-11-25T13:50:01.952607-05:00,2025-11-25T14:05:46.655482-05:00,944698.00,69DE0355A6F28A9CACACFDC0A2B994E6969C0CA0FEE68CC8010F8713AACD0862,8964240,0,false, +2025-11-25T14:01:45.984816-05:00,2025-11-25T14:05:52.282165-05:00,246296.00,A1ACBC2D2F358AB127A379AAF09F304574104DC3AC50ECA32AD09969C5412F9A,8964241,0,false, +2025-11-25T13:59:50.524509-05:00,2025-11-25T14:05:52.629987-05:00,362103.00,0EE36BEB23744277421BF9FFEABE08A1B9F0A84E3BF17F788DD953D84D5910AB,8964241,0,false, +2025-11-25T13:56:26.459526-05:00,2025-11-25T14:05:53.069138-05:00,566607.00,73C3A3275C8AAA44F42636D56BF81E39DFBB41E0E21E7F8F75A4359FDEECC179,8964241,0,false, +2025-11-25T13:58:31.957421-05:00,2025-11-25T14:06:05.98363-05:00,454024.00,D25E3AEADA7D2FEA35C69C15D599E9BAF86C831D94BD179B00256517BC6FC279,8964243,0,false, +2025-11-25T13:59:16.793629-05:00,2025-11-25T14:06:11.287825-05:00,414492.00,D6E86AD7F19B71793E1BD9CC2C496220FF997C70779EA64A5C39BE62B6EB0B2A,8964244,0,false, +2025-11-25T14:04:21.960549-05:00,2025-11-25T14:06:22.301492-05:00,120340.00,35B30567FB14E9F440AEB29CC02E903927299276A24855ECFC9E10B79B114680,8964246,0,false, +2025-11-25T13:49:23.952858-05:00,2025-11-25T14:06:38.53041-05:00,1034573.00,D6CF7947FE934B04953343CC77015EC589D4FAB048059B14AE35B8AD85066BFF,8964248,0,false, +2025-11-25T14:04:40.458448-05:00,2025-11-25T14:06:55.250668-05:00,134791.00,4BAAE70CCD2D27AA746DE00136F395A783EFEDC8EE34E29B599AF7C8270CD3CD,8964251,0,false, +2025-11-25T14:04:57.95765-05:00,2025-11-25T14:06:59.010618-05:00,121052.00,54575589C4597C6D3000F7EE48C307B52A8638818A14AD7D1D72202988183A87,8964252,0,false, +2025-11-25T13:59:28.961726-05:00,2025-11-25T14:07:01.250947-05:00,452287.00,4B4E378E41CFDA7366DB85915E566928230CA4A522B74A2A64C4BCF7893595B7,8964252,0,false, +2025-11-25T14:04:56.966662-05:00,2025-11-25T14:07:01.363106-05:00,124395.00,AD0043726E3D3835E7362387B8CAE348707F5F8AAD86AA17977B2CC4A8C827A9,8964252,0,false, +2025-11-25T14:02:03.460859-05:00,2025-11-25T14:07:18.004756-05:00,314542.00,625F232E477596B64FB0B094E87F37B8E1C64F3B0EC51564317F95DF27364FF1,8964255,0,false, +2025-11-25T14:05:25.502805-05:00,2025-11-25T14:07:18.139264-05:00,112635.00,A15D3322916A1B53B8D53C26660EF6C39A49A7923C09060D57C456F15E22E5BA,8964255,0,false, +2025-11-25T14:05:40.958134-05:00,2025-11-25T14:07:18.261929-05:00,97303.00,A9F3A0F3856CF25216F9CDFD41614345C61A472079B7BBC763150AA56ABA5C49,8964255,0,false, +2025-11-25T14:00:37.983425-05:00,2025-11-25T14:07:24.67958-05:00,406694.00,1958FFDA583AA578061D3844410647251CE967B4623B76C22DE8FE316CA38DDE,8964255,0,false, +2025-11-25T14:03:46.455424-05:00,2025-11-25T14:07:25.900194-05:00,219443.00,0A224C86A7CD01802512A98FDFD9FC4960002283020D26D1C33EF3F3D9AC70EA,8964255,0,false, +2025-11-25T13:59:13.957438-05:00,2025-11-25T14:07:26.839623-05:00,492880.00,16E4D4A85C0B71171107684F346C9276C0A653D7565CD3BCF9632EB2F02823BB,8964255,0,false, +2025-11-25T13:58:26.452833-05:00,2025-11-25T14:07:27.660629-05:00,541205.00,F4BDBF4FE56264845E938BA96311A95A9C4BADA1BDFB4BA5C90E56A051DF4528,8964255,0,false, +2025-11-25T14:05:44.46165-05:00,2025-11-25T14:07:28.38081-05:00,103918.00,B1FD3B3E3454FD2418F9063A5FE0A57F7877C52B665D9684A3A01B4A5713D7B8,8964256,0,false, +2025-11-25T13:56:30.4652-05:00,2025-11-25T14:07:28.990698-05:00,658522.00,484F8E7CB4F3B1D98A6A43054A2507A0B4C0BA41761917AC11D0858E5D72418E,8964256,0,false, +2025-11-25T13:58:15.476399-05:00,2025-11-25T14:07:43.281883-05:00,567803.00,3E9AA445553026F4C083D699D8F2C40F90B800C04AC9A7F5847A168D7B6E35B8,8964259,0,false, +2025-11-25T14:02:32.980609-05:00,2025-11-25T14:07:43.619373-05:00,310637.00,49D7A068A8BEA3B24B0BA65D011E6446C51B2E72A178F47902C7377BB37A50CB,8964259,0,false, +2025-11-25T14:05:51.462268-05:00,2025-11-25T14:07:43.742412-05:00,112279.00,627E2FE1D139B5271987E2466A9C50625C8BB56F3089BC28CA40D17E941290FD,8964259,0,false, +2025-11-25T14:04:42.461917-05:00,2025-11-25T14:07:43.855261-05:00,181392.00,B43030CD4207F078088E7C22E0302A9238BFBF2ECFCEB1081A62C6A504B0221A,8964259,0,false, +2025-11-25T14:05:33.464743-05:00,2025-11-25T14:07:55.572311-05:00,142106.00,9BF61EEF583404F2F6774441D1573F7CE968C39FC9D3F8E9C70F92A2B7BC7D15,8964261,0,false, +2025-11-25T13:55:52.451445-05:00,2025-11-25T14:07:55.910379-05:00,723455.00,97FD20B2DDE8739E38BD0A35DAC3461503233A16BB3F785F7836077151E002D7,8964261,0,false, +2025-11-25T13:52:44.951959-05:00,2025-11-25T14:08:01.219248-05:00,916263.00,09052CD2277DEDD588B26F85429F3305AA85B81D5D40F7FDE355161E2994BB16,8964262,0,false, +2025-11-25T14:04:44.955379-05:00,2025-11-25T14:08:01.661383-05:00,196705.00,8FC7DCB41E285F3E5F4268863DC487AB99291A84AC6DB6CE248FC7951668DB57,8964262,0,false, +2025-11-25T14:06:52.461631-05:00,2025-11-25T14:08:02.101073-05:00,69639.00,087491D06C7E65C0E50FBF40A566F0B89996926D3F7D8AE473CD4D57E5E5DA62,8964262,0,false, +2025-11-25T13:49:14.952159-05:00,2025-11-25T14:08:02.542423-05:00,1127585.00,22B8032BB19EDB8249BA830B71058788E8C9B7B93B6A38795D7EA13A0DDE5D3F,8964262,0,false, +2025-11-25T13:57:16.468181-05:00,2025-11-25T14:08:18.652749-05:00,662181.00,D7D0E753334C9FF73DD3E136944254FCFDB75C2534BC4ED92C2485663A79A4C3,8964265,0,false, +2025-11-25T14:05:49.966399-05:00,2025-11-25T14:08:20.095478-05:00,150128.00,98D1B51771CA653992E8E6FBCF953E78BF8BB5B2022E3D39D0B74959CC67B815,8964265,0,false, +2025-11-25T13:53:20.002026-05:00,2025-11-25T14:08:21.30643-05:00,901300.00,C3328B424BE3574A178E067F9A0E0CE774BC6FD9F8E74E2ED084DDCE131425D2,8964265,0,false, +2025-11-25T13:55:01.969525-05:00,2025-11-25T14:08:25.54806-05:00,803575.00,CA27A3A380E7B72173D867CB050EDC4CC8A3DEE8C0F4EC1892D3A6692A4C05DC,8964266,0,false, +2025-11-25T14:01:35.984218-05:00,2025-11-25T14:08:49.216486-05:00,433230.00,628A47F66A80565EFD74F7DA5565A697C9FC84D0CD03D678F56F1DD458CEB4E8,8964270,0,false, +2025-11-25T14:03:37.97153-05:00,2025-11-25T14:08:49.564031-05:00,311591.00,6B08368D38C9A9EBA15ECF7CB6F3A7B8DA9C7586E5F9FF78DCB52A80717BAA60,8964270,0,false, +2025-11-25T13:59:17.482247-05:00,2025-11-25T14:08:55.302037-05:00,577817.00,3B1047C367D2D7B2532C4E00BC6EDA19F34753FE4F13A91816A248B5EDE7BE2C,8964271,0,false, +2025-11-25T13:56:08.964498-05:00,2025-11-25T14:09:07.313314-05:00,778345.00,71FF6EF3E57D858747CC9427389E052DF40DF2B22C1A81317EF8B61449BB707F,8964273,0,false, +2025-11-25T14:01:34.984594-05:00,2025-11-25T14:09:13.348179-05:00,458361.00,2DF20191C20158A64AD966914EB778371C4E0F64C973B687FA4D3D0562A13AAA,8964274,0,false, +2025-11-25T13:52:09.955848-05:00,2025-11-25T14:09:13.469362-05:00,1023509.00,1A11F35F6463836A99A2D3E5D18C274F03784D4C13BED9722541129C13CF4572,8964274,0,false, +2025-11-25T14:03:35.96353-05:00,2025-11-25T14:09:25.852259-05:00,349887.00,AD84B12DFDE969B4A08B91F13598ABF4E77207BBF8B9D1D00CCFF232DFF0CC4B,8964276,0,false, +2025-11-25T13:57:13.454585-05:00,2025-11-25T14:09:26.31818-05:00,732860.00,8DE89716A2F2733C295F821E694400C73839A7ADD7C91D7B18A73BFBC6F74FCC,8964276,0,false, +2025-11-25T14:07:39.459746-05:00,2025-11-25T14:09:28.228595-05:00,108768.00,D10C145C6A3E3244A15A9F39211FCA0306934EDFC29E5661B467A823D5FACCD7,8964276,0,false, +2025-11-25T13:54:11.458638-05:00,2025-11-25T14:09:40.308672-05:00,928846.00,DE55ACB5E24B203A1BA6F5E2AFE9FF22A1DBAABF908341A6F77597575262089C,8964278,0,false, +2025-11-25T14:07:50.456982-05:00,2025-11-25T14:09:46.878256-05:00,116420.00,6B4A58B75676711A7824A978954A86804A00CB003160875C428682D9B2185CD0,8964279,0,false, +2025-11-25T13:57:59.95505-05:00,2025-11-25T14:09:52.238689-05:00,712280.00,D1ABF1BFD25586F61E5D9423FAD1C6249DBD74E12440AF9F951BE5C72E47AF6A,8964280,0,false, +2025-11-25T14:03:19.495947-05:00,2025-11-25T14:10:12.913712-05:00,413416.00,4A6DD24985528E31E9F33F475B3AF4985C1E19C7A41424542A36557B18E52E2C,8964283,0,false, +2025-11-25T14:08:01.460389-05:00,2025-11-25T14:10:16.227936-05:00,134766.00,D0CBCD62C2CE4237D3251B08F09542149BB6E07C12F15DC298D12855B468301A,8964284,0,false, +2025-11-25T13:57:33.965977-05:00,2025-11-25T14:10:28.202346-05:00,774233.00,251B2353E7B8E5AEE64BE39103B7C92214CCDBC14DEE27B6D819C8ADE2EF1F94,8964286,0,false, +2025-11-25T13:51:25.954867-05:00,2025-11-25T14:10:28.638235-05:00,1142678.00,DFF938A18DD5BF153F3C34E469681A0F3F2CEC7855D6CA0135E3EBDF764E2CDD,8964286,0,false, +2025-11-25T14:08:18.460861-05:00,2025-11-25T14:10:34.202089-05:00,135740.00,254D8283D6BD96AA6CD5990FAF47478302E81E0FD46BC188B07178D1831AE810,8964287,0,false, +2025-11-25T13:52:43.460412-05:00,2025-11-25T14:10:46.53025-05:00,1083065.00,CC23CCC7404FE7F1BC6725D25819DC83E6B37E67B08E84FF8351E2418E3284D0,8964289,0,false, +2025-11-25T14:08:20.918623-05:00,2025-11-25T14:11:08.947846-05:00,168028.00,991D09C7DBECEE66C903753F6AEA01599A5B975CC5E7DC2EF14A817E211650AB,8964292,0,false, +2025-11-25T14:03:29.4555-05:00,2025-11-25T14:11:09.947892-05:00,460490.00,6135912B427141FE4D9B2FFA5545C8A58A1DEA7406A42E9973673C58BA6FCE9A,8964292,0,false, +2025-11-25T14:04:15.459543-05:00,2025-11-25T14:11:10.842702-05:00,415381.00,01C0BDD0C83BEDFD07F6D5CB9C9892DF18D00DFCF76F3D9D4863D99A2E7829DA,8964293,0,false, +2025-11-25T14:08:48.963962-05:00,2025-11-25T14:11:11.548951-05:00,142584.00,2D57CE9449A64885FBA69C2F52C31AD41B56637D7D2CCC3592D7C621D24D4373,8964293,0,false, +2025-11-25T14:03:02.952376-05:00,2025-11-25T14:11:11.983009-05:00,489028.00,52BDC57457DFCBB8151FFCFC7C674DBCAF48D51E797464E5613A4F481CE24088,8964293,0,false, +2025-11-25T14:03:05.505108-05:00,2025-11-25T14:11:30.971543-05:00,505464.00,5B43C91CF8F9DAF524D678F62788C848844AD9CF9E349AC50927698A66FF5ED1,8964296,0,false, +2025-11-25T14:09:10.23742-05:00,2025-11-25T14:11:31.423602-05:00,141185.00,5790F57459CF8D19A595770A99376E7A7B1F3F54B6A9F162CA4E4625D69F8DF2,8964296,0,false, +2025-11-25T14:09:13.469938-05:00,2025-11-25T14:11:31.871605-05:00,138401.00,3D7C636FF917811AA42E06A346FBF898252EF6A8C202D8D816E1949F5F7B08F0,8964296,0,false, +2025-11-25T13:51:55.461105-05:00,2025-11-25T14:11:32.318821-05:00,1176852.00,B2236D297814159A61BDC2E17825B67F6E95464192F6AEA5C8CE84F37F905049,8964296,0,false, +2025-11-25T14:03:11.453093-05:00,2025-11-25T14:11:33.775824-05:00,502320.00,22D868BD52621CF8CB4CC1BEB9CDE5958F4DEB3BAFB71BA368AFCC2592E85A59,8964297,0,false, +2025-11-25T13:57:24.958429-05:00,2025-11-25T14:11:34.067723-05:00,849105.00,E8E449D9F2527851C5FC620231F8271338D9DB7B2E5C31CBB8D9CB163ECDDE35,8964297,0,false, +2025-11-25T13:55:58.463788-05:00,2025-11-25T14:11:52.232747-05:00,953765.00,5D404413E18184F3BAEAF94E3CEFB7E289734C22E8E00EE73D03754854AB4294,8964300,0,false, +2025-11-25T14:03:49.956423-05:00,2025-11-25T14:11:52.687537-05:00,482729.00,C81E877E7E303D74C46B8D09335A7812712A2C3F63266038FAF146FF0C485161,8964300,0,false, +2025-11-25T13:55:50.458973-05:00,2025-11-25T14:11:53.039416-05:00,962576.00,10DCB75C972C4E83E8938E0017C773775739F6BDE057D962DA492116A303C279,8964300,0,false, +2025-11-25T14:02:01.012583-05:00,2025-11-25T14:12:04.360811-05:00,603345.00,13988BC87753FA261A8AC3F1E15C0B7B6E142BFFCA14C3F1A2567024A6939DB0,8964302,0,false, +2025-11-25T14:10:10.487289-05:00,2025-11-25T14:12:06.097528-05:00,115609.00,C70C11A55333FD87C9F534006041FC6E28DE3B0FAFBEECA0C5A04A019779282A,8964302,0,false, +2025-11-25T13:57:57.956937-05:00,2025-11-25T14:12:13.311814-05:00,855351.00,C84554A4E99D487C205E761EEC7079B7E7750B4D17DC35369D47BE913816CD05,8964303,0,false, +2025-11-25T13:49:22.96099-05:00,2025-11-25T14:12:25.35818-05:00,1382391.00,4B2587E46B332DE5E4AA0340B85B9B9F94F83B2987A2822274FE8EFAE1C231E1,8964305,0,false, +2025-11-25T13:50:03.458154-05:00,2025-11-25T14:12:31.356808-05:00,1347893.00,36233EC8C2C6DD242CB64E185331A866030EBD8324FE1F247E58D24EC344E2CD,8964306,0,false, +2025-11-25T13:53:59.96667-05:00,2025-11-25T14:12:49.515528-05:00,1129544.00,711D5D96D9262B3E0D2F81D59B550BDB2DAB142F6F023719759B3AB6D68A2D7B,8964309,0,false, +2025-11-25T13:52:06.459955-05:00,2025-11-25T14:12:49.962449-05:00,1243497.00,2C81ADEC6435EFD7BF0829875A229133C09C54D9D8589E3A60D0A593A6D8C489,8964309,0,false, +2025-11-25T13:52:58.458658-05:00,2025-11-25T14:12:55.072246-05:00,1196608.00,F46340403083ACBB6C9B2C5DCD36865CFEFFABB7D7CD72DF14EFB8279F3AA613,8964310,0,false, +2025-11-25T14:11:09.13664-05:00,2025-11-25T14:12:55.458802-05:00,106321.00,A5B6E08F280943CD98EF90E4D9F2E97D8C076117792CD63EEC75F3E326447276,8964310,0,false, diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index 83126fb7a6..9d6b0c6836 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -75,7 +75,7 @@ type queuedTx struct { } const ( - defaultSequentialQueueSize = 100 + defaultSequentialQueueSize = 50 // Initial capacity for queue slice ) func newSequentialQueue(client *TxClient, accountName string, pollTime time.Duration) *sequentialQueue { @@ -115,16 +115,32 @@ func (q *sequentialQueue) start() { // submitJob adds a new transaction to the queue func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { - q.mu.Lock() - defer q.mu.Unlock() + // Wait for space in queue (backpressure) + for { + q.mu.Lock() + if len(q.queue) < defaultSequentialQueueSize { + // Space available - add transaction + qTx := &queuedTx{ + blobs: job.Blobs, + options: job.Options, + resultsC: job.ResultsC, + } + q.queue = append(q.queue, qTx) + q.mu.Unlock() + return + } - qTx := &queuedTx{ - blobs: job.Blobs, - options: job.Options, - resultsC: job.ResultsC, - } + // Queue full - unlock and wait + q.mu.Unlock() - q.queue = append(q.queue, qTx) + select { + case <-time.After(100 * time.Millisecond): + // Wait a bit then retry + case <-q.ctx.Done(): + // Context cancelled, exit + return + } + } } // GetQueueSize returns the number of transactions in the queue @@ -139,7 +155,7 @@ func (q *sequentialQueue) processNextTx() { startTime := time.Now() // Find first unbroadcast transaction (txHash is empty) - fmt.Println("Processing next tx") + // fmt.Println("Processing next tx") scanStart := time.Now() var qTx *queuedTx @@ -177,8 +193,31 @@ func (q *sequentialQueue) processNextTx() { // otherwise it could mean client is stalled if IsSequenceMismatchError(err) { fmt.Println("Sequence mismatch error") - // return we probably need to resign earlier transactions - // come back to this later + // check expected sequence and check if there is transaction with that sequence + expectedSeq := parseExpectedSequence(err.Error()) + // check if there is transaction with that sequence + for _, txx := range q.queue { + fmt.Println("expectedSeq: ", expectedSeq) + if txx.sequence == expectedSeq { + fmt.Printf("Found transaction with expected sequence with hash %s\n", txx.txHash[:16]) + // check status of tx + txClient := tx.NewTxClient(q.client.GetGRPCConnection()) + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: txx.txHash}) + if err != nil { + fmt.Printf("Failed to check status of tx %s: %v\n", txx.txHash[:16], err) + continue + } + if statusResp.Status == core.TxStatusRejected { + q.handleRejected(txx, statusResp, txClient) + } + fmt.Println("status for this expected hash: ", statusResp.Status) + fmt.Println("status log: ", statusResp.Error) + fmt.Println("shouldResign: ", txx.shouldResign) + return + } + + } + // No transaction found with expected sequence - return return } @@ -200,7 +239,7 @@ func (q *sequentialQueue) processNextTx() { qTx.txHash = resp.TxHash qTx.txBytes = txBytes - qTx.sequence = sequence + qTx.sequence = sequence - 1 // sequence is incremented after successful submission qTx.submittedAt = time.Now() fmt.Printf("Broadcast successful for tx %s - marking as broadcast in queue\n", qTx.txHash[:16]) @@ -209,7 +248,7 @@ func (q *sequentialQueue) processNextTx() { // monitorLoop periodically checks the status of broadcast transactions func (q *sequentialQueue) monitorLoop() { - ticker := time.NewTicker(3 * time.Second) + ticker := time.NewTicker(1 * time.Second) defer ticker.Stop() for { @@ -224,8 +263,8 @@ func (q *sequentialQueue) monitorLoop() { // coordinate coordinates transaction submission with confirmation func (q *sequentialQueue) coordinate() { - // ticker := time.NewTicker(time.Millisecond * 500) //TODO: understand if this acceptable cadence - // defer ticker.Stop() + ticker := time.NewTicker(time.Second) //TODO: it's currently fine without additional delays. Might still be necessary tho. + defer ticker.Stop() for { select { @@ -236,7 +275,7 @@ func (q *sequentialQueue) coordinate() { q.ResignRejected() case qTx := <-q.ResubmitChan: q.ResubmitEvicted(qTx) - default: + case <-ticker.C: q.processNextTx() } } @@ -245,11 +284,11 @@ func (q *sequentialQueue) coordinate() { // ResignRejected resigns a rejected transaction func (q *sequentialQueue) ResignRejected() { startTime := time.Now() - fmt.Println("Resigning rejected tx") q.mu.RLock() var txsToResign []*queuedTx for _, qTx := range q.queue { if qTx.shouldResign { + fmt.Printf("Adding rejected tx to resign list with hash %s and sequence %d\n", qTx.txHash[:16], qTx.sequence) txsToResign = append(txsToResign, qTx) } } @@ -266,6 +305,7 @@ func (q *sequentialQueue) ResignRejected() { qTx.options..., ) if err != nil { + fmt.Printf("rejected and failed to resign with hash %s", qTx.txHash[:16]) // send error and remove from queue select { case qTx.resultsC <- SequentialSubmissionResult{ @@ -280,12 +320,12 @@ func (q *sequentialQueue) ResignRejected() { sequence := q.client.Signer().Account(q.accountName).Sequence() qTx.txHash = resp.TxHash qTx.txBytes = txBytes - qTx.sequence = sequence + qTx.sequence = sequence - 1 // sequence is incremented after successful submission qTx.shouldResign = false q.resignCount++ q.mu.Unlock() resignDuration := time.Since(resignStart) - fmt.Printf("Resigned and submitted tx successfully: %s (took %v)\n", resp.TxHash, resignDuration) + fmt.Printf("Resigned and submitted tx successfully with sequence %d: %s (took %v)\n", sequence, resp.TxHash, resignDuration) } } fmt.Printf("[TIMING] Total ResignRejected took %v\n", time.Since(startTime)) @@ -520,6 +560,10 @@ func (q *sequentialQueue) handleCommitted(qTx *queuedTx, statusResp *tx.TxStatus }: } + q.mu.RLock() + fmt.Printf("LAST CONFIRMED SEQUENCE and HASH: %d, %s\n", q.lastConfirmedSeq, qTx.txHash[:16]) + q.mu.RUnlock() + // Update last confirmed sequence q.setLastConfirmedSeq(qTx.sequence) q.removeFromQueue(qTx) @@ -533,49 +577,29 @@ func (q *sequentialQueue) setLastConfirmedSeq(seq uint64) { // handleRejected processes a rejected transaction func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { - fmt.Println("Handling rejected tx") - // Step 1: Roll back sequence if previous tx was confirmed - if q.isPreviousTxConfirmed(qTx.sequence) { - q.mu.Lock() - q.client.Signer().SetSequence(q.accountName, qTx.sequence) - q.mu.Unlock() - } - - isNonceMismatch := isSequenceMismatchRejection(statusResp.Error) - if isNonceMismatch { - q.mu.Lock() - qTx.shouldResign = true - q.mu.Unlock() - } + fmt.Printf("Handling rejected tx:%s with code %d", qTx.txHash[:16], statusResp.ExecutionCode) - // Step 2: Collect subsequent transactions to check q.mu.RLock() - var subsequentTxs []*queuedTx - for _, subTx := range q.queue { - if subTx.sequence > qTx.sequence && subTx.txHash != "" { - subsequentTxs = append(subsequentTxs, subTx) - } - } + shouldResign := qTx.shouldResign q.mu.RUnlock() - - // Step 3: Batch query subsequent transactions to see if they were also rejected // TODO: in future this should be handled by batch txstatus request - for _, subTx := range subsequentTxs { - if subTx.sequence > qTx.sequence && subTx.txHash != "" { - // TODO: this should also be rejected for sequence mismatch - resp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: subTx.txHash}) - if err == nil && resp.Status == core.TxStatusRejected && resp.ExecutionCode == 32 { - fmt.Println("Sequence mismatch error: ReCheck()") - q.mu.Lock() - subTx.shouldResign = true - q.mu.Unlock() - } - } + if shouldResign { + fmt.Printf("Tx %s is already being resigned - skipping\n", qTx.txHash[:16]) + return } - // Q: should we wait till all txs are marked for resign before sending to resign channel? - q.ResignChan <- qTx + + isNonceMismatch := isSequenceMismatchRejection(statusResp.Error) if !isNonceMismatch { - // Non-nonce error remove from queue and return error back to user + // Non-nonce error - remove from queue and return error to user + // check if previous tx was confirmed or pending. If so, roll back sequence to the previous tx sequence + if q.isPreviousTxCommittedOrPending(qTx.sequence, txClient) { + q.mu.Lock() + fmt.Println("LAST CONFIRMED SEQUENCE: ", q.lastConfirmedSeq) + fmt.Println("SEQUENCE TO ROLL BACK TO: ", qTx.sequence) + q.client.Signer().SetSequence(q.accountName, qTx.sequence) + q.mu.Unlock() + fmt.Printf("Rolled back signer sequence to %d (previous tx)\n", qTx.sequence) + } select { case <-q.ctx.Done(): case qTx.resultsC <- SequentialSubmissionResult{ @@ -586,6 +610,75 @@ func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusR return } + // Nonce/sequence mismatch - scan entire queue from beginning to find all rejected txs + fmt.Printf("Detected rejected tx with sequence %d - scanning queue for all rejections\n", qTx.sequence) + + // Check if already being resigned + q.mu.RLock() + alreadyResigning := qTx.shouldResign + q.mu.RUnlock() + if alreadyResigning { + fmt.Printf("Tx %s is already being resigned - skipping\n", qTx.txHash[:16]) + return + } + + // Step 2: Collect all broadcast transactions to check (including those already marked for resignation) + q.mu.RLock() + var allBroadcastTxs []*queuedTx + for _, tx := range q.queue { + if tx.txHash != "" { + allBroadcastTxs = append(allBroadcastTxs, tx) + } + } + q.mu.RUnlock() + + // Step 3: Check status of each transaction to find all rejected ones + var rejectedTxs []*queuedTx + for _, qTx := range allBroadcastTxs { + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) + if err != nil { + continue + } + if statusResp.Status == core.TxStatusRejected && statusResp.ExecutionCode == 32 { + rejectedTxs = append(rejectedTxs, qTx) + } + } + + // Step 3a: Find the earliest rejected tx and roll back sequence if needed + if len(rejectedTxs) > 0 { + // Find the earliest rejected tx (lowest sequence) + var earliestRejected *queuedTx + for _, rejectedTx := range rejectedTxs { + if earliestRejected == nil || rejectedTx.sequence < earliestRejected.sequence { + earliestRejected = rejectedTx + } + } + + // Check if the transaction before the earliest rejected one was confirmed or pending + fmt.Println("EARLIEST REJECTED TX SEQUENCE: ", earliestRejected.sequence) + if q.isPreviousTxCommittedOrPending(earliestRejected.sequence, txClient) { + fmt.Println("FOR SEQUENCE MISMATCH REJECTIONS") + fmt.Println("LAST CONFIRMED SEQUENCE: ", q.lastConfirmedSeq) + fmt.Println("SEQUENCE TO ROLL BACK TO: ", earliestRejected.sequence) + q.mu.Lock() + q.client.Signer().SetSequence(q.accountName, q.lastConfirmedSeq+1) + q.mu.Unlock() + fmt.Printf("Rolled back signer sequence to %d (earliest rejected tx)\n", earliestRejected.sequence) + } + } + + for _, rejectedTx := range rejectedTxs { + q.mu.Lock() + if !rejectedTx.shouldResign { + rejectedTx.shouldResign = true + q.mu.Unlock() + fmt.Printf("Sending rejected tx (seq %d) to resign channel\n", rejectedTx.sequence) + q.ResignChan <- rejectedTx + } else { + q.mu.Unlock() + fmt.Printf("Skipping rejected tx (seq %d) - already marked for resign\n", rejectedTx.sequence) + } + } } // removeFromQueue removes a transaction from the queue @@ -611,6 +704,48 @@ func (q *sequentialQueue) isPreviousTxConfirmed(seq uint64) bool { return q.lastConfirmedSeq >= seq-1 } +// isPreviousTxCommittedOrPending checks if the previous transaction is COMMITTED or PENDING +func (q *sequentialQueue) isPreviousTxCommittedOrPending(seq uint64, txClient tx.TxClient) bool { + if seq == 0 { + return true + } + prevSeq := seq - 1 + + // First check if it's confirmed via lastConfirmedSeq + q.mu.RLock() + if q.lastConfirmedSeq >= prevSeq { + q.mu.RUnlock() + return true + } + + // Find the previous transaction in the queue + var prevTx *queuedTx + for _, txx := range q.queue { + if txx.sequence == prevSeq && txx.txHash != "" { + prevTx = txx + break + } + } + q.mu.RUnlock() + + if prevTx == nil { + // Previous transaction not in queue - assume it's confirmed + return true + } + + // Check the actual status of the previous transaction + statusResp, err := txClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: prevTx.txHash}) + if err != nil { + // If we can't check status, assume it's confirmed + return true + } + fmt.Println("PREVIOUS TX STATUS Seq: ", prevSeq, " RESPONSE: ", statusResp.Status, "LOG: ", statusResp.Error) + fmt.Println("PREVIOUS TX SHOULD RESIGN: ", prevTx.shouldResign) + + // Return true if COMMITTED or PENDING + return statusResp.Status == core.TxStatusCommitted || statusResp.Status == core.TxStatusPending +} + // isSequenceMismatchRejection checks if an error message indicates sequence mismatch func isSequenceMismatchRejection(errMsg string) bool { return strings.Contains(errMsg, "account sequence mismatch") || From 57afcbb93104a204a02f3059428cfcb374ec2f2a Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Wed, 26 Nov 2025 14:28:09 -0500 Subject: [PATCH 07/12] feat: add service for batch tx status queries --- app/grpc/tx/server.go | 61 +++ app/grpc/tx/tx.pb.go | 698 +++++++++++++++++++++++++++-- app/grpc/tx/tx.pb.gw.go | 81 ++++ go.mod | 59 ++- go.sum | 124 +++-- pkg/user/tx_client.go | 12 + proto/celestia/core/v1/tx/tx.proto | 25 ++ 7 files changed, 935 insertions(+), 125 deletions(-) diff --git a/app/grpc/tx/server.go b/app/grpc/tx/server.go index c6e8229a0b..6a032bebaf 100644 --- a/app/grpc/tx/server.go +++ b/app/grpc/tx/server.go @@ -90,3 +90,64 @@ func (s *txServer) TxStatus(ctx context.Context, req *TxStatusRequest) (*TxStatu Signers: resTx.Signers, }, nil } + +func (s *txServer) TxStatusBatch(ctx context.Context, req *TxStatusBatchRequest) (*TxStatusBatchResponse, error) { + if req == nil { + return nil, status.Error(codes.InvalidArgument, "request cannot be nil") + } + + if len(req.TxIds) > 20 { + return nil, status.Error(codes.InvalidArgument, "maximum of 20 tx ids allowed") + } + + if len(req.TxIds) == 0 { + return nil, status.Error(codes.InvalidArgument, "tx ids cannot be empty") + } + + node, err := s.clientCtx.GetNode() + if err != nil { + return nil, err + } + + nodeTxStatus, ok := node.(rpcclient.SignClient) + if !ok { + return nil, status.Error(codes.Unimplemented, "node does not support tx status batch") + } + + txIDs := make([][]byte, len(req.TxIds)) + for i, txId := range req.TxIds { + txID, err := hex.DecodeString(txId) + if err != nil { + return nil, status.Errorf(codes.InvalidArgument, "invalid tx id: %s", err) + } + txIDs[i] = txID + } + + txStatusBatchResponses, err := nodeTxStatus.TxStatusBatch(ctx, txIDs) + if err != nil { + return nil, err + } + + responses := make([]*TxStatusResponseWithHash, len(txStatusBatchResponses.Statuses)) + for i, status := range txStatusBatchResponses.Statuses { + responses[i] = &TxStatusResponseWithHash{ + TxHash: req.TxIds[i], + Status: &TxStatusResponse{ + + Height: status.Result.Height, + Index: status.Result.Index, + ExecutionCode: status.Result.ExecutionCode, + Error: status.Result.Error, + Status: status.Result.Status, + GasWanted: status.Result.GasWanted, + GasUsed: status.Result.GasUsed, + Codespace: status.Result.Codespace, + Signers: status.Result.Signers, + }, + } + } + + return &TxStatusBatchResponse{ + Responses: responses, + }, nil +} diff --git a/app/grpc/tx/tx.pb.go b/app/grpc/tx/tx.pb.go index 542cef60b3..2c8eca929f 100644 --- a/app/grpc/tx/tx.pb.go +++ b/app/grpc/tx/tx.pb.go @@ -86,12 +86,13 @@ type TxStatusResponse struct { Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"` // status is the status of the transaction. Status string `protobuf:"bytes,5,opt,name=status,proto3" json:"status,omitempty"` - // the error namespace/module + // the error namespace/module. Codespace string `protobuf:"bytes,6,opt,name=codespace,proto3" json:"codespace,omitempty"` - // requested gas limit + // requested gas limit. GasWanted int64 `protobuf:"varint,7,opt,name=gas_wanted,json=gasWanted,proto3" json:"gas_wanted,omitempty"` - // actual gas consumed - GasUsed int64 `protobuf:"varint,8,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` + // actual gas consumed. + GasUsed int64 `protobuf:"varint,8,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` + // signers of the transaction. Signers []string `protobuf:"bytes,9,rep,name=signers,proto3" json:"signers,omitempty"` } @@ -191,41 +192,196 @@ func (m *TxStatusResponse) GetSigners() []string { return nil } +// TxStatusRequestBatch +type TxStatusBatchRequest struct { + // array of hex encoded tx hashes (each hash should be 64 characters long representing 32 bytes) + TxIds []string `protobuf:"bytes,1,rep,name=tx_ids,json=txIds,proto3" json:"tx_ids,omitempty"` +} + +func (m *TxStatusBatchRequest) Reset() { *m = TxStatusBatchRequest{} } +func (m *TxStatusBatchRequest) String() string { return proto.CompactTextString(m) } +func (*TxStatusBatchRequest) ProtoMessage() {} +func (*TxStatusBatchRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7d8b070565b0dcb6, []int{2} +} +func (m *TxStatusBatchRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxStatusBatchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TxStatusBatchRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *TxStatusBatchRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxStatusBatchRequest.Merge(m, src) +} +func (m *TxStatusBatchRequest) XXX_Size() int { + return m.Size() +} +func (m *TxStatusBatchRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TxStatusBatchRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_TxStatusBatchRequest proto.InternalMessageInfo + +func (m *TxStatusBatchRequest) GetTxIds() []string { + if m != nil { + return m.TxIds + } + return nil +} + +// TxStatusResponseWithHash is the tx hash with respective tx info +type TxStatusResponseWithHash struct { + TxHash string `protobuf:"bytes,1,opt,name=tx_hash,json=txHash,proto3" json:"tx_hash,omitempty"` + Status *TxStatusResponse `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` +} + +func (m *TxStatusResponseWithHash) Reset() { *m = TxStatusResponseWithHash{} } +func (m *TxStatusResponseWithHash) String() string { return proto.CompactTextString(m) } +func (*TxStatusResponseWithHash) ProtoMessage() {} +func (*TxStatusResponseWithHash) Descriptor() ([]byte, []int) { + return fileDescriptor_7d8b070565b0dcb6, []int{3} +} +func (m *TxStatusResponseWithHash) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxStatusResponseWithHash) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TxStatusResponseWithHash.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *TxStatusResponseWithHash) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxStatusResponseWithHash.Merge(m, src) +} +func (m *TxStatusResponseWithHash) XXX_Size() int { + return m.Size() +} +func (m *TxStatusResponseWithHash) XXX_DiscardUnknown() { + xxx_messageInfo_TxStatusResponseWithHash.DiscardUnknown(m) +} + +var xxx_messageInfo_TxStatusResponseWithHash proto.InternalMessageInfo + +func (m *TxStatusResponseWithHash) GetTxHash() string { + if m != nil { + return m.TxHash + } + return "" +} + +func (m *TxStatusResponseWithHash) GetStatus() *TxStatusResponse { + if m != nil { + return m.Status + } + return nil +} + +// TxStatusBatchResponse +type TxStatusBatchResponse struct { + Responses []*TxStatusResponseWithHash `protobuf:"bytes,1,rep,name=responses,proto3" json:"responses,omitempty"` +} + +func (m *TxStatusBatchResponse) Reset() { *m = TxStatusBatchResponse{} } +func (m *TxStatusBatchResponse) String() string { return proto.CompactTextString(m) } +func (*TxStatusBatchResponse) ProtoMessage() {} +func (*TxStatusBatchResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_7d8b070565b0dcb6, []int{4} +} +func (m *TxStatusBatchResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxStatusBatchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TxStatusBatchResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *TxStatusBatchResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxStatusBatchResponse.Merge(m, src) +} +func (m *TxStatusBatchResponse) XXX_Size() int { + return m.Size() +} +func (m *TxStatusBatchResponse) XXX_DiscardUnknown() { + xxx_messageInfo_TxStatusBatchResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_TxStatusBatchResponse proto.InternalMessageInfo + +func (m *TxStatusBatchResponse) GetResponses() []*TxStatusResponseWithHash { + if m != nil { + return m.Responses + } + return nil +} + func init() { proto.RegisterType((*TxStatusRequest)(nil), "celestia.core.v1.tx.TxStatusRequest") proto.RegisterType((*TxStatusResponse)(nil), "celestia.core.v1.tx.TxStatusResponse") + proto.RegisterType((*TxStatusBatchRequest)(nil), "celestia.core.v1.tx.TxStatusBatchRequest") + proto.RegisterType((*TxStatusResponseWithHash)(nil), "celestia.core.v1.tx.TxStatusResponseWithHash") + proto.RegisterType((*TxStatusBatchResponse)(nil), "celestia.core.v1.tx.TxStatusBatchResponse") } func init() { proto.RegisterFile("celestia/core/v1/tx/tx.proto", fileDescriptor_7d8b070565b0dcb6) } var fileDescriptor_7d8b070565b0dcb6 = []byte{ - // 404 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x52, 0x4d, 0x8b, 0x14, 0x31, - 0x10, 0x9d, 0xee, 0xd9, 0xf9, 0xe8, 0xc0, 0xaa, 0x64, 0x45, 0xe2, 0xd2, 0x36, 0xc3, 0xb0, 0x2b, - 0x73, 0xb1, 0xc3, 0xea, 0x3f, 0xd0, 0xd3, 0x5e, 0xdb, 0x15, 0xc1, 0xcb, 0x90, 0xed, 0x14, 0x99, - 0xc0, 0xda, 0x69, 0x93, 0xea, 0x35, 0x20, 0x7b, 0xd1, 0x83, 0x57, 0xc1, 0x3f, 0xe5, 0x71, 0xc0, - 0x8b, 0x47, 0x99, 0xf1, 0x87, 0x48, 0xa7, 0xe7, 0x03, 0x64, 0xc0, 0x43, 0x20, 0xef, 0xbd, 0xaa, - 0x7a, 0xe1, 0x55, 0x48, 0x5a, 0xc2, 0x0d, 0x38, 0xd4, 0x82, 0x97, 0xc6, 0x02, 0xbf, 0xbd, 0xe0, - 0xe8, 0x39, 0xfa, 0xbc, 0xb6, 0x06, 0x0d, 0x3d, 0xd9, 0xaa, 0x79, 0xab, 0xe6, 0xb7, 0x17, 0x39, - 0xfa, 0xd3, 0x54, 0x19, 0xa3, 0x6e, 0x80, 0x8b, 0x5a, 0x73, 0x51, 0x55, 0x06, 0x05, 0x6a, 0x53, - 0xb9, 0xae, 0x65, 0xfa, 0x94, 0xdc, 0xbf, 0xf2, 0xaf, 0x51, 0x60, 0xe3, 0x0a, 0xf8, 0xd0, 0x80, - 0x43, 0x7a, 0x42, 0x06, 0xe8, 0xe7, 0x5a, 0xb2, 0x68, 0x12, 0xcd, 0x92, 0xe2, 0x08, 0xfd, 0xa5, - 0x9c, 0x7e, 0x8d, 0xc9, 0x83, 0x7d, 0xa1, 0xab, 0x4d, 0xe5, 0x80, 0x3e, 0x22, 0xc3, 0x05, 0x68, - 0xb5, 0xc0, 0x50, 0xda, 0x2f, 0x36, 0x88, 0x3e, 0x24, 0x03, 0x5d, 0x49, 0xf0, 0x2c, 0x9e, 0x44, - 0xb3, 0xe3, 0xa2, 0x03, 0xf4, 0x9c, 0xdc, 0x03, 0x0f, 0x65, 0xd3, 0xda, 0xcf, 0x4b, 0x23, 0x81, - 0xf5, 0x83, 0x7c, 0xbc, 0x63, 0x5f, 0x19, 0x09, 0x6d, 0x33, 0x58, 0x6b, 0x2c, 0x3b, 0x0a, 0xf6, - 0x1d, 0x68, 0xad, 0x5c, 0x30, 0x67, 0x83, 0x40, 0x6f, 0x10, 0x4d, 0x49, 0xd2, 0x8e, 0x72, 0xb5, - 0x28, 0x81, 0x0d, 0x83, 0xb4, 0x27, 0xe8, 0x13, 0x42, 0x94, 0x70, 0xf3, 0x8f, 0xa2, 0x42, 0x90, - 0x6c, 0x14, 0x1e, 0x99, 0x28, 0xe1, 0xde, 0x06, 0x82, 0x3e, 0x26, 0xe3, 0x56, 0x6e, 0x1c, 0x48, - 0x36, 0x0e, 0xe2, 0x48, 0x09, 0xf7, 0xc6, 0x81, 0xa4, 0x8c, 0x8c, 0x9c, 0x56, 0x15, 0x58, 0xc7, - 0x92, 0x49, 0x7f, 0x96, 0x14, 0x5b, 0xf8, 0xfc, 0x4b, 0x44, 0xe2, 0x2b, 0x4f, 0xef, 0xc8, 0x78, - 0x9b, 0x07, 0x3d, 0xcb, 0x0f, 0x04, 0x9f, 0xff, 0x93, 0xeb, 0xe9, 0xf9, 0x7f, 0xaa, 0xba, 0x50, - 0xa7, 0x67, 0x9f, 0x7f, 0xfe, 0xf9, 0x1e, 0x67, 0x34, 0xe5, 0x87, 0x76, 0xfd, 0x29, 0xac, 0xe6, - 0xee, 0xe5, 0xe5, 0x8f, 0x55, 0x16, 0x2d, 0x57, 0x59, 0xf4, 0x7b, 0x95, 0x45, 0xdf, 0xd6, 0x59, - 0x6f, 0xb9, 0xce, 0x7a, 0xbf, 0xd6, 0x59, 0xef, 0x1d, 0x57, 0x1a, 0x17, 0xcd, 0x75, 0x5e, 0x9a, - 0xf7, 0xbb, 0x09, 0xc6, 0xaa, 0xdd, 0xfd, 0x99, 0xa8, 0x6b, 0xde, 0x1e, 0x65, 0xeb, 0x92, 0xa3, - 0xbf, 0x1e, 0x86, 0x9f, 0xf0, 0xe2, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x71, 0xcb, 0xde, 0x23, - 0x5c, 0x02, 0x00, 0x00, + // 529 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xcf, 0x8b, 0xd3, 0x40, + 0x14, 0x6e, 0xda, 0xed, 0x8f, 0xbc, 0xa5, 0x2a, 0xb3, 0xbb, 0x3a, 0x96, 0x1a, 0x4b, 0xd8, 0x4a, + 0x5d, 0x68, 0xc2, 0xd6, 0x9b, 0xe0, 0x65, 0xbd, 0xb8, 0x78, 0x8b, 0x2b, 0x0b, 0x5e, 0xca, 0x34, + 0x19, 0x92, 0xc0, 0x9a, 0x89, 0x99, 0xd7, 0x75, 0x40, 0xf6, 0xe2, 0xc5, 0x93, 0x20, 0xec, 0x3f, + 0xe5, 0x71, 0xc1, 0x8b, 0x47, 0x69, 0xfd, 0x43, 0x24, 0xd3, 0xa4, 0xd5, 0x52, 0xdc, 0x1e, 0x0a, + 0xf3, 0xde, 0xf7, 0xde, 0xf7, 0xbe, 0xf7, 0xbd, 0x06, 0xba, 0x3e, 0xbf, 0xe0, 0x12, 0x63, 0xe6, + 0xfa, 0x22, 0xe3, 0xee, 0xe5, 0xb1, 0x8b, 0xca, 0x45, 0xe5, 0xa4, 0x99, 0x40, 0x41, 0xf6, 0x4a, + 0xd4, 0xc9, 0x51, 0xe7, 0xf2, 0xd8, 0x41, 0xd5, 0xe9, 0x86, 0x42, 0x84, 0x17, 0xdc, 0x65, 0x69, + 0xec, 0xb2, 0x24, 0x11, 0xc8, 0x30, 0x16, 0x89, 0x5c, 0xb4, 0xd8, 0x4f, 0xe0, 0xee, 0x99, 0x7a, + 0x83, 0x0c, 0xa7, 0xd2, 0xe3, 0x1f, 0xa6, 0x5c, 0x22, 0xd9, 0x83, 0x3a, 0xaa, 0x71, 0x1c, 0x50, + 0xa3, 0x67, 0x0c, 0x4c, 0x6f, 0x07, 0xd5, 0x69, 0x60, 0x7f, 0xa9, 0xc2, 0xbd, 0x55, 0xa1, 0x4c, + 0x45, 0x22, 0x39, 0xb9, 0x0f, 0x8d, 0x88, 0xc7, 0x61, 0x84, 0xba, 0xb4, 0xe6, 0x15, 0x11, 0xd9, + 0x87, 0x7a, 0x9c, 0x04, 0x5c, 0xd1, 0x6a, 0xcf, 0x18, 0xb4, 0xbd, 0x45, 0x40, 0xfa, 0x70, 0x87, + 0x2b, 0xee, 0x4f, 0xf3, 0xf1, 0x63, 0x5f, 0x04, 0x9c, 0xd6, 0x34, 0xdc, 0x5e, 0x66, 0x5f, 0x8a, + 0x80, 0xe7, 0xcd, 0x3c, 0xcb, 0x44, 0x46, 0x77, 0xf4, 0xf8, 0x45, 0x90, 0x8f, 0x92, 0x7a, 0x38, + 0xad, 0xeb, 0x74, 0x11, 0x91, 0x2e, 0x98, 0x39, 0x95, 0x4c, 0x99, 0xcf, 0x69, 0x43, 0x43, 0xab, + 0x04, 0x79, 0x04, 0x10, 0x32, 0x39, 0xfe, 0xc8, 0x12, 0xe4, 0x01, 0x6d, 0x6a, 0x91, 0x66, 0xc8, + 0xe4, 0xb9, 0x4e, 0x90, 0x87, 0xd0, 0xca, 0xe1, 0xa9, 0xe4, 0x01, 0x6d, 0x69, 0xb0, 0x19, 0x32, + 0xf9, 0x56, 0xf2, 0x80, 0x50, 0x68, 0xca, 0x38, 0x4c, 0x78, 0x26, 0xa9, 0xd9, 0xab, 0x0d, 0x4c, + 0xaf, 0x0c, 0xed, 0x21, 0xec, 0x97, 0x46, 0x9c, 0x30, 0xf4, 0xa3, 0xd2, 0xb6, 0x03, 0x68, 0x68, + 0xdb, 0x24, 0x35, 0x74, 0x43, 0x3d, 0xf7, 0x4d, 0xda, 0x19, 0xd0, 0x75, 0xdf, 0xce, 0x63, 0x8c, + 0x5e, 0x31, 0x19, 0x91, 0x07, 0xd0, 0x44, 0x35, 0x8e, 0x98, 0x8c, 0x0a, 0xaf, 0x1b, 0xa8, 0x34, + 0xf0, 0x62, 0xb9, 0x6d, 0xee, 0xe0, 0xee, 0xa8, 0xef, 0x6c, 0xb8, 0xac, 0xb3, 0xce, 0x5b, 0x9a, + 0x62, 0x07, 0x70, 0xb0, 0x26, 0xb1, 0x38, 0xd8, 0x6b, 0x30, 0xb3, 0xe2, 0xbd, 0x90, 0xb9, 0x3b, + 0x1a, 0x6e, 0x45, 0x5d, 0x4a, 0xf6, 0x56, 0xfd, 0xa3, 0xeb, 0x2a, 0x54, 0xcf, 0x14, 0xb9, 0x82, + 0x56, 0x59, 0x4d, 0x0e, 0x6f, 0x21, 0xd3, 0x4e, 0x75, 0xb6, 0xdb, 0xc6, 0x3e, 0xfc, 0xfc, 0xe3, + 0xf7, 0x75, 0xd5, 0x22, 0x5d, 0x77, 0xd3, 0x9f, 0xfe, 0x93, 0x36, 0xfb, 0x8a, 0x7c, 0x35, 0xa0, + 0xfd, 0xcf, 0xb2, 0xe4, 0xe9, 0x7f, 0xe9, 0xff, 0xbe, 0x59, 0xe7, 0x68, 0x9b, 0xd2, 0x42, 0x4e, + 0x5f, 0xcb, 0x79, 0xfc, 0xdc, 0x38, 0xb2, 0x3b, 0x1b, 0x15, 0x4d, 0xf2, 0xf2, 0x93, 0xd3, 0xef, + 0x33, 0xcb, 0xb8, 0x99, 0x59, 0xc6, 0xaf, 0x99, 0x65, 0x7c, 0x9b, 0x5b, 0x95, 0x9b, 0xb9, 0x55, + 0xf9, 0x39, 0xb7, 0x2a, 0xef, 0xdc, 0x30, 0xc6, 0x68, 0x3a, 0x71, 0x7c, 0xf1, 0x7e, 0xd9, 0x2f, + 0xb2, 0x70, 0xf9, 0x1e, 0xb2, 0x34, 0x75, 0xf3, 0x5f, 0x98, 0xa5, 0xbe, 0x8b, 0x6a, 0xd2, 0xd0, + 0x9f, 0xe8, 0xb3, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x8e, 0xc4, 0xc3, 0xef, 0xf5, 0x03, 0x00, + 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -247,6 +403,8 @@ type TxClient interface { // - Evicted // - Unknown TxStatus(ctx context.Context, in *TxStatusRequest, opts ...grpc.CallOption) (*TxStatusResponse, error) + // TxStatusBatch for batch queries + TxStatusBatch(ctx context.Context, in *TxStatusBatchRequest, opts ...grpc.CallOption) (*TxStatusBatchResponse, error) } type txClient struct { @@ -266,6 +424,15 @@ func (c *txClient) TxStatus(ctx context.Context, in *TxStatusRequest, opts ...gr return out, nil } +func (c *txClient) TxStatusBatch(ctx context.Context, in *TxStatusBatchRequest, opts ...grpc.CallOption) (*TxStatusBatchResponse, error) { + out := new(TxStatusBatchResponse) + err := c.cc.Invoke(ctx, "/celestia.core.v1.tx.Tx/TxStatusBatch", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // TxServer is the server API for Tx service. type TxServer interface { // TxStatus returns the status of a transaction. There are four possible @@ -275,6 +442,8 @@ type TxServer interface { // - Evicted // - Unknown TxStatus(context.Context, *TxStatusRequest) (*TxStatusResponse, error) + // TxStatusBatch for batch queries + TxStatusBatch(context.Context, *TxStatusBatchRequest) (*TxStatusBatchResponse, error) } // UnimplementedTxServer can be embedded to have forward compatible implementations. @@ -284,6 +453,9 @@ type UnimplementedTxServer struct { func (*UnimplementedTxServer) TxStatus(ctx context.Context, req *TxStatusRequest) (*TxStatusResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TxStatus not implemented") } +func (*UnimplementedTxServer) TxStatusBatch(ctx context.Context, req *TxStatusBatchRequest) (*TxStatusBatchResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method TxStatusBatch not implemented") +} func RegisterTxServer(s grpc1.Server, srv TxServer) { s.RegisterService(&_Tx_serviceDesc, srv) @@ -307,6 +479,24 @@ func _Tx_TxStatus_Handler(srv interface{}, ctx context.Context, dec func(interfa return interceptor(ctx, in, info, handler) } +func _Tx_TxStatusBatch_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(TxStatusBatchRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TxServer).TxStatusBatch(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/celestia.core.v1.tx.Tx/TxStatusBatch", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TxServer).TxStatusBatch(ctx, req.(*TxStatusBatchRequest)) + } + return interceptor(ctx, in, info, handler) +} + var Tx_serviceDesc = _Tx_serviceDesc var _Tx_serviceDesc = grpc.ServiceDesc{ ServiceName: "celestia.core.v1.tx.Tx", @@ -316,6 +506,10 @@ var _Tx_serviceDesc = grpc.ServiceDesc{ MethodName: "TxStatus", Handler: _Tx_TxStatus_Handler, }, + { + MethodName: "TxStatusBatch", + Handler: _Tx_TxStatusBatch_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "celestia/core/v1/tx/tx.proto", @@ -429,6 +623,117 @@ func (m *TxStatusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *TxStatusBatchRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxStatusBatchRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxStatusBatchRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.TxIds) > 0 { + for iNdEx := len(m.TxIds) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.TxIds[iNdEx]) + copy(dAtA[i:], m.TxIds[iNdEx]) + i = encodeVarintTx(dAtA, i, uint64(len(m.TxIds[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *TxStatusResponseWithHash) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxStatusResponseWithHash) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxStatusResponseWithHash) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Status != nil { + { + size, err := m.Status.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTx(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + if len(m.TxHash) > 0 { + i -= len(m.TxHash) + copy(dAtA[i:], m.TxHash) + i = encodeVarintTx(dAtA, i, uint64(len(m.TxHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *TxStatusBatchResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TxStatusBatchResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TxStatusBatchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Responses) > 0 { + for iNdEx := len(m.Responses) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Responses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTx(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + func encodeVarintTx(dAtA []byte, offset int, v uint64) int { offset -= sovTx(v) base := offset @@ -495,6 +800,53 @@ func (m *TxStatusResponse) Size() (n int) { return n } +func (m *TxStatusBatchRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.TxIds) > 0 { + for _, s := range m.TxIds { + l = len(s) + n += 1 + l + sovTx(uint64(l)) + } + } + return n +} + +func (m *TxStatusResponseWithHash) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.TxHash) + if l > 0 { + n += 1 + l + sovTx(uint64(l)) + } + if m.Status != nil { + l = m.Status.Size() + n += 1 + l + sovTx(uint64(l)) + } + return n +} + +func (m *TxStatusBatchResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Responses) > 0 { + for _, e := range m.Responses { + l = e.Size() + n += 1 + l + sovTx(uint64(l)) + } + } + return n +} + func sovTx(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -856,6 +1208,290 @@ func (m *TxStatusResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *TxStatusBatchRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TxStatusBatchRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TxStatusBatchRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxIds", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTx + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTx + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TxIds = append(m.TxIds, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTx(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTx + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TxStatusResponseWithHash) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TxStatusResponseWithHash: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TxStatusResponseWithHash: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TxHash", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTx + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTx + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TxHash = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTx + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTx + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Status == nil { + m.Status = &TxStatusResponse{} + } + if err := m.Status.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTx(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTx + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TxStatusBatchResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TxStatusBatchResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TxStatusBatchResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTx + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTx + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTx + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Responses = append(m.Responses, &TxStatusResponseWithHash{}) + if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTx(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTx + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipTx(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/app/grpc/tx/tx.pb.gw.go b/app/grpc/tx/tx.pb.gw.go index 8cec56bc0a..614c9efc10 100644 --- a/app/grpc/tx/tx.pb.gw.go +++ b/app/grpc/tx/tx.pb.gw.go @@ -87,6 +87,40 @@ func local_request_Tx_TxStatus_0(ctx context.Context, marshaler runtime.Marshale } +func request_Tx_TxStatusBatch_0(ctx context.Context, marshaler runtime.Marshaler, client TxClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TxStatusBatchRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := client.TxStatusBatch(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD)) + return msg, metadata, err + +} + +func local_request_Tx_TxStatusBatch_0(ctx context.Context, marshaler runtime.Marshaler, server TxServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) { + var protoReq TxStatusBatchRequest + var metadata runtime.ServerMetadata + + newReader, berr := utilities.IOReaderFactory(req.Body) + if berr != nil { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", berr) + } + if err := marshaler.NewDecoder(newReader()).Decode(&protoReq); err != nil && err != io.EOF { + return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err) + } + + msg, err := server.TxStatusBatch(ctx, &protoReq) + return msg, metadata, err + +} + // RegisterTxHandlerServer registers the http handlers for service Tx to "mux". // UnaryRPC :call TxServer directly. // StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906. @@ -116,6 +150,29 @@ func RegisterTxHandlerServer(ctx context.Context, mux *runtime.ServeMux, server }) + mux.Handle("POST", pattern_Tx_TxStatusBatch_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + var stream runtime.ServerTransportStream + ctx = grpc.NewContextWithServerTransportStream(ctx, &stream) + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := local_request_Tx_TxStatusBatch_0(rctx, inboundMarshaler, server, req, pathParams) + md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer()) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Tx_TxStatusBatch_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } @@ -177,13 +234,37 @@ func RegisterTxHandlerClient(ctx context.Context, mux *runtime.ServeMux, client }) + mux.Handle("POST", pattern_Tx_TxStatusBatch_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) { + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req) + rctx, err := runtime.AnnotateContext(ctx, mux, req) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + resp, md, err := request_Tx_TxStatusBatch_0(rctx, inboundMarshaler, client, req, pathParams) + ctx = runtime.NewServerMetadataContext(ctx, md) + if err != nil { + runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err) + return + } + + forward_Tx_TxStatusBatch_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...) + + }) + return nil } var ( pattern_Tx_TxStatus_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 1, 0, 4, 1, 5, 4}, []string{"celestia", "core", "v1", "tx", "tx_id"}, "", runtime.AssumeColonVerbOpt(false))) + + pattern_Tx_TxStatusBatch_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3, 2, 4}, []string{"celestia", "core", "v1", "tx", "batch"}, "", runtime.AssumeColonVerbOpt(false))) ) var ( forward_Tx_TxStatus_0 = runtime.ForwardResponseMessage + + forward_Tx_TxStatusBatch_0 = runtime.ForwardResponseMessage ) diff --git a/go.mod b/go.mod index 39636feca1..3f9610554c 100644 --- a/go.mod +++ b/go.mod @@ -17,11 +17,11 @@ require ( cosmossdk.io/x/feegrant v0.1.1 cosmossdk.io/x/tx v0.13.8 cosmossdk.io/x/upgrade v0.1.4 - github.com/aws/aws-sdk-go-v2 v1.39.6 - github.com/aws/aws-sdk-go-v2/config v1.31.17 - github.com/aws/aws-sdk-go-v2/credentials v1.18.21 + github.com/aws/aws-sdk-go-v2 v1.40.0 + github.com/aws/aws-sdk-go-v2/config v1.31.20 + github.com/aws/aws-sdk-go-v2/credentials v1.18.24 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11 - github.com/aws/aws-sdk-go-v2/service/s3 v1.90.2 + github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0 github.com/bcp-innovations/hyperlane-cosmos v1.0.1 github.com/celestiaorg/go-square/v2 v2.3.3 github.com/celestiaorg/go-square/v3 v3.0.2 @@ -54,8 +54,8 @@ require ( golang.org/x/oauth2 v0.33.0 golang.org/x/sync v0.18.0 google.golang.org/api v0.256.0 - google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c - google.golang.org/grpc v1.76.0 + google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8 + google.golang.org/grpc v1.77.0 google.golang.org/protobuf v1.36.10 gopkg.in/yaml.v2 v2.4.0 ) @@ -76,7 +76,7 @@ require ( github.com/99designs/keyring v1.2.2 // indirect github.com/DataDog/datadog-go v4.8.3+incompatible // indirect github.com/DataDog/zstd v1.5.6 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.53.0 // indirect github.com/Microsoft/go-winio v0.6.2 // indirect @@ -84,17 +84,17 @@ require ( github.com/aws/aws-sdk-go v1.55.5 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3 // indirect github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.30.1 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.39.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.30.3 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.40.2 // indirect github.com/aws/smithy-go v1.23.2 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bgentry/go-netrc v0.0.0-20140422174119-9fd32a8b3d3d // indirect @@ -106,7 +106,7 @@ require ( github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/chzyer/readline v1.5.1 // indirect - github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 // indirect + github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f // indirect github.com/cockroachdb/apd/v2 v2.0.2 // indirect github.com/cockroachdb/errors v1.11.3 // indirect github.com/cockroachdb/fifo v0.0.0-20240816210425-c5d0cb0b6fc0 // indirect @@ -132,14 +132,14 @@ require ( github.com/dustin/go-humanize v1.0.1 // indirect github.com/dvsekhvalnov/jose2go v1.7.0 // indirect github.com/emicklei/dot v1.6.2 // indirect - github.com/envoyproxy/go-control-plane/envoy v1.32.4 // indirect + github.com/envoyproxy/go-control-plane/envoy v1.35.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.2.1 // indirect github.com/ethereum/go-ethereum v1.15.8 // indirect github.com/fatih/color v1.18.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/fsnotify/fsnotify v1.9.0 // indirect github.com/getsentry/sentry-go v0.35.0 // indirect - github.com/go-jose/go-jose/v4 v4.1.2 // indirect + github.com/go-jose/go-jose/v4 v4.1.3 // indirect github.com/go-kit/kit v0.13.0 // indirect github.com/go-logfmt/logfmt v0.6.1 // indirect github.com/go-logr/logr v1.4.3 // indirect @@ -219,7 +219,7 @@ require ( github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus/client_golang v1.23.2 // indirect github.com/prometheus/client_model v0.6.2 // indirect - github.com/prometheus/common v0.67.2 // indirect + github.com/prometheus/common v0.67.4 // indirect github.com/prometheus/procfs v0.16.1 // indirect github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/rogpeppe/go-internal v1.14.1 // indirect @@ -228,21 +228,20 @@ require ( github.com/sasha-s/go-deadlock v0.3.6 // indirect github.com/sourcegraph/conc v0.3.1-0.20240121214520-5f936abd7ae8 // indirect github.com/spf13/afero v1.15.0 // indirect - github.com/spiffe/go-spiffe/v2 v2.5.0 // indirect + github.com/spiffe/go-spiffe/v2 v2.6.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/subosito/gotenv v1.6.0 // indirect github.com/syndtr/goleveldb v1.0.1-0.20220721030215-126854af5e6d // indirect github.com/tendermint/go-amino v0.16.0 // indirect github.com/tidwall/btree v1.7.0 // indirect github.com/ulikunitz/xz v0.5.14 // indirect - github.com/zeebo/errs v1.4.0 // indirect github.com/zondax/golem v0.27.0 // indirect github.com/zondax/hid v0.9.2 // indirect github.com/zondax/ledger-go v1.0.1 // indirect go.etcd.io/bbolt v1.4.0 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/auto/sdk v1.1.0 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.36.0 // indirect + go.opentelemetry.io/auto/sdk v1.2.1 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.38.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.62.0 // indirect go.opentelemetry.io/otel/exporters/stdout/stdouttrace v1.38.0 // indirect @@ -253,12 +252,12 @@ require ( go.uber.org/zap v1.27.0 // indirect go.yaml.in/yaml/v2 v2.4.3 // indirect go.yaml.in/yaml/v3 v3.0.4 // indirect - golang.org/x/crypto v0.43.0 // indirect + golang.org/x/crypto v0.45.0 // indirect golang.org/x/exp v0.0.0-20250911091902-df9299821621 // indirect - golang.org/x/net v0.46.0 // indirect - golang.org/x/sys v0.37.0 // indirect - golang.org/x/term v0.36.0 // indirect - golang.org/x/text v0.30.0 // indirect + golang.org/x/net v0.47.0 // indirect + golang.org/x/sys v0.38.0 // indirect + golang.org/x/term v0.37.0 // indirect + golang.org/x/text v0.31.0 // indirect golang.org/x/time v0.14.0 // indirect google.golang.org/genproto v0.0.0-20250603155806-513f23925822 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20251103181224-f26f9409b101 // indirect @@ -274,7 +273,7 @@ replace ( // f48fea92e627 commit coincides with the v0.51.8 cosmos-sdk release cosmossdk.io/log => github.com/celestiaorg/cosmos-sdk/log v1.1.1-0.20251116153902-f48fea92e627 cosmossdk.io/x/upgrade => github.com/celestiaorg/cosmos-sdk/x/upgrade v0.2.0 - github.com/cometbft/cometbft => github.com/celestiaorg/celestia-core v0.39.15 + github.com/cometbft/cometbft => /Users/ninabarbakadze/Desktop/celestia/celestia-core github.com/cosmos/cosmos-sdk => github.com/celestiaorg/cosmos-sdk v0.51.8 github.com/cosmos/ibc-go/v8 => github.com/celestiaorg/ibc-go/v8 v8.7.2 // Use ledger-cosmos-go v0.16.0 because v0.15.0 causes "hidapi: unknown failure" diff --git a/go.sum b/go.sum index 9536519448..7f7412829f 100644 --- a/go.sum +++ b/go.sum @@ -660,8 +660,8 @@ github.com/DataDog/datadog-go v4.8.3+incompatible h1:fNGaYSuObuQb5nzeTQqowRAd9bp github.com/DataDog/datadog-go v4.8.3+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/DataDog/zstd v1.5.6 h1:LbEglqepa/ipmmQJUDnSsfvA8e8IStVcGaFWDuxvGOY= github.com/DataDog/zstd v1.5.6/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 h1:UQUsRi8WTzhZntp5313l+CHIAT95ojUI2lpP/ExlZa4= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0/go.mod h1:Cz6ft6Dkn3Et6l2v2a9/RpN7epQ1GtDlO6lj8bEcOvw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0 h1:sBEjpZlNHzK1voKq9695PJSX2o5NEXl7/OL3coiIY0c= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.30.0/go.mod h1:P4WPRUkOhJC13W//jWpyfJNDAIpvRbAUIYLX/4jtlE0= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 h1:owcC2UnmsZycprQ5RfRgjydWhuoxg71LUfyiQdijZuM= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0/go.mod h1:ZPpqegjbE99EPKsu3iUWV22A04wzGPcAY/ziSIQEEgs= github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.53.0 h1:4LP6hvB4I5ouTbGgWtixJhgED6xdf67twf9PoY96Tbg= @@ -712,42 +712,42 @@ github.com/aws/aws-sdk-go v1.44.122/go.mod h1:y4AeaBuwd2Lk+GepC1E9v0qOiTws0MIWAX github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU= github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= -github.com/aws/aws-sdk-go-v2 v1.39.6 h1:2JrPCVgWJm7bm83BDwY5z8ietmeJUbh3O2ACnn+Xsqk= -github.com/aws/aws-sdk-go-v2 v1.39.6/go.mod h1:c9pm7VwuW0UPxAEYGyTmyurVcNrbF6Rt/wixFqDhcjE= +github.com/aws/aws-sdk-go-v2 v1.40.0 h1:/WMUA0kjhZExjOQN2z3oLALDREea1A7TobfuiBrKlwc= +github.com/aws/aws-sdk-go-v2 v1.40.0/go.mod h1:c9pm7VwuW0UPxAEYGyTmyurVcNrbF6Rt/wixFqDhcjE= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3 h1:DHctwEM8P8iTXFxC/QK0MRjwEpWQeM9yzidCRjldUz0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.3/go.mod h1:xdCzcZEtnSTKVDOmUZs4l/j3pSV6rpo1WXl5ugNsL8Y= -github.com/aws/aws-sdk-go-v2/config v1.31.17 h1:QFl8lL6RgakNK86vusim14P2k8BFSxjvUkcWLDjgz9Y= -github.com/aws/aws-sdk-go-v2/config v1.31.17/go.mod h1:V8P7ILjp/Uef/aX8TjGk6OHZN6IKPM5YW6S78QnRD5c= -github.com/aws/aws-sdk-go-v2/credentials v1.18.21 h1:56HGpsgnmD+2/KpG0ikvvR8+3v3COCwaF4r+oWwOeNA= -github.com/aws/aws-sdk-go-v2/credentials v1.18.21/go.mod h1:3YELwedmQbw7cXNaII2Wywd+YY58AmLPwX4LzARgmmA= +github.com/aws/aws-sdk-go-v2/config v1.31.20 h1:/jWF4Wu90EhKCgjTdy1DGxcbcbNrjfBHvksEL79tfQc= +github.com/aws/aws-sdk-go-v2/config v1.31.20/go.mod h1:95Hh1Tc5VYKL9NJ7tAkDcqeKt+MCXQB1hQZaRdJIZE0= +github.com/aws/aws-sdk-go-v2/credentials v1.18.24 h1:iJ2FmPT35EaIB0+kMa6TnQ+PwG5A1prEdAw+PsMzfHg= +github.com/aws/aws-sdk-go-v2/credentials v1.18.24/go.mod h1:U91+DrfjAiXPDEGYhh/x29o4p0qHX5HDqG7y5VViv64= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13 h1:T1brd5dR3/fzNFAQch/iBKeX07/ffu/cLu+q+RuzEWk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.13/go.mod h1:Peg/GBAQ6JDt+RoBf4meB1wylmAipb7Kg2ZFakZTlwk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11 h1:w4GjasReY0m9vZA/3YhoBUBi1ZIWUHYQRm61v0BKcZg= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.11/go.mod h1:IPS1CSYQ8lfLYGytpMEPW4erZmVFUdxLpC0RCI/RCn8= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13 h1:a+8/MLcWlIxo1lF9xaGt3J/u3yOZx+CdSveSNwjhD40= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.13/go.mod h1:oGnKwIYZ4XttyU2JWxFrwvhF6YKiK/9/wmE3v3Iu9K8= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13 h1:HBSI2kDkMdWz4ZM7FjwE7e/pWDEZ+nR95x8Ztet1ooY= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.13/go.mod h1:YE94ZoDArI7awZqJzBAZ3PDD2zSfuP7w6P2knOzIn8M= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14 h1:PZHqQACxYb8mYgms4RZbhZG0a7dPW06xOjmaH0EJC/I= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.14/go.mod h1:VymhrMJUWs69D8u0/lZ7jSB6WgaG/NqHi3gX0aYf6U0= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14 h1:bOS19y6zlJwagBfHxs0ESzr1XCOU2KXJCWcq3E2vfjY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.14/go.mod h1:1ipeGBMAxZ0xcTm6y6paC2C/J6f6OO7LBODV9afuAyM= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4 h1:WKuaxf++XKWlHWu9ECbMlha8WOEGm0OUEZqm4K/Gcfk= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.4/go.mod h1:ZWy7j6v1vWGmPReu0iSGvRiise4YI5SkR3OHKTZ6Wuc= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13 h1:eg/WYAa12vqTphzIdWMzqYRVKKnCboVPRlvaybNCqPA= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.13/go.mod h1:/FDdxWhz1486obGrKKC1HONd7krpk38LBt+dutLcN9k= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14 h1:ITi7qiDSv/mSGDSWNpZ4k4Ve0DQR6Ug2SJQ8zEHoDXg= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.14/go.mod h1:k1xtME53H1b6YpZt74YmwlONMWf4ecM+lut1WQLAF/U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3 h1:x2Ibm/Af8Fi+BH+Hsn9TXGdT+hKbDd5XOTZxTMxDk7o= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.3/go.mod h1:IW1jwyrQgMdhisceG8fQLmQIydcT/jWY21rFhzgaKwo= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4 h1:NvMjwvv8hpGUILarKw7Z4Q0w1H9anXKsesMxtw++MA4= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.4/go.mod h1:455WPHSwaGj2waRSpQp7TsnpOnBfw8iDfPfbwl7KPJE= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13 h1:kDqdFvMY4AtKoACfzIGD8A0+hbT41KTKF//gq7jITfM= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.13/go.mod h1:lmKuogqSU3HzQCwZ9ZtcqOc5XGMqtDK7OIc2+DxiUEg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13 h1:zhBJXdhWIFZ1acfDYIhu4+LCzdUS2Vbcum7D01dXlHQ= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.13/go.mod h1:JaaOeCE368qn2Hzi3sEzY6FgAZVCIYcC2nwbro2QCh8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.90.2 h1:DhdbtDl4FdNlj31+xiRXANxEE+eC7n8JQz+/ilwQ8Uc= -github.com/aws/aws-sdk-go-v2/service/s3 v1.90.2/go.mod h1:+wArOOrcHUevqdto9k1tKOF5++YTe9JEcPSc9Tx2ZSw= -github.com/aws/aws-sdk-go-v2/service/sso v1.30.1 h1:0JPwLz1J+5lEOfy/g0SURC9cxhbQ1lIMHMa+AHZSzz0= -github.com/aws/aws-sdk-go-v2/service/sso v1.30.1/go.mod h1:fKvyjJcz63iL/ftA6RaM8sRCtN4r4zl4tjL3qw5ec7k= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5 h1:OWs0/j2UYR5LOGi88sD5/lhN6TDLG6SfA7CqsQO9zF0= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.5/go.mod h1:klO+ejMvYsB4QATfEOIXk8WAEwN4N0aBfJpvC+5SZBo= -github.com/aws/aws-sdk-go-v2/service/sts v1.39.1 h1:mLlUgHn02ue8whiR4BmxxGJLR2gwU6s6ZzJ5wDamBUs= -github.com/aws/aws-sdk-go-v2/service/sts v1.39.1/go.mod h1:E19xDjpzPZC7LS2knI9E6BaRFDK43Eul7vd6rSq2HWk= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5 h1:Hjkh7kE6D81PgrHlE/m9gx+4TyyeLHuY8xJs7yXN5C4= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.9.5/go.mod h1:nPRXgyCfAurhyaTMoBMwRBYBhaHI4lNPAnJmjM0Tslc= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14 h1:FIouAnCE46kyYqyhs0XEBDFFSREtdnr8HQuLPQPLCrY= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.14/go.mod h1:UTwDc5COa5+guonQU8qBikJo1ZJ4ln2r1MkF7Dqag1E= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14 h1:FzQE21lNtUor0Fb7QNgnEyiRCBlolLTX/Z1j65S7teM= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.14/go.mod h1:s1ydyWG9pm3ZwmmYN21HKyG9WzAZhYVW85wMHs5FV6w= +github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0 h1:8FshVvnV2sr9kOSAbOnc/vwVmmAwMjOedKH6JW2ddPM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.92.0/go.mod h1:wYNqY3L02Z3IgRYxOBPH9I1zD9Cjh9hI5QOy/eOjQvw= +github.com/aws/aws-sdk-go-v2/service/sso v1.30.3 h1:NjShtS1t8r5LUfFVtFeI8xLAHQNTa7UI0VawXlrBMFQ= +github.com/aws/aws-sdk-go-v2/service/sso v1.30.3/go.mod h1:fKvyjJcz63iL/ftA6RaM8sRCtN4r4zl4tjL3qw5ec7k= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7 h1:gTsnx0xXNQ6SBbymoDvcoRHL+q4l/dAFsQuKfDWSaGc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.35.7/go.mod h1:klO+ejMvYsB4QATfEOIXk8WAEwN4N0aBfJpvC+5SZBo= +github.com/aws/aws-sdk-go-v2/service/sts v1.40.2 h1:HK5ON3KmQV2HcAunnx4sKLB9aPf3gKGwVAf7xnx0QT0= +github.com/aws/aws-sdk-go-v2/service/sts v1.40.2/go.mod h1:E19xDjpzPZC7LS2knI9E6BaRFDK43Eul7vd6rSq2HWk= github.com/aws/smithy-go v1.23.2 h1:Crv0eatJUQhaManss33hS5r40CG3ZFH+21XSkqMrIUM= github.com/aws/smithy-go v1.23.2/go.mod h1:LEj2LM3rBRQJxPZTB4KuzZkaZYnZPnvgIhb4pu07mx0= github.com/bcp-innovations/hyperlane-cosmos v1.0.1 h1:gT8OqyJ866Q6AHOlIXKxSdLjd0p8crKG9XXERIWoh4c= @@ -775,8 +775,6 @@ github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1/go.mod h1:7SFka0XMvUgj3hfZtyd github.com/bufbuild/protocompile v0.14.1 h1:iA73zAf/fyljNjQKwYzUHD6AD4R8KMasmwa/FBatYVw= github.com/bufbuild/protocompile v0.14.1/go.mod h1:ppVdAIhbr2H8asPk6k4pY7t9zB1OU5DoEw9xY/FUi1c= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= -github.com/celestiaorg/celestia-core v0.39.15 h1:Wpmu0nAPzhEiK2TbSGeTus4c0kikHQGs0Y3Q1q9eo1o= -github.com/celestiaorg/celestia-core v0.39.15/go.mod h1:98r/9+ol56uhrtLQPmTpgGto2DJ4n8ga45T0xdSg4C0= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35 h1:FREwqZwPvYsodr1AqqEIyW+VsBnwTzJNtC6NFdZX8rs= github.com/celestiaorg/celestia-core v1.55.0-tm-v0.34.35/go.mod h1:SI38xqZZ4ccoAxszUJqsJ/a5rOkzQRijzHQQlLKkyUc= github.com/celestiaorg/cosmos-sdk v0.51.8 h1:kU86k7Vf/WynGIpoGbPn42X5ww/4A75K4ldKXn4TlJY= @@ -839,8 +837,8 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 h1:aQ3y1lwWyqYPiWZThqv1aFbZMiM9vblcSArJRf2Irls= -github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= +github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f h1:Y8xYupdHxryycyPlc9Y+bSQAYZnetRJ70VMVKm5CKI0= +github.com/cncf/xds/go v0.0.0-20251022180443-0feb69152e9f/go.mod h1:HlzOvOjVBOfTGSRXRyY0OiCS/3J1akRGQQpRO/7zyF4= github.com/cockroachdb/apd/v2 v2.0.2 h1:weh8u7Cneje73dDh+2tEVLUvyBc89iwepWCD8b8034E= github.com/cockroachdb/apd/v2 v2.0.2/go.mod h1:DDxRlzC2lo3/vSlmSoS7JkqbbrARPuFOGr0B9pvN3Gw= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= @@ -949,10 +947,10 @@ github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go. github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go.mod h1:KJwIaB5Mv44NWtYuAOFCVOjcI94vtpEz2JU/D2v6IjE= github.com/envoyproxy/go-control-plane v0.10.3/go.mod h1:fJJn/j26vwOu972OllsvAgJJM//w9BV6Fxbg2LuVd34= github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q= -github.com/envoyproxy/go-control-plane v0.13.4 h1:zEqyPVyku6IvWCFwux4x9RxkLOMUL+1vC9xUFv5l2/M= -github.com/envoyproxy/go-control-plane v0.13.4/go.mod h1:kDfuBlDVsSj2MjrLEtRWtHlsWIFcGyB2RMO44Dc5GZA= -github.com/envoyproxy/go-control-plane/envoy v1.32.4 h1:jb83lalDRZSpPWW2Z7Mck/8kXZ5CQAFYVjQcdVIr83A= -github.com/envoyproxy/go-control-plane/envoy v1.32.4/go.mod h1:Gzjc5k8JcJswLjAx1Zm+wSYE20UrLtt7JZMWiWQXQEw= +github.com/envoyproxy/go-control-plane v0.13.5-0.20251024222203-75eaa193e329 h1:K+fnvUM0VZ7ZFJf0n4L/BRlnsb9pL/GuDG6FqaH+PwM= +github.com/envoyproxy/go-control-plane v0.13.5-0.20251024222203-75eaa193e329/go.mod h1:Alz8LEClvR7xKsrq3qzoc4N0guvVNSS8KmSChGYr9hs= +github.com/envoyproxy/go-control-plane/envoy v1.35.0 h1:ixjkELDE+ru6idPxcHLj8LBVc2bFP7iBytj353BoHUo= +github.com/envoyproxy/go-control-plane/envoy v1.35.0/go.mod h1:09qwbGVuSWWAyN5t/b3iyVfz5+z8QWGrzkoqm/8SbEs= github.com/envoyproxy/go-control-plane/ratelimit v0.1.0 h1:/G9QYbddjL25KvtKTv3an9lx6VBE2cnb8wp1vEGNYGI= github.com/envoyproxy/go-control-plane/ratelimit v0.1.0/go.mod h1:Wk+tMFAFbCXaJPzVVHnPgRKdUdwW/KdbRt94AzgRee4= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= @@ -996,8 +994,8 @@ github.com/go-fonts/stix v0.1.0/go.mod h1:w/c1f0ldAUlJmLBvlbkvVXLAD+tAMqobIIQpmn github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-jose/go-jose/v4 v4.1.2 h1:TK/7NqRQZfgAh+Td8AlsrvtPoUyiHh0LqVvokh+1vHI= -github.com/go-jose/go-jose/v4 v4.1.2/go.mod h1:22cg9HWM1pOlnRiY+9cQYJ9XHmya1bYW8OeDM6Ku6Oo= +github.com/go-jose/go-jose/v4 v4.1.3 h1:CVLmWDhDVRa6Mi/IgCgaopNosCaHz7zrMeF9MlZRkrs= +github.com/go-jose/go-jose/v4 v4.1.3/go.mod h1:x4oUasVrzR7071A4TnHLGSPpNOm2a21K9Kf04k1rs08= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= @@ -1519,8 +1517,8 @@ github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8b github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.67.2 h1:PcBAckGFTIHt2+L3I33uNRTlKTplNzFctXcWhPyAEN8= -github.com/prometheus/common v0.67.2/go.mod h1:63W3KZb1JOKgcjlIr64WW/LvFGAqKPj0atm+knVGEko= +github.com/prometheus/common v0.67.4 h1:yR3NqWO1/UyO1w2PhUvXlGQs/PtFmoveVO0KZ4+Lvsc= +github.com/prometheus/common v0.67.4/go.mod h1:gP0fq6YjjNCLssJCQp0yk4M8W6ikLURwkdd/YKtTbyI= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= @@ -1590,8 +1588,8 @@ github.com/spf13/pflag v1.0.10 h1:4EBh2KAYBwaONj6b2Ye1GiHfwjqyROoF4RwYO+vPwFk= github.com/spf13/pflag v1.0.10/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.21.0 h1:x5S+0EU27Lbphp4UKm1C+1oQO+rKx36vfCoaVebLFSU= github.com/spf13/viper v1.21.0/go.mod h1:P0lhsswPGWD/1lZJ9ny3fYnVqxiegrlNrEmgLjbTCAY= -github.com/spiffe/go-spiffe/v2 v2.5.0 h1:N2I01KCUkv1FAjZXJMwh95KK1ZIQLYbPfhaxw8WS0hE= -github.com/spiffe/go-spiffe/v2 v2.5.0/go.mod h1:P+NxobPc6wXhVtINNtFjNWGBTreew1GBUCwT2wPmb7g= +github.com/spiffe/go-spiffe/v2 v2.6.0 h1:l+DolpxNWYgruGQVV0xsfeya3CsC7m8iBzDnMpsbLuo= +github.com/spiffe/go-spiffe/v2 v2.6.0/go.mod h1:gm2SeUoMZEtpnzPNs2Csc0D/gX33k1xIx7lEzqblHEs= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= @@ -1649,8 +1647,6 @@ github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1 github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0= -github.com/zeebo/errs v1.4.0 h1:XNdoD/RRMKP7HD0UhJnIzUy74ISdGGxURlYG8HSWSfM= -github.com/zeebo/errs v1.4.0/go.mod h1:sgbWHsvVuTPHcqJJGQ1WhI5KbWlHYz+2+2C/LSEtCw4= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= github.com/zondax/golem v0.27.0 h1:IbBjGIXF3SoGOZHsILJvIM/F/ylwJzMcHAcggiqniPw= github.com/zondax/golem v0.27.0/go.mod h1:AmorCgJPt00L8xN1VrMBe13PSifoZksnQ1Ge906bu4A= @@ -1678,10 +1674,10 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= -go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= -go.opentelemetry.io/contrib/detectors/gcp v1.36.0 h1:F7q2tNlCaHY9nMKHR6XH9/qkp8FktLnIcy6jJNyOCQw= -go.opentelemetry.io/contrib/detectors/gcp v1.36.0/go.mod h1:IbBN8uAIIx734PTonTPxAxnjc2pQTxWNkwfstZ+6H2k= +go.opentelemetry.io/auto/sdk v1.2.1 h1:jXsnJ4Lmnqd11kwkBV2LgLoFMZKizbCi5fNZ/ipaZ64= +go.opentelemetry.io/auto/sdk v1.2.1/go.mod h1:KRTj+aOaElaLi+wW1kO/DZRXwkF4C5xPbEe3ZiIhN7Y= +go.opentelemetry.io/contrib/detectors/gcp v1.38.0 h1:ZoYbqX7OaA/TAikspPl3ozPI6iY6LiIY9I8cUfm+pJs= +go.opentelemetry.io/contrib/detectors/gcp v1.38.0/go.mod h1:SU+iU7nu5ud4oCb3LQOhIZ3nRLj6FNVrKgtflbaf2ts= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0 h1:q4XOmH/0opmeuJtPsbFNivyl7bCt7yRBbeEm2sC/XtQ= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0/go.mod h1:snMWehoOh2wsEwnvvwtDyFCxVeDAODenXHtn5vzrKjo= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.62.0 h1:Hf9xI/XLML9ElpiHVDNwvqI0hIFlzV8dgIr35kV1kRU= @@ -1749,8 +1745,8 @@ golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliY golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.32.0/go.mod h1:ZnnJkOaASj8g0AjIduWNlq2NRxL0PlBrbKVyZ6V/Ugc= -golang.org/x/crypto v0.43.0 h1:dduJYIi3A3KOfdGOHX8AVZ/jGiyPa3IbBozJ5kNuE04= -golang.org/x/crypto v0.43.0/go.mod h1:BFbav4mRNlXJL4wNeejLpWxB7wMbc79PdRGhWKncxR0= +golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= +golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1888,8 +1884,8 @@ golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.34.0/go.mod h1:di0qlW3YNM5oh6GqDGQr92MyTozJPmybPK4Ev/Gm31k= -golang.org/x/net v0.46.0 h1:giFlY12I07fugqwPuWJi68oOnpfqFnJIJzaIIm2JVV4= -golang.org/x/net v0.46.0/go.mod h1:Q9BGdFy1y4nkUwiLvT5qtyhAnEHgnQ/zd8PfU6nc210= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -2055,8 +2051,8 @@ golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.29.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.37.0 h1:fdNQudmxPjkdUTPnLn5mdQv7Zwvbvpaxqs831goi9kQ= -golang.org/x/sys v0.37.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/sys v0.38.0 h1:3yZWxaJjBmCWXqhN1qh02AkOnCQ1poK6oF+a7xWL6Gc= +golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -2071,8 +2067,8 @@ golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.28.0/go.mod h1:Sw/lC2IAUZ92udQNf3WodGtn4k/XoLyZoh8v/8uiwek= -golang.org/x/term v0.36.0 h1:zMPR+aF8gfksFprF/Nc/rd1wRS1EI6nDBGyWAvDzx2Q= -golang.org/x/term v0.36.0/go.mod h1:Qu394IJq6V6dCBRgwqshf3mPF85AqzYEzofzRdZkWss= +golang.org/x/term v0.37.0 h1:8EGAD0qCmHYZg6J17DvsMy9/wJ7/D/4pV/wfnld5lTU= +golang.org/x/term v0.37.0/go.mod h1:5pB4lxRNYYVZuTLmy8oR2BH8dflOR+IbTYFD8fi3254= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -2093,8 +2089,8 @@ golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= -golang.org/x/text v0.30.0 h1:yznKA/E9zq54KzlzBEAWn1NXSQ8DIp/NYMy88xJjl4k= -golang.org/x/text v0.30.0/go.mod h1:yDdHFIX9t+tORqspjENWgzaCVXgk0yYnYuSZ8UzzBVM= +golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= +golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -2179,8 +2175,8 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= -golang.org/x/tools v0.37.0 h1:DVSRzp7FwePZW356yEAChSdNcQo6Nsp+fex1SUW09lE= -golang.org/x/tools v0.37.0/go.mod h1:MBN5QPQtLMHVdvsbtarmTNukZDdgwdwlO5qGacAzF0w= +golang.org/x/tools v0.38.0 h1:Hx2Xv8hISq8Lm16jvBZ2VQf+RLmbd7wVUsALibYI/IQ= +golang.org/x/tools v0.38.0/go.mod h1:yEsQ/d/YK8cjh0L6rZlY8tgtlKiBNTL14pGDJPJpYQs= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -2406,8 +2402,8 @@ google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20250603155806-513f23925822 h1:rHWScKit0gvAPuOnu87KpaYtjK5zBMLcULh7gxkCXu4= google.golang.org/genproto v0.0.0-20250603155806-513f23925822/go.mod h1:HubltRL7rMh0LfnQPkMH4NPDFEWp0jw3vixw7jEM53s= -google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c h1:AtEkQdl5b6zsybXcbz00j1LwNodDuH6hVifIaNqk7NQ= -google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c/go.mod h1:ea2MjsO70ssTfCjiwHgI0ZFqcw45Ksuk2ckf9G468GA= +google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8 h1:mepRgnBZa07I4TRuomDE4sTIYieg/osKmzIf4USdWS4= +google.golang.org/genproto/googleapis/api v0.0.0-20251022142026-3a174f9686a8/go.mod h1:fDMmzKV90WSg1NbozdqrE64fkuTv6mlq2zxo9ad+3yo= google.golang.org/genproto/googleapis/rpc v0.0.0-20251103181224-f26f9409b101 h1:tRPGkdGHuewF4UisLzzHHr1spKw92qLM98nIzxbC0wY= google.golang.org/genproto/googleapis/rpc v0.0.0-20251103181224-f26f9409b101/go.mod h1:7i2o+ce6H/6BluujYR+kqX3GKH+dChPTQU19wjRPiGk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= @@ -2457,8 +2453,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.76.0 h1:UnVkv1+uMLYXoIz6o7chp59WfQUYA2ex/BXQ9rHZu7A= -google.golang.org/grpc v1.76.0/go.mod h1:Ju12QI8M6iQJtbcsV+awF5a4hfJMLi4X0JLo94ULZ6c= +google.golang.org/grpc v1.77.0 h1:wVVY6/8cGA6vvffn+wWK5ToddbgdU3d8MNENr4evgXM= +google.golang.org/grpc v1.77.0/go.mod h1:z0BY1iVj0q8E1uSQCjL9cppRj+gnZjzDnzV0dHhrNig= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= diff --git a/pkg/user/tx_client.go b/pkg/user/tx_client.go index 577529eb30..9c19fac852 100644 --- a/pkg/user/tx_client.go +++ b/pkg/user/tx_client.go @@ -757,6 +757,18 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon defer pollTicker.Stop() var evictionPollTimeStart *time.Time + txStatusBatchRequest := &tx.TxStatusBatchRequest{ + TxIds: []string{txHash, txHash, txHash, txHash}, + } + txStatusBatchResponse, err := txClient.TxStatusBatch(ctx, txStatusBatchRequest) + if err != nil { + return nil, err + } + + for _, resp := range txStatusBatchResponse.Responses { + fmt.Println("resp from batch: ", resp.TxHash, resp.Status.Status) + } + for { span.AddEvent("txclient/ConfirmTx: polling for TxStatus") resp, err := txClient.TxStatus(ctx, &tx.TxStatusRequest{TxId: txHash}) diff --git a/proto/celestia/core/v1/tx/tx.proto b/proto/celestia/core/v1/tx/tx.proto index 72c22e6d71..1fa910e9e4 100644 --- a/proto/celestia/core/v1/tx/tx.proto +++ b/proto/celestia/core/v1/tx/tx.proto @@ -18,6 +18,14 @@ service Tx { get: "/celestia/core/v1/tx/{tx_id}" }; } + + // TxStatusBatch for batch queries + rpc TxStatusBatch(TxStatusBatchRequest) returns (TxStatusBatchResponse) { + option (google.api.http) = { + post: "/celestia/core/v1/tx/batch" + body: "*" + }; + } } // TxStatusRequest is the request type for the TxStatus gRPC method. @@ -47,3 +55,20 @@ message TxStatusResponse { // signers of the transaction. repeated string signers = 9; } + +// TxStatusRequestBatch +message TxStatusBatchRequest { + // array of hex encoded tx hashes (each hash should be 64 characters long representing 32 bytes) + repeated string tx_ids = 1; +} + +// TxStatusResponseWithHash is the tx hash with respective tx info +message TxStatusResponseWithHash { + string tx_hash = 1; + TxStatusResponse status = 2; +} + +// TxStatusBatchResponse +message TxStatusBatchResponse { + repeated TxStatusResponseWithHash responses = 1; +} From 6836a9ecd9811c54aa4859fbc9bbc97a016a3d59 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Tue, 2 Dec 2025 16:12:38 +0100 Subject: [PATCH 08/12] chore: update the proto comments --- app/grpc/tx/tx.pb.go | 142 ++++++++++++++--------------- proto/celestia/core/v1/tx/tx.proto | 10 +- 2 files changed, 76 insertions(+), 76 deletions(-) diff --git a/app/grpc/tx/tx.pb.go b/app/grpc/tx/tx.pb.go index 2c8eca929f..7a40eb8d7b 100644 --- a/app/grpc/tx/tx.pb.go +++ b/app/grpc/tx/tx.pb.go @@ -192,7 +192,7 @@ func (m *TxStatusResponse) GetSigners() []string { return nil } -// TxStatusRequestBatch +// TxStatusBatchRequest is the request type for the batch TxStatus gRPC method. type TxStatusBatchRequest struct { // array of hex encoded tx hashes (each hash should be 64 characters long representing 32 bytes) TxIds []string `protobuf:"bytes,1,rep,name=tx_ids,json=txIds,proto3" json:"tx_ids,omitempty"` @@ -238,24 +238,24 @@ func (m *TxStatusBatchRequest) GetTxIds() []string { return nil } -// TxStatusResponseWithHash is the tx hash with respective tx info -type TxStatusResponseWithHash struct { +// TxStatusResult represents a single transaction status result in a batch response. +type TxStatusResult struct { TxHash string `protobuf:"bytes,1,opt,name=tx_hash,json=txHash,proto3" json:"tx_hash,omitempty"` Status *TxStatusResponse `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` } -func (m *TxStatusResponseWithHash) Reset() { *m = TxStatusResponseWithHash{} } -func (m *TxStatusResponseWithHash) String() string { return proto.CompactTextString(m) } -func (*TxStatusResponseWithHash) ProtoMessage() {} -func (*TxStatusResponseWithHash) Descriptor() ([]byte, []int) { +func (m *TxStatusResult) Reset() { *m = TxStatusResult{} } +func (m *TxStatusResult) String() string { return proto.CompactTextString(m) } +func (*TxStatusResult) ProtoMessage() {} +func (*TxStatusResult) Descriptor() ([]byte, []int) { return fileDescriptor_7d8b070565b0dcb6, []int{3} } -func (m *TxStatusResponseWithHash) XXX_Unmarshal(b []byte) error { +func (m *TxStatusResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) } -func (m *TxStatusResponseWithHash) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { +func (m *TxStatusResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { if deterministic { - return xxx_messageInfo_TxStatusResponseWithHash.Marshal(b, m, deterministic) + return xxx_messageInfo_TxStatusResult.Marshal(b, m, deterministic) } else { b = b[:cap(b)] n, err := m.MarshalToSizedBuffer(b) @@ -265,35 +265,35 @@ func (m *TxStatusResponseWithHash) XXX_Marshal(b []byte, deterministic bool) ([] return b[:n], nil } } -func (m *TxStatusResponseWithHash) XXX_Merge(src proto.Message) { - xxx_messageInfo_TxStatusResponseWithHash.Merge(m, src) +func (m *TxStatusResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxStatusResult.Merge(m, src) } -func (m *TxStatusResponseWithHash) XXX_Size() int { +func (m *TxStatusResult) XXX_Size() int { return m.Size() } -func (m *TxStatusResponseWithHash) XXX_DiscardUnknown() { - xxx_messageInfo_TxStatusResponseWithHash.DiscardUnknown(m) +func (m *TxStatusResult) XXX_DiscardUnknown() { + xxx_messageInfo_TxStatusResult.DiscardUnknown(m) } -var xxx_messageInfo_TxStatusResponseWithHash proto.InternalMessageInfo +var xxx_messageInfo_TxStatusResult proto.InternalMessageInfo -func (m *TxStatusResponseWithHash) GetTxHash() string { +func (m *TxStatusResult) GetTxHash() string { if m != nil { return m.TxHash } return "" } -func (m *TxStatusResponseWithHash) GetStatus() *TxStatusResponse { +func (m *TxStatusResult) GetStatus() *TxStatusResponse { if m != nil { return m.Status } return nil } -// TxStatusBatchResponse +// TxStatusBatchResponse is a response type for batched TxStatus query. It contains an array of transaction status results. type TxStatusBatchResponse struct { - Responses []*TxStatusResponseWithHash `protobuf:"bytes,1,rep,name=responses,proto3" json:"responses,omitempty"` + Results []*TxStatusResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty"` } func (m *TxStatusBatchResponse) Reset() { *m = TxStatusBatchResponse{} } @@ -329,9 +329,9 @@ func (m *TxStatusBatchResponse) XXX_DiscardUnknown() { var xxx_messageInfo_TxStatusBatchResponse proto.InternalMessageInfo -func (m *TxStatusBatchResponse) GetResponses() []*TxStatusResponseWithHash { +func (m *TxStatusBatchResponse) GetResults() []*TxStatusResult { if m != nil { - return m.Responses + return m.Results } return nil } @@ -340,7 +340,7 @@ func init() { proto.RegisterType((*TxStatusRequest)(nil), "celestia.core.v1.tx.TxStatusRequest") proto.RegisterType((*TxStatusResponse)(nil), "celestia.core.v1.tx.TxStatusResponse") proto.RegisterType((*TxStatusBatchRequest)(nil), "celestia.core.v1.tx.TxStatusBatchRequest") - proto.RegisterType((*TxStatusResponseWithHash)(nil), "celestia.core.v1.tx.TxStatusResponseWithHash") + proto.RegisterType((*TxStatusResult)(nil), "celestia.core.v1.tx.TxStatusResult") proto.RegisterType((*TxStatusBatchResponse)(nil), "celestia.core.v1.tx.TxStatusBatchResponse") } @@ -348,39 +348,39 @@ func init() { proto.RegisterFile("celestia/core/v1/tx/tx.proto", fileDescriptor_ var fileDescriptor_7d8b070565b0dcb6 = []byte{ // 529 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0xcf, 0x8b, 0xd3, 0x40, - 0x14, 0x6e, 0xda, 0xed, 0x8f, 0xbc, 0xa5, 0x2a, 0xb3, 0xbb, 0x3a, 0x96, 0x1a, 0x4b, 0xd8, 0x4a, - 0x5d, 0x68, 0xc2, 0xd6, 0x9b, 0xe0, 0x65, 0xbd, 0xb8, 0x78, 0x8b, 0x2b, 0x0b, 0x5e, 0xca, 0x34, - 0x19, 0x92, 0xc0, 0x9a, 0x89, 0x99, 0xd7, 0x75, 0x40, 0xf6, 0xe2, 0xc5, 0x93, 0x20, 0xec, 0x3f, - 0xe5, 0x71, 0xc1, 0x8b, 0x47, 0x69, 0xfd, 0x43, 0x24, 0xd3, 0xa4, 0xd5, 0x52, 0xdc, 0x1e, 0x0a, - 0xf3, 0xde, 0xf7, 0xde, 0xf7, 0xbe, 0xf7, 0xbd, 0x06, 0xba, 0x3e, 0xbf, 0xe0, 0x12, 0x63, 0xe6, - 0xfa, 0x22, 0xe3, 0xee, 0xe5, 0xb1, 0x8b, 0xca, 0x45, 0xe5, 0xa4, 0x99, 0x40, 0x41, 0xf6, 0x4a, - 0xd4, 0xc9, 0x51, 0xe7, 0xf2, 0xd8, 0x41, 0xd5, 0xe9, 0x86, 0x42, 0x84, 0x17, 0xdc, 0x65, 0x69, - 0xec, 0xb2, 0x24, 0x11, 0xc8, 0x30, 0x16, 0x89, 0x5c, 0xb4, 0xd8, 0x4f, 0xe0, 0xee, 0x99, 0x7a, - 0x83, 0x0c, 0xa7, 0xd2, 0xe3, 0x1f, 0xa6, 0x5c, 0x22, 0xd9, 0x83, 0x3a, 0xaa, 0x71, 0x1c, 0x50, - 0xa3, 0x67, 0x0c, 0x4c, 0x6f, 0x07, 0xd5, 0x69, 0x60, 0x7f, 0xa9, 0xc2, 0xbd, 0x55, 0xa1, 0x4c, - 0x45, 0x22, 0x39, 0xb9, 0x0f, 0x8d, 0x88, 0xc7, 0x61, 0x84, 0xba, 0xb4, 0xe6, 0x15, 0x11, 0xd9, - 0x87, 0x7a, 0x9c, 0x04, 0x5c, 0xd1, 0x6a, 0xcf, 0x18, 0xb4, 0xbd, 0x45, 0x40, 0xfa, 0x70, 0x87, - 0x2b, 0xee, 0x4f, 0xf3, 0xf1, 0x63, 0x5f, 0x04, 0x9c, 0xd6, 0x34, 0xdc, 0x5e, 0x66, 0x5f, 0x8a, - 0x80, 0xe7, 0xcd, 0x3c, 0xcb, 0x44, 0x46, 0x77, 0xf4, 0xf8, 0x45, 0x90, 0x8f, 0x92, 0x7a, 0x38, - 0xad, 0xeb, 0x74, 0x11, 0x91, 0x2e, 0x98, 0x39, 0x95, 0x4c, 0x99, 0xcf, 0x69, 0x43, 0x43, 0xab, - 0x04, 0x79, 0x04, 0x10, 0x32, 0x39, 0xfe, 0xc8, 0x12, 0xe4, 0x01, 0x6d, 0x6a, 0x91, 0x66, 0xc8, - 0xe4, 0xb9, 0x4e, 0x90, 0x87, 0xd0, 0xca, 0xe1, 0xa9, 0xe4, 0x01, 0x6d, 0x69, 0xb0, 0x19, 0x32, - 0xf9, 0x56, 0xf2, 0x80, 0x50, 0x68, 0xca, 0x38, 0x4c, 0x78, 0x26, 0xa9, 0xd9, 0xab, 0x0d, 0x4c, - 0xaf, 0x0c, 0xed, 0x21, 0xec, 0x97, 0x46, 0x9c, 0x30, 0xf4, 0xa3, 0xd2, 0xb6, 0x03, 0x68, 0x68, - 0xdb, 0x24, 0x35, 0x74, 0x43, 0x3d, 0xf7, 0x4d, 0xda, 0x19, 0xd0, 0x75, 0xdf, 0xce, 0x63, 0x8c, - 0x5e, 0x31, 0x19, 0x91, 0x07, 0xd0, 0x44, 0x35, 0x8e, 0x98, 0x8c, 0x0a, 0xaf, 0x1b, 0xa8, 0x34, - 0xf0, 0x62, 0xb9, 0x6d, 0xee, 0xe0, 0xee, 0xa8, 0xef, 0x6c, 0xb8, 0xac, 0xb3, 0xce, 0x5b, 0x9a, - 0x62, 0x07, 0x70, 0xb0, 0x26, 0xb1, 0x38, 0xd8, 0x6b, 0x30, 0xb3, 0xe2, 0xbd, 0x90, 0xb9, 0x3b, - 0x1a, 0x6e, 0x45, 0x5d, 0x4a, 0xf6, 0x56, 0xfd, 0xa3, 0xeb, 0x2a, 0x54, 0xcf, 0x14, 0xb9, 0x82, - 0x56, 0x59, 0x4d, 0x0e, 0x6f, 0x21, 0xd3, 0x4e, 0x75, 0xb6, 0xdb, 0xc6, 0x3e, 0xfc, 0xfc, 0xe3, - 0xf7, 0x75, 0xd5, 0x22, 0x5d, 0x77, 0xd3, 0x9f, 0xfe, 0x93, 0x36, 0xfb, 0x8a, 0x7c, 0x35, 0xa0, - 0xfd, 0xcf, 0xb2, 0xe4, 0xe9, 0x7f, 0xe9, 0xff, 0xbe, 0x59, 0xe7, 0x68, 0x9b, 0xd2, 0x42, 0x4e, - 0x5f, 0xcb, 0x79, 0xfc, 0xdc, 0x38, 0xb2, 0x3b, 0x1b, 0x15, 0x4d, 0xf2, 0xf2, 0x93, 0xd3, 0xef, - 0x33, 0xcb, 0xb8, 0x99, 0x59, 0xc6, 0xaf, 0x99, 0x65, 0x7c, 0x9b, 0x5b, 0x95, 0x9b, 0xb9, 0x55, - 0xf9, 0x39, 0xb7, 0x2a, 0xef, 0xdc, 0x30, 0xc6, 0x68, 0x3a, 0x71, 0x7c, 0xf1, 0x7e, 0xd9, 0x2f, - 0xb2, 0x70, 0xf9, 0x1e, 0xb2, 0x34, 0x75, 0xf3, 0x5f, 0x98, 0xa5, 0xbe, 0x8b, 0x6a, 0xd2, 0xd0, - 0x9f, 0xe8, 0xb3, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x8e, 0xc4, 0xc3, 0xef, 0xf5, 0x03, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4f, 0x6f, 0xd3, 0x4e, + 0x10, 0x8d, 0x93, 0xe6, 0xdf, 0x54, 0xe9, 0xef, 0xa7, 0x6d, 0x0b, 0x4b, 0x14, 0x4c, 0x64, 0x1a, + 0x14, 0x2a, 0xd5, 0x56, 0xc3, 0x0d, 0xa9, 0x97, 0x72, 0xa1, 0x57, 0x53, 0x40, 0xe2, 0x12, 0x6d, + 0xec, 0x95, 0x6d, 0x29, 0x78, 0x8d, 0x77, 0x5c, 0x56, 0x42, 0xbd, 0x70, 0xe1, 0x84, 0x84, 0xd4, + 0x2f, 0xc5, 0xb1, 0x12, 0x17, 0x8e, 0x28, 0xe1, 0x83, 0xa0, 0xdd, 0xc4, 0x69, 0xa9, 0x22, 0xda, + 0x83, 0x25, 0xcf, 0xbc, 0x99, 0x79, 0x6f, 0xde, 0xd8, 0xd0, 0x0b, 0xf8, 0x94, 0x4b, 0x4c, 0x98, + 0x17, 0x88, 0x9c, 0x7b, 0x67, 0x87, 0x1e, 0x2a, 0x0f, 0x95, 0x9b, 0xe5, 0x02, 0x05, 0xd9, 0x2e, + 0x51, 0x57, 0xa3, 0xee, 0xd9, 0xa1, 0x8b, 0xaa, 0xdb, 0x8b, 0x84, 0x88, 0xa6, 0xdc, 0x63, 0x59, + 0xe2, 0xb1, 0x34, 0x15, 0xc8, 0x30, 0x11, 0xa9, 0x5c, 0xb4, 0x38, 0x4f, 0xe0, 0xbf, 0x53, 0xf5, + 0x0a, 0x19, 0x16, 0xd2, 0xe7, 0x1f, 0x0a, 0x2e, 0x91, 0x6c, 0x43, 0x1d, 0xd5, 0x38, 0x09, 0xa9, + 0xd5, 0xb7, 0x86, 0x6d, 0x7f, 0x03, 0xd5, 0x49, 0xe8, 0x7c, 0xa9, 0xc2, 0xff, 0x57, 0x85, 0x32, + 0x13, 0xa9, 0xe4, 0xe4, 0x1e, 0x34, 0x62, 0x9e, 0x44, 0x31, 0x9a, 0xd2, 0x9a, 0xbf, 0x8c, 0xc8, + 0x0e, 0xd4, 0x93, 0x34, 0xe4, 0x8a, 0x56, 0xfb, 0xd6, 0xb0, 0xe3, 0x2f, 0x02, 0x32, 0x80, 0x2d, + 0xae, 0x78, 0x50, 0x68, 0xfa, 0x71, 0x20, 0x42, 0x4e, 0x6b, 0x06, 0xee, 0xac, 0xb2, 0x2f, 0x44, + 0xc8, 0x75, 0x33, 0xcf, 0x73, 0x91, 0xd3, 0x0d, 0x43, 0xbf, 0x08, 0x34, 0x95, 0x34, 0xe4, 0xb4, + 0x6e, 0xd2, 0xcb, 0x88, 0xf4, 0xa0, 0xad, 0x47, 0xc9, 0x8c, 0x05, 0x9c, 0x36, 0x0c, 0x74, 0x95, + 0x20, 0x0f, 0x01, 0x22, 0x26, 0xc7, 0x1f, 0x59, 0x8a, 0x3c, 0xa4, 0x4d, 0x23, 0xb2, 0x1d, 0x31, + 0xf9, 0xd6, 0x24, 0xc8, 0x03, 0x68, 0x69, 0xb8, 0x90, 0x3c, 0xa4, 0x2d, 0x03, 0x36, 0x23, 0x26, + 0x5f, 0x4b, 0x1e, 0x12, 0x0a, 0x4d, 0x99, 0x44, 0x29, 0xcf, 0x25, 0x6d, 0xf7, 0x6b, 0xc3, 0xb6, + 0x5f, 0x86, 0xce, 0x01, 0xec, 0x94, 0x46, 0x1c, 0x33, 0x0c, 0xe2, 0xd2, 0xb6, 0x5d, 0x68, 0x18, + 0xdb, 0x24, 0xb5, 0x4c, 0x43, 0x5d, 0xfb, 0x26, 0x9d, 0x18, 0xb6, 0xae, 0xf9, 0x56, 0x4c, 0x91, + 0xdc, 0x87, 0x26, 0xaa, 0x71, 0xcc, 0x64, 0xbc, 0x74, 0xb8, 0x81, 0xea, 0x25, 0x93, 0x31, 0x39, + 0x5a, 0xed, 0xa8, 0x7d, 0xdb, 0x1c, 0x0d, 0xdc, 0x35, 0xf7, 0x74, 0x6f, 0x5e, 0xa1, 0xb4, 0xc2, + 0x79, 0x03, 0xbb, 0x37, 0x84, 0x2d, 0xcf, 0x74, 0x04, 0xcd, 0xdc, 0x50, 0x2f, 0xa4, 0x6d, 0x8e, + 0x1e, 0xdf, 0x36, 0xb8, 0x98, 0xa2, 0x5f, 0xf6, 0x8c, 0x2e, 0xaa, 0x50, 0x3d, 0x55, 0xe4, 0x1c, + 0x5a, 0x65, 0x05, 0xd9, 0xbb, 0x65, 0x80, 0x71, 0xa4, 0x7b, 0x37, 0xfd, 0xce, 0xde, 0xe7, 0x1f, + 0xbf, 0x2f, 0xaa, 0x36, 0xe9, 0x79, 0xeb, 0x3e, 0xee, 0x4f, 0xc6, 0xd4, 0x73, 0xf2, 0xd5, 0x82, + 0xce, 0x5f, 0xeb, 0x91, 0xa7, 0xff, 0x1c, 0x7f, 0xfd, 0x36, 0xdd, 0xfd, 0xbb, 0x94, 0x2e, 0xe5, + 0x0c, 0x8c, 0x9c, 0x47, 0xcf, 0xad, 0x7d, 0xa7, 0xbb, 0x56, 0xd1, 0x44, 0x97, 0x1f, 0x9f, 0x7c, + 0x9f, 0xd9, 0xd6, 0xe5, 0xcc, 0xb6, 0x7e, 0xcd, 0x6c, 0xeb, 0xdb, 0xdc, 0xae, 0x5c, 0xce, 0xed, + 0xca, 0xcf, 0xb9, 0x5d, 0x79, 0xe7, 0x45, 0x09, 0xc6, 0xc5, 0xc4, 0x0d, 0xc4, 0xfb, 0x55, 0xbf, + 0xc8, 0xa3, 0xd5, 0xfb, 0x01, 0xcb, 0x32, 0x4f, 0x3f, 0x51, 0x9e, 0x05, 0x1e, 0xaa, 0x49, 0xc3, + 0xfc, 0x8a, 0xcf, 0xfe, 0x04, 0x00, 0x00, 0xff, 0xff, 0xf8, 0x96, 0xbd, 0xf7, 0xdd, 0x03, 0x00, 0x00, } @@ -655,7 +655,7 @@ func (m *TxStatusBatchRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *TxStatusResponseWithHash) Marshal() (dAtA []byte, err error) { +func (m *TxStatusResult) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -665,12 +665,12 @@ func (m *TxStatusResponseWithHash) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *TxStatusResponseWithHash) MarshalTo(dAtA []byte) (int, error) { +func (m *TxStatusResult) MarshalTo(dAtA []byte) (int, error) { size := m.Size() return m.MarshalToSizedBuffer(dAtA[:size]) } -func (m *TxStatusResponseWithHash) MarshalToSizedBuffer(dAtA []byte) (int, error) { +func (m *TxStatusResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { i := len(dAtA) _ = i var l int @@ -717,10 +717,10 @@ func (m *TxStatusBatchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Responses) > 0 { - for iNdEx := len(m.Responses) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Results) > 0 { + for iNdEx := len(m.Results) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Responses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Results[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -815,7 +815,7 @@ func (m *TxStatusBatchRequest) Size() (n int) { return n } -func (m *TxStatusResponseWithHash) Size() (n int) { +func (m *TxStatusResult) Size() (n int) { if m == nil { return 0 } @@ -838,8 +838,8 @@ func (m *TxStatusBatchResponse) Size() (n int) { } var l int _ = l - if len(m.Responses) > 0 { - for _, e := range m.Responses { + if len(m.Results) > 0 { + for _, e := range m.Results { l = e.Size() n += 1 + l + sovTx(uint64(l)) } @@ -1290,7 +1290,7 @@ func (m *TxStatusBatchRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *TxStatusResponseWithHash) Unmarshal(dAtA []byte) error { +func (m *TxStatusResult) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1313,10 +1313,10 @@ func (m *TxStatusResponseWithHash) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TxStatusResponseWithHash: wiretype end group for non-group") + return fmt.Errorf("proto: TxStatusResult: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TxStatusResponseWithHash: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxStatusResult: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -1439,7 +1439,7 @@ func (m *TxStatusBatchResponse) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Responses", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1466,8 +1466,8 @@ func (m *TxStatusBatchResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Responses = append(m.Responses, &TxStatusResponseWithHash{}) - if err := m.Responses[len(m.Responses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Results = append(m.Results, &TxStatusResult{}) + if err := m.Results[len(m.Results)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex diff --git a/proto/celestia/core/v1/tx/tx.proto b/proto/celestia/core/v1/tx/tx.proto index 1fa910e9e4..25acfd35ce 100644 --- a/proto/celestia/core/v1/tx/tx.proto +++ b/proto/celestia/core/v1/tx/tx.proto @@ -56,19 +56,19 @@ message TxStatusResponse { repeated string signers = 9; } -// TxStatusRequestBatch +// TxStatusBatchRequest is the request type for the batch TxStatus gRPC method. message TxStatusBatchRequest { // array of hex encoded tx hashes (each hash should be 64 characters long representing 32 bytes) repeated string tx_ids = 1; } -// TxStatusResponseWithHash is the tx hash with respective tx info -message TxStatusResponseWithHash { +// TxStatusResult represents a single transaction status result in a batch response. +message TxStatusResult { string tx_hash = 1; TxStatusResponse status = 2; } -// TxStatusBatchResponse +// TxStatusBatchResponse is a response type for batched TxStatus query. It contains an array of transaction status results. message TxStatusBatchResponse { - repeated TxStatusResponseWithHash responses = 1; + repeated TxStatusResult results = 1; } From b86d3ffd84dbcff131dd146d5ff3ca5d859f0fa4 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Tue, 2 Dec 2025 16:28:13 +0100 Subject: [PATCH 09/12] refactor: the bathc method on grpc server --- app/grpc/tx/server.go | 6 +- app/grpc/tx/tx.pb.go | 88 +++++++++++++++--------------- pkg/user/tx_client.go | 12 ---- proto/celestia/core/v1/tx/tx.proto | 2 +- 4 files changed, 48 insertions(+), 60 deletions(-) diff --git a/app/grpc/tx/server.go b/app/grpc/tx/server.go index 6a032bebaf..70511ff4aa 100644 --- a/app/grpc/tx/server.go +++ b/app/grpc/tx/server.go @@ -128,9 +128,9 @@ func (s *txServer) TxStatusBatch(ctx context.Context, req *TxStatusBatchRequest) return nil, err } - responses := make([]*TxStatusResponseWithHash, len(txStatusBatchResponses.Statuses)) + responses := make([]*TxStatusResult, len(txStatusBatchResponses.Statuses)) for i, status := range txStatusBatchResponses.Statuses { - responses[i] = &TxStatusResponseWithHash{ + responses[i] = &TxStatusResult{ TxHash: req.TxIds[i], Status: &TxStatusResponse{ @@ -148,6 +148,6 @@ func (s *txServer) TxStatusBatch(ctx context.Context, req *TxStatusBatchRequest) } return &TxStatusBatchResponse{ - Responses: responses, + Statuses: responses, }, nil } diff --git a/app/grpc/tx/tx.pb.go b/app/grpc/tx/tx.pb.go index 7a40eb8d7b..2a04581a4a 100644 --- a/app/grpc/tx/tx.pb.go +++ b/app/grpc/tx/tx.pb.go @@ -293,7 +293,7 @@ func (m *TxStatusResult) GetStatus() *TxStatusResponse { // TxStatusBatchResponse is a response type for batched TxStatus query. It contains an array of transaction status results. type TxStatusBatchResponse struct { - Results []*TxStatusResult `protobuf:"bytes,1,rep,name=results,proto3" json:"results,omitempty"` + Statuses []*TxStatusResult `protobuf:"bytes,1,rep,name=statuses,proto3" json:"statuses,omitempty"` } func (m *TxStatusBatchResponse) Reset() { *m = TxStatusBatchResponse{} } @@ -329,9 +329,9 @@ func (m *TxStatusBatchResponse) XXX_DiscardUnknown() { var xxx_messageInfo_TxStatusBatchResponse proto.InternalMessageInfo -func (m *TxStatusBatchResponse) GetResults() []*TxStatusResult { +func (m *TxStatusBatchResponse) GetStatuses() []*TxStatusResult { if m != nil { - return m.Results + return m.Statuses } return nil } @@ -347,41 +347,41 @@ func init() { func init() { proto.RegisterFile("celestia/core/v1/tx/tx.proto", fileDescriptor_7d8b070565b0dcb6) } var fileDescriptor_7d8b070565b0dcb6 = []byte{ - // 529 bytes of a gzipped FileDescriptorProto + // 530 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4f, 0x6f, 0xd3, 0x4e, 0x10, 0x8d, 0x93, 0xe6, 0xdf, 0x54, 0xe9, 0xef, 0xa7, 0x6d, 0x0b, 0x4b, 0x14, 0x4c, 0x64, 0x1a, - 0x14, 0x2a, 0xd5, 0x56, 0xc3, 0x0d, 0xa9, 0x97, 0x72, 0xa1, 0x57, 0x53, 0x40, 0xe2, 0x12, 0x6d, - 0xec, 0x95, 0x6d, 0x29, 0x78, 0x8d, 0x77, 0x5c, 0x56, 0x42, 0xbd, 0x70, 0xe1, 0x84, 0x84, 0xd4, - 0x2f, 0xc5, 0xb1, 0x12, 0x17, 0x8e, 0x28, 0xe1, 0x83, 0xa0, 0xdd, 0xc4, 0x69, 0xa9, 0x22, 0xda, - 0x83, 0x25, 0xcf, 0xbc, 0x99, 0x79, 0x6f, 0xde, 0xd8, 0xd0, 0x0b, 0xf8, 0x94, 0x4b, 0x4c, 0x98, - 0x17, 0x88, 0x9c, 0x7b, 0x67, 0x87, 0x1e, 0x2a, 0x0f, 0x95, 0x9b, 0xe5, 0x02, 0x05, 0xd9, 0x2e, - 0x51, 0x57, 0xa3, 0xee, 0xd9, 0xa1, 0x8b, 0xaa, 0xdb, 0x8b, 0x84, 0x88, 0xa6, 0xdc, 0x63, 0x59, - 0xe2, 0xb1, 0x34, 0x15, 0xc8, 0x30, 0x11, 0xa9, 0x5c, 0xb4, 0x38, 0x4f, 0xe0, 0xbf, 0x53, 0xf5, - 0x0a, 0x19, 0x16, 0xd2, 0xe7, 0x1f, 0x0a, 0x2e, 0x91, 0x6c, 0x43, 0x1d, 0xd5, 0x38, 0x09, 0xa9, - 0xd5, 0xb7, 0x86, 0x6d, 0x7f, 0x03, 0xd5, 0x49, 0xe8, 0x7c, 0xa9, 0xc2, 0xff, 0x57, 0x85, 0x32, - 0x13, 0xa9, 0xe4, 0xe4, 0x1e, 0x34, 0x62, 0x9e, 0x44, 0x31, 0x9a, 0xd2, 0x9a, 0xbf, 0x8c, 0xc8, - 0x0e, 0xd4, 0x93, 0x34, 0xe4, 0x8a, 0x56, 0xfb, 0xd6, 0xb0, 0xe3, 0x2f, 0x02, 0x32, 0x80, 0x2d, - 0xae, 0x78, 0x50, 0x68, 0xfa, 0x71, 0x20, 0x42, 0x4e, 0x6b, 0x06, 0xee, 0xac, 0xb2, 0x2f, 0x44, - 0xc8, 0x75, 0x33, 0xcf, 0x73, 0x91, 0xd3, 0x0d, 0x43, 0xbf, 0x08, 0x34, 0x95, 0x34, 0xe4, 0xb4, - 0x6e, 0xd2, 0xcb, 0x88, 0xf4, 0xa0, 0xad, 0x47, 0xc9, 0x8c, 0x05, 0x9c, 0x36, 0x0c, 0x74, 0x95, - 0x20, 0x0f, 0x01, 0x22, 0x26, 0xc7, 0x1f, 0x59, 0x8a, 0x3c, 0xa4, 0x4d, 0x23, 0xb2, 0x1d, 0x31, - 0xf9, 0xd6, 0x24, 0xc8, 0x03, 0x68, 0x69, 0xb8, 0x90, 0x3c, 0xa4, 0x2d, 0x03, 0x36, 0x23, 0x26, - 0x5f, 0x4b, 0x1e, 0x12, 0x0a, 0x4d, 0x99, 0x44, 0x29, 0xcf, 0x25, 0x6d, 0xf7, 0x6b, 0xc3, 0xb6, - 0x5f, 0x86, 0xce, 0x01, 0xec, 0x94, 0x46, 0x1c, 0x33, 0x0c, 0xe2, 0xd2, 0xb6, 0x5d, 0x68, 0x18, - 0xdb, 0x24, 0xb5, 0x4c, 0x43, 0x5d, 0xfb, 0x26, 0x9d, 0x18, 0xb6, 0xae, 0xf9, 0x56, 0x4c, 0x91, - 0xdc, 0x87, 0x26, 0xaa, 0x71, 0xcc, 0x64, 0xbc, 0x74, 0xb8, 0x81, 0xea, 0x25, 0x93, 0x31, 0x39, - 0x5a, 0xed, 0xa8, 0x7d, 0xdb, 0x1c, 0x0d, 0xdc, 0x35, 0xf7, 0x74, 0x6f, 0x5e, 0xa1, 0xb4, 0xc2, - 0x79, 0x03, 0xbb, 0x37, 0x84, 0x2d, 0xcf, 0x74, 0x04, 0xcd, 0xdc, 0x50, 0x2f, 0xa4, 0x6d, 0x8e, - 0x1e, 0xdf, 0x36, 0xb8, 0x98, 0xa2, 0x5f, 0xf6, 0x8c, 0x2e, 0xaa, 0x50, 0x3d, 0x55, 0xe4, 0x1c, - 0x5a, 0x65, 0x05, 0xd9, 0xbb, 0x65, 0x80, 0x71, 0xa4, 0x7b, 0x37, 0xfd, 0xce, 0xde, 0xe7, 0x1f, - 0xbf, 0x2f, 0xaa, 0x36, 0xe9, 0x79, 0xeb, 0x3e, 0xee, 0x4f, 0xc6, 0xd4, 0x73, 0xf2, 0xd5, 0x82, - 0xce, 0x5f, 0xeb, 0x91, 0xa7, 0xff, 0x1c, 0x7f, 0xfd, 0x36, 0xdd, 0xfd, 0xbb, 0x94, 0x2e, 0xe5, - 0x0c, 0x8c, 0x9c, 0x47, 0xcf, 0xad, 0x7d, 0xa7, 0xbb, 0x56, 0xd1, 0x44, 0x97, 0x1f, 0x9f, 0x7c, - 0x9f, 0xd9, 0xd6, 0xe5, 0xcc, 0xb6, 0x7e, 0xcd, 0x6c, 0xeb, 0xdb, 0xdc, 0xae, 0x5c, 0xce, 0xed, - 0xca, 0xcf, 0xb9, 0x5d, 0x79, 0xe7, 0x45, 0x09, 0xc6, 0xc5, 0xc4, 0x0d, 0xc4, 0xfb, 0x55, 0xbf, - 0xc8, 0xa3, 0xd5, 0xfb, 0x01, 0xcb, 0x32, 0x4f, 0x3f, 0x51, 0x9e, 0x05, 0x1e, 0xaa, 0x49, 0xc3, - 0xfc, 0x8a, 0xcf, 0xfe, 0x04, 0x00, 0x00, 0xff, 0xff, 0xf8, 0x96, 0xbd, 0xf7, 0xdd, 0x03, 0x00, - 0x00, + 0x14, 0x2a, 0xd5, 0x56, 0xc3, 0x0d, 0x09, 0x21, 0x95, 0x0b, 0xbd, 0x9a, 0x22, 0x10, 0x97, 0x68, + 0x63, 0xaf, 0x6c, 0x4b, 0xc1, 0x6b, 0xbc, 0xe3, 0xb2, 0x12, 0xea, 0x85, 0x0b, 0x27, 0x24, 0xa4, + 0x7e, 0x29, 0x8e, 0x95, 0xb8, 0x70, 0x44, 0x09, 0x1f, 0x04, 0xed, 0x26, 0x4e, 0x4b, 0x15, 0xd1, + 0x1e, 0x2c, 0x79, 0xe6, 0xbd, 0x99, 0x79, 0xf3, 0xc6, 0x86, 0x5e, 0xc0, 0xa7, 0x5c, 0x62, 0xc2, + 0xbc, 0x40, 0xe4, 0xdc, 0x3b, 0x3d, 0xf4, 0x50, 0x79, 0xa8, 0xdc, 0x2c, 0x17, 0x28, 0xc8, 0x76, + 0x89, 0xba, 0x1a, 0x75, 0x4f, 0x0f, 0x5d, 0x54, 0xdd, 0x5e, 0x24, 0x44, 0x34, 0xe5, 0x1e, 0xcb, + 0x12, 0x8f, 0xa5, 0xa9, 0x40, 0x86, 0x89, 0x48, 0xe5, 0xa2, 0xc4, 0x79, 0x04, 0xff, 0x9d, 0xa8, + 0x57, 0xc8, 0xb0, 0x90, 0x3e, 0xff, 0x50, 0x70, 0x89, 0x64, 0x1b, 0xea, 0xa8, 0xc6, 0x49, 0x48, + 0xad, 0xbe, 0x35, 0x6c, 0xfb, 0x1b, 0xa8, 0x8e, 0x43, 0xe7, 0x4b, 0x15, 0xfe, 0xbf, 0x24, 0xca, + 0x4c, 0xa4, 0x92, 0x93, 0x3b, 0xd0, 0x88, 0x79, 0x12, 0xc5, 0x68, 0xa8, 0x35, 0x7f, 0x19, 0x91, + 0x1d, 0xa8, 0x27, 0x69, 0xc8, 0x15, 0xad, 0xf6, 0xad, 0x61, 0xc7, 0x5f, 0x04, 0x64, 0x00, 0x5b, + 0x5c, 0xf1, 0xa0, 0xd0, 0xe3, 0xc7, 0x81, 0x08, 0x39, 0xad, 0x19, 0xb8, 0xb3, 0xca, 0xbe, 0x10, + 0x21, 0xd7, 0xc5, 0x3c, 0xcf, 0x45, 0x4e, 0x37, 0xcc, 0xf8, 0x45, 0xa0, 0x47, 0x49, 0x33, 0x9c, + 0xd6, 0x4d, 0x7a, 0x19, 0x91, 0x1e, 0xb4, 0x75, 0x2b, 0x99, 0xb1, 0x80, 0xd3, 0x86, 0x81, 0x2e, + 0x13, 0xe4, 0x3e, 0x40, 0xc4, 0xe4, 0xf8, 0x23, 0x4b, 0x91, 0x87, 0xb4, 0x69, 0x44, 0xb6, 0x23, + 0x26, 0xdf, 0x98, 0x04, 0xb9, 0x07, 0x2d, 0x0d, 0x17, 0x92, 0x87, 0xb4, 0x65, 0xc0, 0x66, 0xc4, + 0xe4, 0x6b, 0xc9, 0x43, 0x42, 0xa1, 0x29, 0x93, 0x28, 0xe5, 0xb9, 0xa4, 0xed, 0x7e, 0x6d, 0xd8, + 0xf6, 0xcb, 0xd0, 0x39, 0x80, 0x9d, 0xd2, 0x88, 0x23, 0x86, 0x41, 0x5c, 0xda, 0xb6, 0x0b, 0x0d, + 0x63, 0x9b, 0xa4, 0x96, 0x29, 0xa8, 0x6b, 0xdf, 0xa4, 0x13, 0xc3, 0xd6, 0x15, 0xdf, 0x8a, 0x29, + 0x92, 0xbb, 0xd0, 0x44, 0x35, 0x8e, 0x99, 0x8c, 0x97, 0x0e, 0x37, 0x50, 0xbd, 0x64, 0x32, 0x26, + 0xcf, 0x56, 0x3b, 0x6a, 0xdf, 0x36, 0x47, 0x03, 0x77, 0xcd, 0x3d, 0xdd, 0xeb, 0x57, 0x28, 0xad, + 0x70, 0xde, 0xc2, 0xee, 0x35, 0x61, 0xcb, 0x33, 0x3d, 0x87, 0xd6, 0x82, 0xc2, 0x17, 0xda, 0x36, + 0x47, 0x0f, 0x6f, 0xea, 0x5c, 0x4c, 0xd1, 0x5f, 0x15, 0x8d, 0xce, 0xab, 0x50, 0x3d, 0x51, 0xe4, + 0x0c, 0x5a, 0x25, 0x85, 0xec, 0xdd, 0xd0, 0xc1, 0x78, 0xd2, 0xbd, 0xdd, 0x06, 0xce, 0xde, 0xe7, + 0x1f, 0xbf, 0xcf, 0xab, 0x36, 0xe9, 0x79, 0xeb, 0x3e, 0xef, 0x4f, 0xc6, 0xd6, 0x33, 0xf2, 0xd5, + 0x82, 0xce, 0x5f, 0x0b, 0x92, 0xc7, 0xff, 0x6c, 0x7f, 0xf5, 0x3a, 0xdd, 0xfd, 0xdb, 0x50, 0x97, + 0x72, 0x06, 0x46, 0xce, 0x83, 0xa7, 0xd6, 0xbe, 0xd3, 0x5d, 0xab, 0x68, 0xa2, 0xe9, 0x47, 0xc7, + 0xdf, 0x67, 0xb6, 0x75, 0x31, 0xb3, 0xad, 0x5f, 0x33, 0xdb, 0xfa, 0x36, 0xb7, 0x2b, 0x17, 0x73, + 0xbb, 0xf2, 0x73, 0x6e, 0x57, 0xde, 0x79, 0x51, 0x82, 0x71, 0x31, 0x71, 0x03, 0xf1, 0x7e, 0x55, + 0x2f, 0xf2, 0x68, 0xf5, 0x7e, 0xc0, 0xb2, 0xcc, 0xd3, 0x4f, 0x94, 0x67, 0x81, 0x87, 0x6a, 0xd2, + 0x30, 0x3f, 0xe3, 0x93, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xf9, 0xc7, 0x7f, 0xa8, 0xdf, 0x03, + 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -717,10 +717,10 @@ func (m *TxStatusBatchResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Results) > 0 { - for iNdEx := len(m.Results) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Statuses) > 0 { + for iNdEx := len(m.Statuses) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Results[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.Statuses[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -838,8 +838,8 @@ func (m *TxStatusBatchResponse) Size() (n int) { } var l int _ = l - if len(m.Results) > 0 { - for _, e := range m.Results { + if len(m.Statuses) > 0 { + for _, e := range m.Statuses { l = e.Size() n += 1 + l + sovTx(uint64(l)) } @@ -1439,7 +1439,7 @@ func (m *TxStatusBatchResponse) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Results", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Statuses", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1466,8 +1466,8 @@ func (m *TxStatusBatchResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Results = append(m.Results, &TxStatusResult{}) - if err := m.Results[len(m.Results)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Statuses = append(m.Statuses, &TxStatusResult{}) + if err := m.Statuses[len(m.Statuses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex diff --git a/pkg/user/tx_client.go b/pkg/user/tx_client.go index 9c19fac852..577529eb30 100644 --- a/pkg/user/tx_client.go +++ b/pkg/user/tx_client.go @@ -757,18 +757,6 @@ func (client *TxClient) ConfirmTx(ctx context.Context, txHash string) (*TxRespon defer pollTicker.Stop() var evictionPollTimeStart *time.Time - txStatusBatchRequest := &tx.TxStatusBatchRequest{ - TxIds: []string{txHash, txHash, txHash, txHash}, - } - txStatusBatchResponse, err := txClient.TxStatusBatch(ctx, txStatusBatchRequest) - if err != nil { - return nil, err - } - - for _, resp := range txStatusBatchResponse.Responses { - fmt.Println("resp from batch: ", resp.TxHash, resp.Status.Status) - } - for { span.AddEvent("txclient/ConfirmTx: polling for TxStatus") resp, err := txClient.TxStatus(ctx, &tx.TxStatusRequest{TxId: txHash}) diff --git a/proto/celestia/core/v1/tx/tx.proto b/proto/celestia/core/v1/tx/tx.proto index 25acfd35ce..f0b08d0073 100644 --- a/proto/celestia/core/v1/tx/tx.proto +++ b/proto/celestia/core/v1/tx/tx.proto @@ -70,5 +70,5 @@ message TxStatusResult { // TxStatusBatchResponse is a response type for batched TxStatus query. It contains an array of transaction status results. message TxStatusBatchResponse { - repeated TxStatusResult results = 1; + repeated TxStatusResult statuses = 1; } From b3a29c64b686670ba7931d65ee2486c05b218686 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Tue, 2 Dec 2025 16:53:33 +0100 Subject: [PATCH 10/12] refacotor: remove rejections --- pkg/user/v2/sequential_queue.go | 168 +------------------------------- 1 file changed, 5 insertions(+), 163 deletions(-) diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index 9d6b0c6836..c3e166d8f9 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -70,7 +70,6 @@ type queuedTx struct { txBytes []byte // Set after broadcast, used for eviction resubmission sequence uint64 // Set after broadcast submittedAt time.Time // Set after broadcast - shouldResign bool // Set after broadcast isResubmitting bool // True if transaction is currently being resubmitted (prevents duplicates) } @@ -92,8 +91,7 @@ func newSequentialQueue(client *TxClient, accountName string, pollTime time.Dura ctx: ctx, cancel: cancel, queue: make([]*queuedTx, 0, defaultSequentialQueueSize), - ResignChan: make(chan *queuedTx, 50), // Buffered channel for resign requests - ResubmitChan: make(chan *queuedTx, 200), // Buffered channel for resubmit requests (large to prevent blocking) + ResubmitChan: make(chan *queuedTx, 10), // Buffered channel for resubmit requests (large to prevent blocking) lastMetricsLog: now, metricsStartTime: now, } @@ -212,7 +210,6 @@ func (q *sequentialQueue) processNextTx() { } fmt.Println("status for this expected hash: ", statusResp.Status) fmt.Println("status log: ", statusResp.Error) - fmt.Println("shouldResign: ", txx.shouldResign) return } @@ -271,8 +268,7 @@ func (q *sequentialQueue) coordinate() { case <-q.ctx.Done(): return case <-q.ResignChan: - fmt.Println("Resigning rejected tx") - q.ResignRejected() + // TODO: decide if we want to do anything during rejections case qTx := <-q.ResubmitChan: q.ResubmitEvicted(qTx) case <-ticker.C: @@ -281,56 +277,6 @@ func (q *sequentialQueue) coordinate() { } } -// ResignRejected resigns a rejected transaction -func (q *sequentialQueue) ResignRejected() { - startTime := time.Now() - q.mu.RLock() - var txsToResign []*queuedTx - for _, qTx := range q.queue { - if qTx.shouldResign { - fmt.Printf("Adding rejected tx to resign list with hash %s and sequence %d\n", qTx.txHash[:16], qTx.sequence) - txsToResign = append(txsToResign, qTx) - } - } - q.mu.RUnlock() - - for _, qTx := range txsToResign { - if qTx.shouldResign { - // resign the tx - resignStart := time.Now() - resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( - q.ctx, - q.accountName, - qTx.blobs, - qTx.options..., - ) - if err != nil { - fmt.Printf("rejected and failed to resign with hash %s", qTx.txHash[:16]) - // send error and remove from queue - select { - case qTx.resultsC <- SequentialSubmissionResult{ - Error: fmt.Errorf("rejected and failed to resign: %w", err), - }: - case <-q.ctx.Done(): - } - q.removeFromQueue(qTx) - return - } - q.mu.Lock() - sequence := q.client.Signer().Account(q.accountName).Sequence() - qTx.txHash = resp.TxHash - qTx.txBytes = txBytes - qTx.sequence = sequence - 1 // sequence is incremented after successful submission - qTx.shouldResign = false - q.resignCount++ - q.mu.Unlock() - resignDuration := time.Since(resignStart) - fmt.Printf("Resigned and submitted tx successfully with sequence %d: %s (took %v)\n", sequence, resp.TxHash, resignDuration) - } - } - fmt.Printf("[TIMING] Total ResignRejected took %v\n", time.Since(startTime)) -} - // TODO: come back to this and see if it makes sense // func (q *sequentialQueue) setTxInfo(qTx *queuedTx, resp *sdktypes.TxResponse, txBytes []byte, sequence uint64) { // q.mu.Lock() @@ -355,49 +301,6 @@ func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { resubmitDuration := time.Since(resubmitStart) fmt.Printf("[TIMING] Resubmit network call took %v\n", resubmitDuration) if err != nil || resubmitResp.Code != 0 { - // Check if this is a sequence mismatch - // if IsSequenceMismatchError(err) { - // // Sequence mismatch means blockchain is at earlier sequence than this tx - // // All txs from blockchain sequence onwards are stale - remove them all at once - // expectedSeq := parseExpectedSequence(err.Error()) - // fmt.Printf("Sequence mismatch: blockchain at %d but tx at %d. Removing all stale txs >= %d\n", - // expectedSeq, qTx.sequence, expectedSeq) - - // // Collect all transactions with sequence >= expectedSeq - // q.mu.RLock() - // var staleTxs []*queuedTx - // for _, tx := range q.queue { - // if tx.sequence >= expectedSeq { - // staleTxs = append(staleTxs, tx) - // } - // } - // q.mu.RUnlock() - - // // check the first tx to see if it was evicted then we can be sure that all txs are evicted - // TxClient := tx.NewTxClient(q.client.GetGRPCConnection()) - // statusResp, err := TxClient.TxStatus(q.ctx, &tx.TxStatusRequest{TxId: qTx.txHash}) - // if err != nil { - // fmt.Printf("Failed to check status of expected sequence tx: %v\n", err) - // // Reset flag and return - let next poll cycle handle it - // q.mu.Lock() - // qTx.isResubmitting = false - // q.mu.Unlock() - // return - // } - // // lets just log for now - // fmt.Printf("TX STATUS OF EXPECTED SEQUENCE %d: %s\n", expectedSeq, statusResp.Status) - - // if statusResp.Status == core.TxStatusEvicted { - // // All stale txs are evicted. Reset current tx flag and return. - // // Next poll cycle will scan from beginning and handle all evicted txs properly. - // fmt.Printf("Confirmed: all txs from seq %d onwards are evicted. Resetting flag for next poll cycle.\n", expectedSeq) - // q.mu.Lock() - // qTx.isResubmitting = false - // q.mu.Unlock() - // return - // } - // } - select { case qTx.resultsC <- SequentialSubmissionResult{ Error: fmt.Errorf("evicted and failed to resubmit with hash %s: %w", qTx.txHash[:16], err), @@ -426,7 +329,7 @@ func (q *sequentialQueue) checkBroadcastTransactions() { scanStart := time.Now() q.mu.RLock() // Collect all broadcast transactions (those with non-empty txHash) - var broadcastTxs []*queuedTx // TODO: cap the size + var broadcastTxs []*queuedTx for _, tx := range q.queue { if tx.txHash != "" { broadcastTxs = append(broadcastTxs, tx) @@ -523,7 +426,7 @@ func (q *sequentialQueue) checkBroadcastTransactions() { } func (q *sequentialQueue) handleEvicted(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { - // TODO: move evicted logic here + } // handleCommitted processes a confirmed transaction @@ -577,51 +480,9 @@ func (q *sequentialQueue) setLastConfirmedSeq(seq uint64) { // handleRejected processes a rejected transaction func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { - fmt.Printf("Handling rejected tx:%s with code %d", qTx.txHash[:16], statusResp.ExecutionCode) - - q.mu.RLock() - shouldResign := qTx.shouldResign - q.mu.RUnlock() - if shouldResign { - fmt.Printf("Tx %s is already being resigned - skipping\n", qTx.txHash[:16]) - return - } - - isNonceMismatch := isSequenceMismatchRejection(statusResp.Error) - - if !isNonceMismatch { - // Non-nonce error - remove from queue and return error to user - // check if previous tx was confirmed or pending. If so, roll back sequence to the previous tx sequence - if q.isPreviousTxCommittedOrPending(qTx.sequence, txClient) { - q.mu.Lock() - fmt.Println("LAST CONFIRMED SEQUENCE: ", q.lastConfirmedSeq) - fmt.Println("SEQUENCE TO ROLL BACK TO: ", qTx.sequence) - q.client.Signer().SetSequence(q.accountName, qTx.sequence) - q.mu.Unlock() - fmt.Printf("Rolled back signer sequence to %d (previous tx)\n", qTx.sequence) - } - select { - case <-q.ctx.Done(): - case qTx.resultsC <- SequentialSubmissionResult{ - Error: fmt.Errorf("tx rejected: %s", statusResp.Error), - }: - } - q.removeFromQueue(qTx) - return - } - // Nonce/sequence mismatch - scan entire queue from beginning to find all rejected txs fmt.Printf("Detected rejected tx with sequence %d - scanning queue for all rejections\n", qTx.sequence) - // Check if already being resigned - q.mu.RLock() - alreadyResigning := qTx.shouldResign - q.mu.RUnlock() - if alreadyResigning { - fmt.Printf("Tx %s is already being resigned - skipping\n", qTx.txHash[:16]) - return - } - // Step 2: Collect all broadcast transactions to check (including those already marked for resignation) q.mu.RLock() var allBroadcastTxs []*queuedTx @@ -647,12 +508,7 @@ func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusR // Step 3a: Find the earliest rejected tx and roll back sequence if needed if len(rejectedTxs) > 0 { // Find the earliest rejected tx (lowest sequence) - var earliestRejected *queuedTx - for _, rejectedTx := range rejectedTxs { - if earliestRejected == nil || rejectedTx.sequence < earliestRejected.sequence { - earliestRejected = rejectedTx - } - } + earliestRejected := rejectedTxs[0] // Check if the transaction before the earliest rejected one was confirmed or pending fmt.Println("EARLIEST REJECTED TX SEQUENCE: ", earliestRejected.sequence) @@ -666,19 +522,6 @@ func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusR fmt.Printf("Rolled back signer sequence to %d (earliest rejected tx)\n", earliestRejected.sequence) } } - - for _, rejectedTx := range rejectedTxs { - q.mu.Lock() - if !rejectedTx.shouldResign { - rejectedTx.shouldResign = true - q.mu.Unlock() - fmt.Printf("Sending rejected tx (seq %d) to resign channel\n", rejectedTx.sequence) - q.ResignChan <- rejectedTx - } else { - q.mu.Unlock() - fmt.Printf("Skipping rejected tx (seq %d) - already marked for resign\n", rejectedTx.sequence) - } - } } // removeFromQueue removes a transaction from the queue @@ -740,7 +583,6 @@ func (q *sequentialQueue) isPreviousTxCommittedOrPending(seq uint64, txClient tx return true } fmt.Println("PREVIOUS TX STATUS Seq: ", prevSeq, " RESPONSE: ", statusResp.Status, "LOG: ", statusResp.Error) - fmt.Println("PREVIOUS TX SHOULD RESIGN: ", prevTx.shouldResign) // Return true if COMMITTED or PENDING return statusResp.Status == core.TxStatusCommitted || statusResp.Status == core.TxStatusPending From bf91cc71195b39bb76507c80932fc571160d4932 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Tue, 2 Dec 2025 17:00:27 +0100 Subject: [PATCH 11/12] style: fix lint --- app/grpc/tx/server.go | 1 - 1 file changed, 1 deletion(-) diff --git a/app/grpc/tx/server.go b/app/grpc/tx/server.go index 70511ff4aa..4e319468b5 100644 --- a/app/grpc/tx/server.go +++ b/app/grpc/tx/server.go @@ -133,7 +133,6 @@ func (s *txServer) TxStatusBatch(ctx context.Context, req *TxStatusBatchRequest) responses[i] = &TxStatusResult{ TxHash: req.TxIds[i], Status: &TxStatusResponse{ - Height: status.Result.Height, Index: status.Result.Index, ExecutionCode: status.Result.ExecutionCode, From 17b54dc2702356c1333e66b76c79881bfae4c192 Mon Sep 17 00:00:00 2001 From: Nina Barbakadze Date: Mon, 8 Dec 2025 15:01:19 +0100 Subject: [PATCH 12/12] refactor: cap by size and remove interval --- latency_results.csv | 45 ---------- pkg/user/v2/sequential_queue.go | 149 +++++++++++++++++++++++++++----- 2 files changed, 129 insertions(+), 65 deletions(-) diff --git a/latency_results.csv b/latency_results.csv index e388be66d1..f6d984805e 100644 --- a/latency_results.csv +++ b/latency_results.csv @@ -1,46 +1 @@ Submit Time,Commit Time,Latency (ms),Tx Hash,Height,Code,Failed,Error -2025-12-03T01:07:41.654671+01:00,2025-12-03T01:08:57.654513+01:00,75998.00,F432BDD5AE8544895E5CA17176088B9AC1EBB24DD391F8B3D231A32C2A03BA6E,9061414,0,false, -2025-12-03T01:07:42.656887+01:00,2025-12-03T01:09:08.653531+01:00,85995.00,9484A9AF965B80B6EED461D00AA6918E2414DB70360BA246F15CA7BDAC3AF75D,9061416,0,false, -2025-12-03T01:07:43.656047+01:00,2025-12-03T01:09:24.655047+01:00,100997.00,729BC16200FD40EAB12F2B076E5B68110A19102279079090A08846C3047E4A04,9061419,0,false, -2025-12-03T01:07:44.655946+01:00,2025-12-03T01:09:30.654036+01:00,105996.00,7D607D6367B01D61006A2E024BAC6000851F4C169603110B876155B4C345E70E,9061420,0,false, -2025-12-03T01:07:45.654955+01:00,2025-12-03T01:09:41.653549+01:00,115997.00,C112660C4A70927BCB397DD0064023DD24151D052ED2F1459C8EC2FA533582DA,9061422,0,false, -2025-12-03T01:07:46.65758+01:00,2025-12-03T01:10:07.653786+01:00,140994.00,4538B00E3DF0D143DEA7E5420BFA1F86DDE87B514F2301CE7E6E8E3884698AA1,9061426,0,false, -2025-12-03T01:07:47.656339+01:00,2025-12-03T01:10:26.658015+01:00,158999.00,B6CA9BB39FC13D7EC682BDEA443AFCBDD2808AD6D2C6A0B6386F8539CAA4ED30,9061429,0,false, -2025-12-03T01:07:48.654735+01:00,2025-12-03T01:10:39.755044+01:00,171098.00,5EE0427A3B8CEE9A4B62C26A45F646329B9F6EE1235C20867B93F305D0FF0804,9061431,0,false, -2025-12-03T01:07:49.656668+01:00,2025-12-03T01:11:09.654316+01:00,199995.00,CFF16124F7248ACD29B543729E86B859C38C1BC79C5C0052EDA8F8DB45E422DE,9061435,0,false, -2025-12-03T01:07:50.656756+01:00,2025-12-03T01:11:19.653711+01:00,208994.00,D990992F8202877FE4B9D18C6BB4A73AFA12FE195203D4D9C65E2BBFD041C8CA,9061437,0,false, -2025-12-03T01:07:51.656638+01:00,2025-12-03T01:11:41.656695+01:00,229997.00,8BE16DC58B4BBBC40490A948239B282C9E179B1355440ECD0DF6712FF5E2516D,9061441,0,false, -2025-12-03T01:07:52.65868+01:00,2025-12-03T01:11:52.655753+01:00,239994.00,020D748ABF52004187B74BE29854C3D34C4EFDF8A71B773537AA498363938D8B,9061443,0,false, -2025-12-03T01:07:53.654634+01:00,2025-12-03T01:12:11.656702+01:00,257998.00,E03A1433EC8168CF8025F2BC7DA8CBEAC3750E2A027E6F98101C7DA2F4AEF01E,9061446,0,false, -2025-12-03T01:07:54.656361+01:00,2025-12-03T01:12:23.656244+01:00,268996.00,B9FB7A1B2C00BC48ECF08D4E411F0A49EEBACAC1AFAF9F52CE49E441E2B89C0F,9061448,0,false, -2025-12-03T01:07:55.656055+01:00,2025-12-03T01:12:40.679304+01:00,285019.00,529479B7B275E044B207E2C98BF619371B26DB3BC907D4CCB6B3C8C27B42284A,9061451,0,false, -2025-12-03T01:07:56.656888+01:00,2025-12-03T01:12:58.656871+01:00,301996.00,A51996EC5C10058687452660F8C71521272B4B3011276C9FECE2B4E1DE08F2D1,9061454,0,false, -2025-12-03T01:07:57.656694+01:00,2025-12-03T01:13:18.657271+01:00,320996.00,6842FE444064B2AB6E01B2D8234A1EC35657ADEA56424A6BE0572E0987764721,9061457,0,false, -2025-12-03T01:07:58.657432+01:00,2025-12-03T01:13:35.665767+01:00,337004.00,234AC87D769950CEABCA4A05A82D897C08C095F94245B3AB7D1EB0405212DAE9,9061460,0,false, -2025-12-03T01:07:59.657741+01:00,2025-12-03T01:13:59.672266+01:00,360009.00,E60A5FE8794F7B2C5CAE93DC39F365D550668E229714893E07D4C1EC346EBAEE,9061464,0,false, -2025-12-03T01:08:00.658416+01:00,2025-12-03T01:14:16.656789+01:00,375993.00,DDC93560F40D5A89C1CD64CE7E57366792A12A00B6B6B5CBADAA4734DA0CFF1D,9061467,0,false, -2025-12-03T01:08:57.656545+01:00,2025-12-03T01:14:32.657257+01:00,334996.00,F0CB66CED0ED054374A7D7A3EA8A4AD7542FB2A6A61480ACCEA9E5FC18408741,9061470,0,false, -2025-12-03T01:09:08.656449+01:00,2025-12-03T01:14:38.669547+01:00,330008.00,5D6470FB0059BCFC45A1FE12BB869E0D6DD9E94F86E93E35D03EC29174C57716,9061471,0,false, -2025-12-03T01:09:24.657495+01:00,2025-12-03T01:14:56.656822+01:00,331995.00,DEEEEE77615E3D7DD7F569A635A92957780C0F809C2F0A7046310E48D3FA6474,9061474,0,false, -2025-12-03T01:09:30.656134+01:00,2025-12-03T01:15:08.657778+01:00,337997.00,8CA86E1322C2DBAB7B360125541855DD93997C3372226CA215B77353CADB97A5,9061476,0,false, -2025-12-03T01:09:41.657271+01:00,2025-12-03T01:15:31.658924+01:00,349997.00,077CA470CAAD24601493CFEA80BCED5ED634806AF799CC3FDC95185BFD6CDCEE,9061480,0,false, -2025-12-03T01:10:07.65804+01:00,2025-12-03T01:15:44.659447+01:00,336997.00,06E2A5CDA33F8F51BE509E8F080F877FA246409809A8AC89F2B46FB992E85803,9061482,0,false, -2025-12-03T01:10:25.65697+01:00,2025-12-03T01:15:54.69282+01:00,329031.00,AA4EEBB1F6ECBBE46B319E804DCBDFAB8D9E428B73509C4E19D83C31053E11FA,9061484,0,false, -2025-12-03T01:08:09.657133+01:00,2025-12-03T01:16:12.660289+01:00,482996.00,4E0301E21767D14EF19ABEBF20097CF9E2BC524854793F430387F788B0D98FA9,9061487,0,false, -2025-12-03T01:11:09.658462+01:00,2025-12-03T01:16:23.658332+01:00,313995.00,A56BAB0379286F5905BB8F1FA3BA20C07C216DC7511493D476DCF271CD4E143E,9061489,0,false, -2025-12-03T01:11:19.659267+01:00,2025-12-03T01:16:47.659384+01:00,327995.00,1A0134893B3F25E1BF811EC3153C5E6672747EAF0191F1AEAB220C84AF31F434,9061493,0,false, -2025-12-03T01:11:41.659337+01:00,2025-12-03T01:17:05.659372+01:00,323995.00,905BF9F9C362B1E47C3EB318BA2BA4151B38CD99AF12586A6BCDF03CA46D8FEA,9061496,0,false, -2025-12-03T01:11:52.659873+01:00,2025-12-03T01:17:22.65984+01:00,329995.00,B00FD01313067608206DB17AF5639726275D091217A8EC8183AB50A83D8E1E21,9061499,0,false, -2025-12-03T01:12:11.659042+01:00,2025-12-03T01:17:39.662044+01:00,327998.00,510CFC0B51C02E1CBE1CB63D69240A47E388459A8EDDFEC3BCC2418D9DE67CBE,9061502,0,false, -2025-12-03T01:12:23.658987+01:00,2025-12-03T01:17:45.663438+01:00,322000.00,04E881925581217464155FC1CED569162C3935EED8FC4E0956D26BCBECE56CAB,9061503,0,false, -2025-12-03T01:09:18.658342+01:00,2025-12-03T01:18:09.65982+01:00,530994.00,C2094A20C21CF62E8E7319B4918973DCE2393D4486B696CC7529F47888252BBC,9061507,0,false, -2025-12-03T01:12:58.659984+01:00,2025-12-03T01:18:20.66541+01:00,322001.00,6F958B5B6A9C0028C52F10C0C29D2982FF1FFF1F75A7C84782B12B475B24AAF1,9061509,0,false, -2025-12-03T01:13:16.659591+01:00,2025-12-03T01:18:31.663011+01:00,314999.00,553D487F59888A98F5F5C94B1D416F24868E613024E53A48240F33F8902F23C4,9061511,0,false, -2025-12-03T01:10:47.658755+01:00,2025-12-03T01:18:48.660913+01:00,480996.00,09969EF82FE1755C7A0D167EF8075F72FDF8B6DBD845A20A897C423A02F074FD,9061513,0,false, -2025-12-03T01:13:56.664093+01:00,2025-12-03T01:19:03.662422+01:00,306994.00,BAA610E3C5239F54B7A2774C5F5AF5027FF2AFEB0B59DB743B4E4D809C5239CD,9061516,0,false, -2025-12-03T01:09:50.658529+01:00,2025-12-03T01:19:24.661864+01:00,573996.00,FAD3EC738E6F02D328A12825F43B6F92BE3E7E571C4A999E008AFD905213FA3C,9061519,0,false, -2025-12-03T01:14:32.661543+01:00,2025-12-03T01:19:35.660445+01:00,302995.00,B9E3168B2C25277931151298505593A0388F8AE74DD6BC24BFA61B681B7E4F68,9061521,0,false, -2025-12-03T01:14:35.661106+01:00,2025-12-03T01:19:46.666372+01:00,311001.00,FCFE8B9E7A25491A51B1D5251F86E3B0E0878F99D68793FD30005CA61A5ED6E6,9061523,0,false, -2025-12-03T01:14:56.660103+01:00,2025-12-03T01:19:58.662854+01:00,301998.00,DD1EA23CC7098E29C2A95889336FC67C572B4A966771756FE825FFC125E473EB,9061525,0,false, -2025-12-03T01:15:08.660255+01:00,2025-12-03T01:20:15.661918+01:00,306997.00,8C9788CBDF1B6B4508ED8046D6AA1B0329A7216E07A5DF07685348A8D9357C1B,9061528,0,false, -2025-12-03T01:14:50.659855+01:00,2025-12-03T01:20:26.66142+01:00,335997.00,82D935B0E46AB5DCAB4C34E1092756E97BDEC68D097F8CAD4E745277DEFD3C27,9061530,0,false, diff --git a/pkg/user/v2/sequential_queue.go b/pkg/user/v2/sequential_queue.go index da53e59ed8..5cd46b88bc 100644 --- a/pkg/user/v2/sequential_queue.go +++ b/pkg/user/v2/sequential_queue.go @@ -6,6 +6,7 @@ import ( "regexp" "strconv" "sync" + "sync/atomic" "time" "github.com/celestiaorg/celestia-app/v6/app/grpc/tx" @@ -41,10 +42,12 @@ type sequentialQueue struct { pollTime time.Duration // Single unified queue - transactions stay here until confirmed - mu sync.RWMutex - queue []*queuedTx // All transactions from submission to confirmation - ResignChan chan *queuedTx // Channel for all rejected transactions that need to be resigned - ResubmitChan chan *queuedTx // Channel for all evicted transactions that need to be resubmitted + mu sync.RWMutex + queue []*queuedTx // All transactions from submission to confirmation + queueMemoryBytes uint64 // Total memory used by blobs in queue (in bytes) + maxMemoryBytes uint64 // Maximum memory allowed for queue (in bytes) + ResignChan chan *queuedTx // Channel for all rejected transactions that need to be resigned + ResubmitChan chan *queuedTx // Channel for all evicted transactions that need to be resubmitted // Track last confirmed sequence for rollback logic lastConfirmedSeq uint64 @@ -52,6 +55,8 @@ type sequentialQueue struct { // Track last rejected sequence for rollback logic lastRejectedSeq uint64 + isRecovering atomic.Bool + // Submission tracking metrics newBroadcastCount uint64 // Count of new transaction broadcasts resubmitCount uint64 // Count of resubmissions (evicted txs) @@ -76,8 +81,9 @@ type queuedTx struct { } const ( - // defaultSequentialQueueSize is the maximum number of pending transactions allowed in the queue at once - defaultSequentialQueueSize = 20 + // defaultMaxQueueMemoryMB is the maximum memory (in MB) allowed for the queue to prevent OOM + // This limits the total size of blob data held in memory at once + defaultMaxQueueMemoryMB = 200 // 100MB default ) func newSequentialQueue(client *TxClient, accountName string, pollTime time.Duration) *sequentialQueue { @@ -93,8 +99,10 @@ func newSequentialQueue(client *TxClient, accountName string, pollTime time.Dura pollTime: pollTime, ctx: ctx, cancel: cancel, - queue: make([]*queuedTx, 0, defaultSequentialQueueSize), - ResubmitChan: make(chan *queuedTx, 10), // Buffered channel for resubmit requests (large to prevent blocking) + queue: make([]*queuedTx, 0), + queueMemoryBytes: 0, + maxMemoryBytes: defaultMaxQueueMemoryMB * 1024 * 1024, // Convert MB to bytes + ResubmitChan: make(chan *queuedTx, 10), lastMetricsLog: now, metricsStartTime: now, } @@ -114,27 +122,57 @@ func (q *sequentialQueue) start() { }() } +func (q *sequentialQueue) setRecovering(v bool) { + // q.mu.Lock() + q.isRecovering.Store(v) + // q.mu.Unlock() +} + +func (q *sequentialQueue) getRecovering() bool { + // q.mu.RLock() + return q.isRecovering.Load() + // q.mu.RUnlock() +} + // submitJob adds a new transaction to the queue -// It enforces a maximum of defaultSequentialQueueSize (20) pending transactions at a time +// It enforces memory limits to prevent OOM by blocking until sufficient memory is available func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { - // Wait for space in queue (backpressure) - ensures we never have more than 20 pending txs + // Calculate memory size of this transaction's blobs + blobsMemory := calculateBlobsMemory(job.Blobs) + + // Wait for memory space in queue (backpressure) - prevents memory exhaustion for { q.mu.Lock() - if len(q.queue) < defaultSequentialQueueSize { - // Space available - add transaction + if q.queueMemoryBytes+blobsMemory <= q.maxMemoryBytes { + // Sufficient memory available - add transaction qTx := &queuedTx{ blobs: job.Blobs, options: job.Options, resultsC: job.ResultsC, } q.queue = append(q.queue, qTx) + q.queueMemoryBytes += blobsMemory + + currentMemMB := float64(q.queueMemoryBytes) / (1024 * 1024) + maxMemMB := float64(q.maxMemoryBytes) / (1024 * 1024) q.mu.Unlock() + + // Log when approaching capacity (>80%) + if q.queueMemoryBytes > (q.maxMemoryBytes * 80 / 100) { + fmt.Printf("[MEMORY] Queue approaching capacity: %.2f/%.2f MB (%.1f%%)\n", + currentMemMB, maxMemMB, (currentMemMB/maxMemMB)*100) + } return } - // Queue full (20 pending txs) - unlock and wait for space + // Queue memory full - unlock and wait for space + // currentMemMB := float64(q.queueMemoryBytes) / (1024 * 1024) + // maxMemMB := float64(q.maxMemoryBytes) / (1024 * 1024) q.mu.Unlock() + // fmt.Printf("[BACKPRESSURE] Queue memory full (%.2f/%.2f MB), waiting to prevent OOM\n", + // currentMemMB, maxMemMB) + select { case <-time.After(100 * time.Millisecond): // Wait a bit then retry @@ -145,6 +183,17 @@ func (q *sequentialQueue) submitJob(job *SequentialSubmissionJob) { } } +// calculateBlobsMemory returns the total memory size of blobs in bytes +func calculateBlobsMemory(blobs []*share.Blob) uint64 { + var total uint64 + for _, blob := range blobs { + if blob != nil { + total += uint64(len(blob.Data())) + } + } + return total +} + // GetQueueSize returns the number of transactions in the queue func (q *sequentialQueue) GetQueueSize() int { q.mu.RLock() @@ -156,6 +205,12 @@ func (q *sequentialQueue) GetQueueSize() int { func (q *sequentialQueue) processNextTx() { startTime := time.Now() + fmt.Println("Recovering?", q.getRecovering()) + if q.getRecovering() { + fmt.Println("Recovering from previous rejection/eviction - skipping current tx") + return + } + scanStart := time.Now() var qTx *queuedTx q.mu.RLock() @@ -175,6 +230,10 @@ func (q *sequentialQueue) processNextTx() { fmt.Printf("[TIMING] Queue scan took %v (queue size: %d)\n", scanDuration, queueSize) + // Log current signer sequence before broadcast + currentSeq := q.client.Signer().Account(q.accountName).Sequence() + fmt.Printf("[DEBUG] Attempting broadcast with signer sequence: %d\n", currentSeq) + broadcastStart := time.Now() resp, txBytes, err := q.client.BroadcastPayForBlobWithoutRetry( q.ctx, @@ -216,7 +275,7 @@ func (q *sequentialQueue) processNextTx() { // } // } - // // No transaction found with expected sequence - return + // return because we are probably blocked, we will try again return } @@ -247,7 +306,7 @@ func (q *sequentialQueue) processNextTx() { // monitorLoop periodically checks the status of broadcast transactions func (q *sequentialQueue) monitorLoop() { - ticker := time.NewTicker(1 * time.Second) + ticker := time.NewTicker(2 * time.Second) defer ticker.Stop() for { @@ -273,7 +332,7 @@ func (q *sequentialQueue) coordinate() { // TODO: decide if we want to do anything during rejections case qTx := <-q.ResubmitChan: q.ResubmitEvicted(qTx) - case <-ticker.C: + default: q.processNextTx() } } @@ -319,6 +378,10 @@ func (q *sequentialQueue) ResubmitEvicted(qTx *queuedTx) { qTx.isResubmitting = false q.resubmitCount++ q.mu.Unlock() + + // Exit recovery mode after successful resubmission to allow new txs to be broadcast + // q.setRecovering(false) + fmt.Printf("Successfully resubmitted tx %s\n", qTx.txHash[:16]) fmt.Printf("[TIMING] Total ResubmitEvicted took %v\n", time.Since(startTime)) } @@ -330,13 +393,18 @@ func (q *sequentialQueue) checkBroadcastTransactions() { scanStart := time.Now() q.mu.RLock() - // Collect all broadcast transaction hashes and transaction bytes + // Collect broadcast transaction hashes (cap at 20 per batch for efficiency) + const maxBatchSize = 20 var broadcastTxHashes []string var broadcastTxs []*queuedTx for _, tx := range q.queue { if tx.txHash != "" { broadcastTxHashes = append(broadcastTxHashes, tx.txHash) broadcastTxs = append(broadcastTxs, tx) + // Cap at 20 transactions per status check batch + if len(broadcastTxHashes) >= maxBatchSize { + break + } } } fmt.Printf("Broadcast txs: %d\n", len(broadcastTxHashes)) @@ -355,8 +423,11 @@ func (q *sequentialQueue) checkBroadcastTransactions() { txClient := tx.NewTxClient(q.client.GetGRPCConnection()) statusCheckStart := time.Now() - // do a batch status check + + // Try batch status check first statusResp, err := txClient.TxStatusBatch(q.ctx, &tx.TxStatusBatchRequest{TxIds: broadcastTxHashes}) + + // If batch is not supported, fall back to individual status checks if err != nil { return } @@ -370,6 +441,7 @@ func (q *sequentialQueue) checkBroadcastTransactions() { case core.TxStatusCommitted: q.handleCommitted(broadcastTxs[i], statusRespp.Status) case core.TxStatusEvicted: + q.setRecovering(true) // Found an evicted tx - check if already being resubmitted to avoid duplicates q.mu.Lock() if qTx.isResubmitting { @@ -384,6 +456,7 @@ func (q *sequentialQueue) checkBroadcastTransactions() { q.ResubmitChan <- qTx case core.TxStatusRejected: + q.setRecovering(true) prevStatus := "" if i > 0 { prevStatus = statusResp.Statuses[i-1].Status.Status @@ -400,6 +473,7 @@ func (q *sequentialQueue) checkBroadcastTransactions() { } } } + q.recomputeRecoveryState(statusResp.Statuses) statusCheckDuration := time.Since(statusCheckStart) if statusCheckCount > 0 { @@ -409,8 +483,31 @@ func (q *sequentialQueue) checkBroadcastTransactions() { fmt.Printf("[TIMING] Total checkBroadcastTransactions took %v\n", time.Since(startTime)) } -func (q *sequentialQueue) handleEvicted(qTx *queuedTx, statusResp *tx.TxStatusResponse, txClient tx.TxClient) { +func (q *sequentialQueue) recomputeRecoveryState(statuses []*tx.TxStatusResult) { + q.mu.Lock() + defer q.mu.Unlock() + + inRecovery := false + for _, st := range statuses { + switch st.Status.Status { + case core.TxStatusRejected, core.TxStatusEvicted: + // still bad + inRecovery = true + } + } + + if inRecovery { + if !q.isRecovering.Load() { + fmt.Println("Entering recovery mode") + } + q.isRecovering.Store(true) + } else { + if q.isRecovering.Load() { + fmt.Println("Exiting recovery mode") + } + q.isRecovering.Store(false) + } } // handleCommitted processes a confirmed transaction @@ -498,13 +595,25 @@ func (q *sequentialQueue) handleRejected(qTx *queuedTx, statusResp *tx.TxStatusR q.lastRejectedSeq = rejectedSeq } -// removeFromQueue removes a transaction from the queue +// removeFromQueue removes a transaction from the queue and frees its memory func (q *sequentialQueue) removeFromQueue(qTx *queuedTx) { q.mu.Lock() defer q.mu.Unlock() for i, tx := range q.queue { if tx == qTx { + // Decrement memory counter + blobsMemory := calculateBlobsMemory(qTx.blobs) + if q.queueMemoryBytes >= blobsMemory { + q.queueMemoryBytes -= blobsMemory + } else { + // Safety check - should never happen + fmt.Printf("[WARNING] Memory accounting error: queueMemory=%d < blobsMemory=%d\n", + q.queueMemoryBytes, blobsMemory) + q.queueMemoryBytes = 0 + } + + // Remove from queue q.queue = append(q.queue[:i], q.queue[i+1:]...) return }