Skip to content

Conversation

Eric-Warehime
Copy link
Contributor

@Eric-Warehime Eric-Warehime commented Sep 9, 2025

Description

Upstream blockstm

Closes: #XXXX

Summary by CodeRabbit

  • New Features

    • Parallel transaction execution for faster block processing.
    • Object store support across context and runtime.
    • Bank module: virtual-account transfers with end-of-block crediting; emits transfer/message events on SendCoins; recipient auto-creation.
    • Configurable signer extraction in proposal handling.
  • Breaking Changes

    • Public APIs for stores and transaction execution updated (generic KV interfaces, object store type, execution entrypoints).
  • Performance

    • Improved block finalization throughput via parallel execution.
  • Bug Fixes

    • Cache wrapping enabled for trace and listen KV stores.
    • EndBlockers include Bank first.
  • Documentation

    • Release notes and changelog updated; minor dependency bumps.

yihuang and others added 30 commits March 27, 2025 16:15
generic interface

generic btree

generic cachekv

generic transient store

support ObjStore

changelog

Update CHANGELOG.md

Signed-off-by: yihuang <[email protected]>

object store key

Apply review suggestions

fix merge conflict

fix snapshot

revert dependers

prefix store support object store (#236)

Problem: snapshot for object store is not skipped (#585)

resolve
…) (#24604)

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: dependabot[bot] <dependabot[bot]@users.noreply.github.com>
Co-authored-by: Alex | Interchain Labs <[email protected]>
Co-authored-by: Alex | Interchain Labs <[email protected]>
Comment on lines +51 to +54
go func() {
defer wg.Done()
e.Run()
}()

Check notice

Code scanning / CodeQL

Spawning a Go routine Note

Spawning a Go routine may be a possible source of non-determinism
@Eric-Warehime Eric-Warehime marked this pull request as ready for review September 16, 2025 16:02
Copy link
Contributor

coderabbitai bot commented Sep 16, 2025

📝 Walkthrough

Walkthrough

Introduces BlockSTM parallel execution and a public BaseApp RunTx. Adds generic store interfaces with object-store support across store packages. Updates Context with execution indices and incarnation cache. Adds bank virtual-accounts transfers with an EndBlock credit step. Provides runtime object-store DI. Adjusts tests/docs, minor Makefile changes.

Changes

Cohort / File(s) Summary
Release docs
CHANGELOG.md, RELEASE_NOTES.md, client/v2/CHANGELOG.md, x/protocolpool/README.md, client/v2/internal/testpbgogo/msg.proto, client/v2/internal/testpbpulsar/msg.proto, store/CHANGELOG.md
Bumps versions, updates links/notes; whitespace/trailing newline fixes; adds store feature note. No code changes.
Makefile housekeeping
Makefile
Whitespace/newline tweaks. No functional change.
BaseApp execution and APIs
baseapp/abci.go, baseapp/baseapp.go, baseapp/genesis.go, baseapp/options.go, baseapp/test_helpers.go
Adds executor-based FinalizeBlock using a TxRunner; exports RunTx with expanded signature; integrates aborts in proposal paths; supports object stores mounting; test helpers updated.
Context extensions
types/context.go
Adds tx/msg indices, block gas metrics, tx count, and incarnation cache; exposes ObjectStore accessor and builders.
BlockSTM subsystem (new)
blockstm/*
Adds STM scheduler, MV memory, B-Tree wrappers/iterators, MV data/iterators, executor loop, runners (sequential/STM), multi-store in-memory backends, utilities, tests, and benchmarks.
Store generics and object-store support
store/types/*, store/internal/btree/*, store/internal/btreeadaptor.go, store/cachekv/*, store/cachemulti/store.go, store/gaskv/store.go, store/prefix/store.go, store/transient/store.go, store/listenkv/*, store/tracekv/*, store/rootmulti/*, runtime/store.go, runtime/module.go, server/mock/store.go, contrib/x/group/internal/orm/testsupport.go, store/cachekv/search_benchmark_test.go, store/rootmulti/store_test.go
Introduces generic KV interfaces (typed iterators), ObjectStoreKey and StoreTypeObject; adapts cachekv/gaskv/prefix/transient to generics; adds object store paths in rootmulti and cachemulti; updates internal btree to generics; runtime DI for object stores; iterator signatures adjusted; mocks/tests updated.
Test utilities
testutil/context.go
Adds DefaultContextWithObjectStore helper to mount object store in tests.
Auth ante caching
x/auth/ante/sigverify.go
Splits logic; adds per-incarnation caching for signature verification results.
Bank module: virtual accounts and events
x/bank/keeper/keeper.go, x/bank/keeper/send.go, x/bank/keeper/virtual.go, x/bank/module.go, x/bank/types/keys.go, x/bank/keeper/*_test.go, x/bank/keeper/collections_test.go, simapp/app.go, simapp/app_config.go
Adds object-store key injection; virtual transfer APIs and EndBlock credit; send path emits events and ensures recipient account creation; defines ObjectStoreKey; tests use object-store context; simapp adds bank EndBlocker.

Sequence Diagram(s)

sequenceDiagram
  autonumber
  actor CometBFT as CometBFT
  participant BaseApp
  participant TxRunner
  participant Scheduler
  participant MVMemory
  participant DeliverTx

  CometBFT->>BaseApp: FinalizeBlock(req)
  BaseApp->>BaseApp: init txRunner (Default/STM)
  BaseApp->>TxRunner: Run(ctx, multiStore, txs, deliverTx)
  alt STM runner
    TxRunner->>Scheduler: NewScheduler(block_size)
    TxRunner->>MVMemory: NewMVMemory(stores, storage, scheduler)
    par Executors (N)
      loop Until done
        TxRunner->>Scheduler: NextTask()
        alt Execution
          TxRunner->>MVMemory: View(txn)
          TxRunner->>DeliverTx: deliverTx(tx, msWrapper, txIndex, incarnationCache)
          DeliverTx-->>TxRunner: ExecTxResult
          TxRunner->>MVMemory: Record(version, view)
          TxRunner->>Scheduler: FinishExecution(version, wroteNewPath)
        else Validation
          TxRunner->>MVMemory: ValidateReadSet(txn)
          TxRunner->>Scheduler: FinishValidation(txn, aborted?)
        end
      end
    end
    TxRunner->>MVMemory: WriteSnapshot(storage)
  else Default runner
    loop tx in txs
      TxRunner->>DeliverTx: deliverTx(tx, nil, idx, nil)
      DeliverTx-->>TxRunner: ExecTxResult
    end
  end
  TxRunner-->>BaseApp: ExecTxResult[]
  BaseApp->>BaseApp: aggregate GasUsed/Wanted, TxCount
  BaseApp-->>CometBFT: FinalizeBlockResponse
Loading
sequenceDiagram
  autonumber
  participant MsgSender as Account/Module
  participant Bank as Bank Keeper
  participant Ctx as Context
  participant ObjStore as Object Store (transient)
  participant KVStore as Bank KV

  Note over Ctx,ObjStore: Per-tx execution (TxIndex available)
  MsgSender->>Bank: SendCoinsToVirtual(from, to, amt)
  Bank->>Ctx: TxIndex()
  Bank->>ObjStore: addVirtualCoins(key=to||txIndex, +amt)
  Bank-->>MsgSender: ok

  MsgSender->>Bank: SendCoinsFromVirtual(from, to, amt)
  Bank->>ObjStore: subVirtualCoins(key=from||txIndex, -amt)
  Bank-->>MsgSender: ok

  Note over Bank: EndBlock
  Bank->>Bank: CreditVirtualAccounts()
  loop per address (grouped)
    Bank->>ObjStore: iterate keys
    Bank->>KVStore: addCoins(address, sum)
    Bank->>Ctx: ensureAccountCreated(address)
  end
  Bank-->>Ctx: EndBlock complete
Loading

Estimated code review effort

🎯 5 (Critical) | ⏱️ ~120+ minutes

Pre-merge checks and finishing touches

❌ Failed checks (1 warning)
Check name Status Explanation Resolution
Docstring Coverage ⚠️ Warning Docstring coverage is 26.15% which is insufficient. The required threshold is 80.00%. You can run @coderabbitai generate docstrings to improve docstring coverage.
✅ Passed checks (2 passed)
Check name Status Explanation
Description Check ✅ Passed Check skipped - CodeRabbit’s high-level summary is enabled.
Title Check ✅ Passed The PR title "feat!: block stm upstream" is concise and accurately summarizes the primary change (upstreaming the block-STM feature) and the breaking-change intent via the "!" marker, which corresponds to the large blockstm and BaseApp changes in the diff. It is specific enough for a reviewer quickly scanning history to understand the main purpose of the changeset.
✨ Finishing touches
  • 📝 Generate Docstrings
🧪 Generate unit tests
  • Create PR with unit tests
  • Post copyable unit tests in a comment
  • Commit unit tests in branch eric/block-stm-upstream

Tip

👮 Agentic pre-merge checks are now available in preview!

Pro plan users can now enable pre-merge checks in their settings to enforce checklists before merging PRs.

  • Built-in checks – Quickly apply ready-made checks to enforce title conventions, require pull request descriptions that follow templates, validate linked issues for compliance, and more.
  • Custom agentic checks – Define your own rules using CodeRabbit’s advanced agentic capabilities to enforce organization-specific policies and workflows. For example, you can instruct CodeRabbit’s agent to verify that API documentation is updated whenever API schema files are modified in a PR. Note: Upto 5 custom checks are currently allowed during the preview period. Pricing for this feature will be announced in a few weeks.

Please see the documentation for more information.

Example:

reviews:
  pre_merge_checks:
    custom_checks:
      - name: "Undocumented Breaking Changes"
        mode: "warning"
        instructions: |
          Pass/fail criteria: All breaking changes to public APIs, CLI flags, environment variables, configuration keys, database schemas, or HTTP/GraphQL endpoints must be documented in the "Breaking Change" section of the PR description and in CHANGELOG.md. Exclude purely internal or private changes (e.g., code not exported from package entry points or explicitly marked as internal).

Please share your feedback with us on this Discord post.


Thanks for using CodeRabbit! It's free for OSS, and your support helps us grow. If you like it, consider giving us a shout-out.

❤️ Share

Comment @coderabbitai help to get the list of available commands and usage tips.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 30

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (5)
runtime/store.go (1)

159-165: Same here for ReverseIterator.

Apply:

-func (s kvStoreAdapter) ReverseIterator(start, end []byte) storetypes.Iterator {
+func (s kvStoreAdapter) ReverseIterator(start, end []byte) storetypes.Iterator {
 	it, err := s.store.ReverseIterator(start, end)
 	if err != nil {
 		panic(err)
 	}
-	return it
+	return it.(storetypes.Iterator)
 }
x/bank/keeper/send.go (1)

250-260: Redundant account existence check

The ensureAccountCreated function is called after addCoins, but account creation should happen before adding coins to ensure proper initialization. Additionally, this duplicates the account creation logic already present in InputOutputCoins (lines 194-198).

Consider moving the account creation before the coin transfer to ensure proper sequencing:

func (k BaseSendKeeper) SendCoins(ctx context.Context, fromAddr, toAddr sdk.AccAddress, amt sdk.Coins) error {
	if !amt.IsValid() {
		return errorsmod.Wrap(sdkerrors.ErrInvalidCoins, amt.String())
	}

	var err error
	toAddr, err = k.sendRestriction.apply(ctx, fromAddr, toAddr, amt)
	if err != nil {
		return err
	}

+	k.ensureAccountCreated(ctx, toAddr)
+
	err = k.subUnlockedCoins(ctx, fromAddr, amt)
	if err != nil {
		return err
	}

	err = k.addCoins(ctx, toAddr, amt)
	if err != nil {
		return err
	}

-	k.ensureAccountCreated(ctx, toAddr)
	k.emitSendCoinsEvents(ctx, fromAddr, toAddr, amt)
	return nil
}
blockstm/multimvview.go (1)

59-66: Same sizing bug for write locations.
Size MultiLocations to mv.stores length.

- newLocations := make(MultiLocations, len(mv.views))
+ newLocations := make(MultiLocations, len(mv.stores))
 for key, view := range mv.views {
   newLocations[mv.stores[key]] = view.ApplyWriteSet(version)
 }
baseapp/baseapp.go (2)

724-762: BlockGasMeter is not concurrency‑safe; avoid per‑tx consumption in parallel path.

RunTx defers per‑tx BlockGasMeter ConsumeGas and checks IsOutOfGas even when invoked concurrently by the block‑stm runner, which can race on a shared block gas meter.

Apply both guards so Finalize mode only reads/consumes the shared BlockGasMeter when not in the parallel path and not disabled:

- if mode == sdk.ExecModeFinalize && ctx.BlockGasMeter().IsOutOfGas() {
+ if mode == sdk.ExecModeFinalize && txMultiStore == nil && !app.disableBlockGasMeter && ctx.BlockGasMeter().IsOutOfGas() {
   return gInfo, nil, nil, errorsmod.Wrap(sdkerrors.ErrOutOfGas, "no block gas left to run tx")
 }

- if mode == execModeFinalize {
+ if mode == execModeFinalize && txMultiStore == nil && !app.disableBlockGasMeter {
   defer consumeBlockGas()
 }

Follow‑up: rely on the aggregated block gas computation in internalFinalizeBlock (already present) for the parallel path.

Also applies to: 883-889


831-841: Concurrent mempool.Remove across workers may race.

deliverTx calls RunTx→mempool.Remove in Finalize mode for each tx, and block‑stm runs these concurrently. Most mempool implementations aren’t documented as goroutine‑safe for Remove.

Option A (minimal): serialize Remove in Finalize mode.

 case execModeFinalize:
-  err = app.mempool.Remove(tx)
+  // serialize removal to avoid concurrent access in parallel execution
+  app.mu.Lock()
+  err = app.mempool.Remove(tx)
+  app.mu.Unlock()

Option B (preferred): skip Remove inside RunTx when txMultiStore != nil and remove all finalized txs sequentially after execution (aggregator step). If you want, I can draft that change across executeTxsWithExecutor/internalFinalizeBlock.

♻️ Duplicate comments (11)
blockstm/txnrunner.go (1)

213-224: Goroutine spawn flagged for non-determinism (tracking prior automated review).

Parallel preEstimates writes into disjoint indices, which should be deterministic, but calling it out as this was previously flagged. Consider a worker pool with bounded concurrency if needed for resource control.

x/bank/module.go (1)

206-208: EndBlock error path: ensure no panics and deterministic behavior.

Confirm keeper.CreditVirtualAccounts(ctx) never panics (return errors instead) and doesn’t introduce non‑determinism.

Search for panics inside the credit path:

#!/bin/bash
rg -nP -C2 '(func\s+CreditVirtualAccounts\(|\bpanic\()' x/bank/keeper
store/rootmulti/store.go (1)

689-691: Potential panic without recovery mechanism

Multiple panic calls exist in consensus-critical paths that could cause chain halts. These were flagged by static analysis.

These panics in lines 690, 707, and 711 should be converted to error returns or wrapped with recovery mechanisms to prevent chain halts during consensus methods.

blockstm/stm.go (2)

7-7: Importing runtime flagged earlier for non-determinism checks.
Echoing prior bot note; see executor-count determinism above.


51-54: Goroutine spawning flagged earlier for determinism.
Echoing prior bot note; verify results are invariant to concurrency.

x/bank/keeper/virtual.go (2)

91-94: Potential integer overflow when converting TxIndex to uint64

The static analysis tool flagged a potential integer overflow. While sdkCtx.TxIndex() returns an int32 which can safely fit into uint64, the conversion should be explicit about handling negative values, which could theoretically occur in edge cases.

Consider adding a bounds check:

-binary.BigEndian.PutUint64(key[len(addr):], uint64(sdkCtx.TxIndex()))
+txIndex := sdkCtx.TxIndex()
+if txIndex < 0 {
+    return fmt.Errorf("negative transaction index: %d", txIndex)
+}
+binary.BigEndian.PutUint64(key[len(addr):], uint64(txIndex))

108-111: Potential integer overflow (duplicate of line 94)

Same integer overflow concern as in addVirtualCoins.

blockstm/mvmemory.go (3)

36-38: Non-deterministic map iteration identified by static analysis.

The iteration over the stores map on lines 36-38 can result in non-deterministic behavior, as map iteration order is undefined in Go. This could lead to inconsistent initialization order of data elements across different runs.

Consider converting the stores map to a sorted slice before iteration:

 data := make([]MVStore, len(stores))
-for key, i := range stores {
-    data[i] = NewMVStore(key)
-}
+keys := make([]storetypes.StoreKey, 0, len(stores))
+for key := range stores {
+    keys = append(keys, key)
+}
+// Sort keys by a deterministic property (e.g., string representation)
+sort.Slice(keys, func(i, j int) bool {
+    return keys[i].Name() < keys[j].Name()
+})
+for _, key := range keys {
+    data[stores[key]] = NewMVStore(key)
+}

115-119: Non-deterministic map iteration causing potential validation inconsistency.

The iteration over the rs map is non-deterministic. While the validation logic itself checks all entries, the order of checking could affect error reporting or debugging when validation fails.

Consider iterating over stores in a deterministic order:

-for store, readSet := range rs {
-    if !mv.data[store].ValidateReadSet(txn, readSet) {
-        return false
-    }
-}
+// Create a sorted list of store indices
+storeIndices := make([]int, 0, len(rs))
+for store := range rs {
+    storeIndices = append(storeIndices, store)
+}
+sort.Ints(storeIndices)
+for _, store := range storeIndices {
+    if !mv.data[store].ValidateReadSet(txn, rs[store]) {
+        return false
+    }
+}

132-134: Non-deterministic map iteration in WriteSnapshot.

The iteration order over mv.stores is non-deterministic, which could lead to inconsistent snapshot ordering across different executions.

Apply the same deterministic iteration pattern as suggested for lines 36-38 to ensure consistent snapshot generation.

blockstm/memdb.go (1)

184-197: Map iteration non-determinism — safe here; confirm no order-dependent use.

You only copy keys from input map to another map; no iteration order is persisted or used for writes/commits, so consensus safety isn’t impacted. If any downstream logic iterates mmdb.dbs, ensure it never relies on iteration order.

🧹 Nitpick comments (97)
Makefile (1)

109-111: Add .PHONY: all and verify prerequisite tools target exists

  • checkmake warns that all isn’t declared phony. Add it to avoid collisions if a file named all appears.
  • all depends on tools, but there’s no tools: rule in this file. Either define a no-op/tools aggregator target or drop it from all.

Apply this minimal fix (you can place .PHONY: all near other PHONY blocks if you prefer):

 all: tools build lint test vulncheck
+
+.PHONY: all

If tools is not defined elsewhere, also add:

+tools:
+	@true
x/protocolpool/README.md (1)

57-57: Docs nits

  • Remove the double space after “NOTE:” on Line 57.
  • Optional: standardize “Note:” vs “:::tip/:::warning” usage for consistency.
-NOTE:  This feature is designed to work with the SDK's default bond denom.
+NOTE: This feature is designed to work with the SDK's default bond denom.

Also applies to: 94-95, 113-114, 141-141

baseapp/genesis.go (1)

16-16: Replace magic -1 with a named sentinel and document semantics

Using -1 for gas is opaque. Define a named constant (e.g., unlimitedTxGas) and add a brief comment confirming that negative means “unlimited” in deliverTx.

+const unlimitedTxGas = int64(-1) // negative => unlimited/max gas for genesis delivery
 ...
-	res := ba.deliverTx(tx, nil, -1, nil)
+	res := ba.deliverTx(tx, nil, unlimitedTxGas, nil)

Please confirm deliverTx treats negative maxGas as unlimited; if not, use math.MaxInt64 instead.

client/v2/CHANGELOG.md (1)

37-42: Grammar fix

“Fix msg parsing in when …” -> “Fix msg parsing when …”.

-* [#24722](https://github.com/cosmos/cosmos-sdk/pull/24722) Fix msg parsing in when no pulsar file is present.
+* [#24722](https://github.com/cosmos/cosmos-sdk/pull/24722) Fix msg parsing when no pulsar file is present.
CHANGELOG.md (3)

39-46: Wrong link target for v0.53.4

The v0.53.4 header links to tag v0.53.3.

-## [v0.53.4](https://github.com/cosmos/cosmos-sdk/releases/tag/v0.53.3) - 2025-07-25
+## [v0.53.4](https://github.com/cosmos/cosmos-sdk/releases/tag/v0.53.4) - 2025-07-25

43-46: Avoid duplicated feature entry or mark as backport

“The ability to assign a custom signer extraction adapter” appears under both v0.53.4 and v0.53.0. If v0.53.4 backports it, annotate as such; otherwise, remove one instance.

-* (abci_utils) [#25008](https://github.com/cosmos/cosmos-sdk/pull/24861) add the ability to assign a custom signer extraction adapter in `DefaultProposalHandler`.
+* (abci_utils) [#25008](https://github.com/cosmos/cosmos-sdk/pull/24861) Add custom signer extraction adapter in `DefaultProposalHandler`. (backported)

Also applies to: 76-81


37-41: Keep an Unreleased section stub at the top

Standard practice is to keep an Unreleased section for ongoing changes.

 # Changelog
-## [v0.53.4]...
+## [Unreleased]
+
+### Features
+### Improvements
+### Bug Fixes
+### API Breaking
+### State Machine Breaking
+
+## [v0.53.4]...
simapp/app.go (1)

525-531: EndBlockers order confirmed — add a brief comment explaining bank is first.

Both EndBlockers lists start with bank: simapp/app_config.go (EndBlockers — lines 111–113) and simapp/app.go (SetOrderEndBlockers — lines 524–526).
Add a one-line comment above the EndBlockers ordering (e.g., above SetOrderEndBlockers in simapp/app.go) stating bank must run first because virtual-account credits are applied in EndBlock.

RELEASE_NOTES.md (1)

11-11: Update UPGRADING.md link to v0.53.4.

Link still points to v0.53.0.

Apply:

-For more upgrade information, check out our [upgrading guide](https://github.com/cosmos/cosmos-sdk/blob/v0.53.0/UPGRADING.md)
+For more upgrade information, check out our [upgrading guide](https://github.com/cosmos/cosmos-sdk/blob/v0.53.4/UPGRADING.md)
store/types/validity.go (2)

30-36: Doc wording: “nil” vs zero value for generics.

For non-pointer types, “nil” is misleading. Suggest updating the comment to “non‑zero” to match isZero.

-// AssertValidValueGeneric checks if the value is valid(value is not nil and within length limit)
+// AssertValidValueGeneric checks if the value is valid (non-zero and within length limit)

38-43: Optional: guard against negative lengths from custom valueLen.

Defensive check; cheap and clarifies contract.

 func AssertValidValueLength(l int) {
-	if l > MaxValueLength {
+	if l < 0 {
+		panic("value length is negative")
+	}
+	if l > MaxValueLength {
 		panic("value is too large")
 	}
 }
blockstm/condvar.go (2)

5-15: Prefer pointer cond and explicit mutex binding; avoid copying sync.Cond.

Store *sync.Cond and bind to the embedded mutex explicitly. It’s clearer and avoids accidental copies.

 type Condvar struct {
 	sync.Mutex
 	notified bool
-	cond     sync.Cond
+	cond     *sync.Cond
 }
 
 func NewCondvar() *Condvar {
-	c := &Condvar{}
-	c.cond = *sync.NewCond(c)
+	c := &Condvar{}
+	c.cond = sync.NewCond(&c.Mutex)
 	return c
 }

17-23: Minor: use defer for unlock in Wait.

Keeps the critical section exception-safe and conventional.

 func (cv *Condvar) Wait() {
-	cv.Lock()
-	for !cv.notified {
-		cv.cond.Wait()
-	}
-	cv.Unlock()
+	cv.Lock()
+	defer cv.Unlock()
+	for !cv.notified {
+		cv.cond.Wait()
+	}
 }
blockstm/bench_test.go (2)

29-47: Capture loop vars inside sub-bench closures.

Standard Go pitfall; future refactors or parallel runs can bite.

-	for _, tc := range testCases {
+	for _, tc := range testCases {
+		tc := tc
 		b.Run(tc.name+"-sequential", func(b *testing.B) {
 			b.ResetTimer()
 			for i := 0; i < b.N; i++ {
 				runSequential(storage, tc.block)
 			}
 		})
-		for _, worker := range []int{1, 5, 10, 15, 20} {
+		for _, worker := range []int{1, 5, 10, 15, 20} {
+			worker := worker
 			b.Run(tc.name+"-worker-"+strconv.Itoa(worker), func(b *testing.B) {
 				b.ResetTimer()
 				for i := 0; i < b.N; i++ {
 					require.NoError(
 						b,
 						ExecuteBlock(context.Background(), tc.block.Size(), stores, storage, worker, tc.block.ExecuteTx),
 					)
 				}
 			})
 		}
 	}

13-28: Optional: isolate per-iteration state for fair timing.

Reusing storage/block across iterations skews results due to warmed state. Consider reinitializing inside the timed loop with b.StopTimer()/StartTimer().

blockstm/mock_block.go (4)

154-167: Receiver credited even when sender lacks funds. Intentional?

This mints value on underfunded sends; fine for synthetic load, but it hides conflict/abort paths. If unintentional, return an error on insufficient balance.

-	if senderBalance >= amount {
-		// avoid the failure
-		senderBalance -= amount
-	}
-
-	receiverBalance += amount
+	if senderBalance < amount {
+		return fmt.Errorf("insufficient funds: have=%d need=%d", senderBalance, amount)
+	}
+	senderBalance -= amount
+	receiverBalance += amount

105-117: Ignore result explicitly or assert; avoid accidental misuse.

Make intent clear by assigning to _ or checking the bool.

-	return func() {
-		pubKey.VerifySignature(signBytes, sig)
-	}
+	return func() {
+		_ = pubKey.VerifySignature(signBytes, sig)
+	}

119-137: Remove unused parameter i.

It’s not used; simplifies call sites.

-func increaseNonce(i int, sender string, store storetypes.KVStore) error {
+func increaseNonce(sender string, store storetypes.KVStore) error {

And update callers:

-		return increaseNonce(i, sender, store.GetKVStore(StoreKeyAuth))
+		return increaseNonce(sender, store.GetKVStore(StoreKeyAuth))

139-169: Remove unused parameter i.

Not referenced in bankTransfer.

-func bankTransfer(i int, sender, receiver string, amount uint64, store storetypes.KVStore) error {
+func bankTransfer(sender, receiver string, amount uint64, store storetypes.KVStore) error {

And update callers:

-		return bankTransfer(i, sender, receiver, amount, store.GetKVStore(StoreKeyBank))
+		return bankTransfer(sender, receiver, amount, store.GetKVStore(StoreKeyBank))
blockstm/txnrunner.go (2)

98-108: Store-name heuristics (“acc”, “bank”) are brittle.

Relying on k.Name() == "acc"/"bank" can break with module key renames or forks. Prefer explicit store key injection or a lookup passed by the caller to avoid mis-estimation and unnecessary aborts.


115-119: Incarnation cache handoff is sound but consider zeroing map between incarnations.

Swap/Store ensures exclusive cache ownership per incarnation. If the cache can accumulate stale entries, consider clearing it on Store back to reduce memory/cross-incarnation bleed.

store/listenkv/store.go (1)

133-143: Consider documenting event emission semantics with CacheWrap.

Under a cache, listener writes are emitted on Write(), not on intermediate Set/Delete calls. A brief comment here can prevent misuse.

store/listenkv/store_test.go (1)

273-276: Strengthen test: assert cache semantics and listener writes on Write().

Current test only calls CacheWrap(). Add basic assertions that no events emit until Write, then exactly one write event occurs.

Apply this diff:

 func TestListenKVStoreCacheWrap(t *testing.T) {
-	store := newEmptyListenKVStore(nil)
-	store.CacheWrap()
+	listener := types.NewMemoryListener()
+	store := newEmptyListenKVStore(listener)
+	cw := store.CacheWrap()
+	cw.Set([]byte("k"), []byte("v"))
+	require.Empty(t, listener.PopStateCache(), "cache writes must not emit until Write()")
+	cw.Write()
+	cache := listener.PopStateCache()
+	require.NotEmpty(t, cache)
+	require.Equal(t, testStoreKey.Name(), cache[0].StoreKey)
 }
store/tracekv/store_test.go (1)

284-287: Strengthen test: trace should emit on Write(), not on cache Set.

Ensure CacheWrap defers tracing until Write().

Apply this diff:

 func TestTraceKVStoreCacheWrap(t *testing.T) {
-	store := newEmptyTraceKVStore(nil)
-	store.CacheWrap()
+	var buf bytes.Buffer
+	store := newEmptyTraceKVStore(&buf)
+	cw := store.CacheWrap()
+	cw.Set([]byte("k"), []byte("v"))
+	require.Equal(t, "", buf.String(), "cache writes must not be traced until Write()")
+	cw.Write()
+	require.Contains(t, buf.String(), "\"operation\":\"write\"")
 }
baseapp/options.go (1)

128-132: Clarify lifecycle or prevent runtime swaps for SetBlockSTMTxRunner

txRunner is wired into execution (baseapp/baseapp.go:166–168) and used in executeTxsWithExecutor (baseapp/abci.go:874–880) where a default runner is set if nil; SetBlockSTMTxRunner mutates app.txRunner with no sealed check or locking — either document that it must be set during initialization (pre-seal) or add a sealed check/mutex to prevent swapping at runtime.

testutil/context.go (1)

84-97: Type the keys to prevent misuse.

Accepting generic StoreKey allows passing the wrong key type at compile‑time. Prefer concrete types.

Apply:

-func DefaultContextWithObjectStore(tb testing.TB, key, tkey, okey storetypes.StoreKey) TestContext {
+func DefaultContextWithObjectStore(tb testing.TB, key *storetypes.KVStoreKey, tkey *storetypes.TransientStoreKey, okey *storetypes.ObjectStoreKey) TestContext {
blockstm/utils_test.go (1)

71-79: Optional: exercise early‑exit semantics.

Add a case where the callback returns false to ensure iteration stops correctly.

contrib/x/group/internal/orm/testsupport.go (1)

29-31: Align with GetCommitStore API consistently.

Mixed usage of GetCommitStore and deprecated GetCommitKVStore. Use one path and assert type.

Apply:

 func (m MockContext) KVStore(key storetypes.StoreKey) storetypes.KVStore {
-	if s := m.store.GetCommitStore(key); s != nil {
-		return s.(storetypes.KVStore)
-	}
+	if s := m.store.GetCommitStore(key); s != nil {
+		if kv, ok := s.(storetypes.KVStore); ok {
+			return kv
+		}
+		panic("commit store is not a KVStore")
+	}
 	m.store.MountStoreWithDB(key, storetypes.StoreTypeIAVL, m.db)
 	if err := m.store.LoadLatestVersion(); err != nil {
 		panic(err)
 	}
-	return m.store.GetCommitKVStore(key)
+	s := m.store.GetCommitStore(key)
+	if kv, ok := s.(storetypes.KVStore); ok {
+		return kv
+	}
+	panic("commit store is not a KVStore")
 }

Also applies to: 36-37

store/internal/btree/btree_test.go (1)

51-56: Prefer ErrorIs over direct equality for error checks.

Use require.ErrorIs(t, err, errKeyEmpty) to be resilient to wrapped errors.

blockstm/stm_test.go (2)

61-73: Spelling: determistic → deterministic.

Minor clarity nit; rename function and references.

Apply:

-func determisticBlock() *MockBlock {
+func deterministicBlock() *MockBlock {

And update the test case name and call (Lines 93-96):

-			name:      "determisticBlock(),5",
-			blk:       determisticBlock(),
+			name:      "deterministicBlock(),5",
+			blk:       deterministicBlock(),

147-149: Remove redundant continue.

The continue after increment is unnecessary.

Apply:

-				total += binary.BigEndian.Uint64(it.Value())
-				continue
+				total += binary.BigEndian.Uint64(it.Value())
baseapp/test_helpers.go (2)

22-23: Confirm discarding the 3rd RunTx return value is intentional.

If this carries events or execution metadata now, consider exposing it in sims; otherwise underscore is fine. Also, the comment above still says “runTx”; consider updating to “RunTx”.


50-52: FinalizeBlock helper mirrors SimDeliver.

LGTM; keep comments in sync with RunTx naming.

x/bank/keeper/send.go (1)

263-279: Consider using a single event emission

The emitSendCoinsEvents function could be simplified by using a more idiomatic event emission pattern. The current implementation forces grouping of events that might better be emitted separately in different contexts.

Consider this more flexible approach:

-func (k BaseSendKeeper) emitSendCoinsEvents(ctx context.Context, fromAddr, toAddr sdk.AccAddress, amt sdk.Coins) {
-	// bech32 encoding is expensive! Only do it once for fromAddr
-	fromAddrString := fromAddr.String()
-	sdkCtx := sdk.UnwrapSDKContext(ctx)
-	sdkCtx.EventManager().EmitEvents(sdk.Events{
-		sdk.NewEvent(
-			types.EventTypeTransfer,
-			sdk.NewAttribute(types.AttributeKeyRecipient, toAddr.String()),
-			sdk.NewAttribute(types.AttributeKeySender, fromAddrString),
-			sdk.NewAttribute(sdk.AttributeKeyAmount, amt.String()),
-		),
-		sdk.NewEvent(
-			sdk.EventTypeMessage,
-			sdk.NewAttribute(types.AttributeKeySender, fromAddrString),
-		),
-	})
-}
+func (k BaseSendKeeper) emitTransferEvent(ctx context.Context, fromAddr, toAddr sdk.AccAddress, amt sdk.Coins) {
+	sdkCtx := sdk.UnwrapSDKContext(ctx)
+	sdkCtx.EventManager().EmitEvent(
+		sdk.NewEvent(
+			types.EventTypeTransfer,
+			sdk.NewAttribute(types.AttributeKeyRecipient, toAddr.String()),
+			sdk.NewAttribute(types.AttributeKeySender, fromAddr.String()),
+			sdk.NewAttribute(sdk.AttributeKeyAmount, amt.String()),
+		),
+	)
+	sdkCtx.EventManager().EmitEvent(
+		sdk.NewEvent(
+			sdk.EventTypeMessage,
+			sdk.NewAttribute(types.AttributeKeySender, fromAddr.String()),
+		),
+	)
+}
store/internal/btree/memiterator.go (1)

12-12: Generic type assertion limits flexibility

The type assertion var _ types.Iterator = (*memIterator[[]byte])(nil) only verifies that memIterator[[]byte] implements the interface, not other instantiations of the generic type.

Consider adding a comment explaining that this assertion is specifically for the byte array specialization:

-var _ types.Iterator = (*memIterator[[]byte])(nil)
+// Verify that memIterator[[]byte] implements types.Iterator.
+// Note: This only validates the []byte specialization, which is the primary use case.
+var _ types.Iterator = (*memIterator[[]byte])(nil)
blockstm/mvview_test.go (1)

12-36: Test coverage for concurrent scenarios

Given that this is testing BlockSTM (Software Transactional Memory) functionality, the test should include concurrent transaction scenarios to validate isolation and consistency guarantees.

Would you like me to generate additional test cases that cover concurrent transaction scenarios, including:

  • Concurrent reads and writes to the same keys
  • Transaction conflicts and retries
  • Isolation level validation
store/cachekv/search_benchmark_test.go (1)

24-24: Function signature inconsistency

The function signature still returns *Store but the implementation returns *GStore[[]byte]. This appears to be incomplete refactoring to generics.

Update the function signature to match the return type:

-func generateStore() *Store {
+func generateStore() *GStore[[]byte] {
store/rootmulti/store.go (2)

1091-1095: New StoreTypeObject case lacks documentation

The new object store type handling has been added without any documentation explaining its purpose or usage patterns.

Add documentation above the case statement:

+	// StoreTypeObject creates a transient object store that supports 
+	// generic value types beyond []byte. Used for BlockSTM parallel execution
+	// to store intermediate computation results.
	case types.StoreTypeObject:
		if _, ok := key.(*types.ObjectStoreKey); !ok {
			return nil, fmt.Errorf("unexpected key type for a ObjectStoreKey; got: %s, %T", key.String(), key)
		}

		return transient.NewObjStore(), nil

583-590: Type assertion before store wrapping could fail

The code attempts to wrap stores without checking if the type assertion will succeed, which could lead to unexpected behavior for object stores.

Consider adding explicit handling for different store types:

func (rs *Store) CacheMultiStore() types.CacheMultiStore {
	stores := make(map[types.StoreKey]types.CacheWrapper)
	for k, v := range rs.stores {
		store := types.CacheWrapper(v)
-		if kv, ok := store.(types.KVStore); ok {
-			// Wire the listenkv.Store to allow listeners to observe the writes from the cache store,
-			// set same listeners on cache store will observe duplicated writes.
-			if rs.ListeningEnabled(k) {
-				store = listenkv.NewStore(kv, k, rs.listeners[k])
-			}
+		switch s := store.(type) {
+		case types.KVStore:
+			if rs.ListeningEnabled(k) {
+				store = listenkv.NewStore(s, k, rs.listeners[k])
+			}
+		case types.ObjKVStore:
+			// Object stores don't support listening currently
+		default:
+			// Other cache wrapper types
		}
		stores[k] = store
	}
	return cachemulti.NewStore(stores, rs.traceWriter, rs.getTracingContext())
}
x/auth/ante/sigverify.go (3)

35-37: Prefer const for cache key (avoid accidental mutation).
Declare SigVerificationResultCacheKey as a constant.

 var (
   // simulation signature values used to estimate gas consumption
   key                = make([]byte, secp256k1.PubKeySize)
   simSecp256k1Pubkey = &secp256k1.PubKey{Key: key}
   simSecp256k1Sig    [64]byte
-
-  SigVerificationResultCacheKey = "ante:SigVerificationResult"
 )
+
+const SigVerificationResultCacheKey = "ante:SigVerificationResult"

383-394: Don’t ignore NewAnyWithValue error.
If packing the pubkey into Any fails, bail early.

- anyPk, _ := codectypes.NewAnyWithValue(pubKey)
+ anyPk, err := codectypes.NewAnyWithValue(pubKey)
+ if err != nil {
+   return errorsmod.Wrapf(sdkerrors.ErrInvalidPubKey, "pack pubkey: %v", err)
+ }

419-433: Guard cache usage for simulate mode.
Avoid caching simulate results to prevent cross-mode contamination if the same incarnation is reused.

 func (svd SigVerificationDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) {
+  // Do not memoize in simulate mode.
+  if simulate || ctx.ExecMode() == sdk.ExecModeSimulate {
+    if err := svd.anteHandle(ctx, tx, simulate); err != nil {
+      return ctx, err
+    }
+    return next(ctx, tx, simulate)
+  }
   if v, ok := ctx.GetIncarnationCache(SigVerificationResultCacheKey); ok {
     // can't convert `nil` to interface
     if v != nil {
       err = v.(error)
     }
   } else {
     err = svd.anteHandle(ctx, tx, simulate)
     ctx.SetIncarnationCache(SigVerificationResultCacheKey, err)
   }
   if err != nil {
     return ctx, err
   }
   return next(ctx, tx, simulate)
 }
blockstm/stm.go (4)

27-35: Validate txExecutor input.
Nil txExecutor will panic inside executors; fail fast.

 func ExecuteBlockWithEstimates(
   ctx context.Context,
   blockSize int,
   stores map[storetypes.StoreKey]int,
   storage MultiStore,
   executors int,
   estimates []MultiLocations, // txn -> multi-locations
   txExecutor TxExecutor,
 ) error {
+  if txExecutor == nil {
+    return fmt.Errorf("txExecutor must not be nil")
+  }

51-55: Pass loop variables to goroutine for clarity.
Current code is safe, but explicit capture improves readability.

-    go func() {
+    go func(exec *Executor) {
       defer wg.Done()
-      e.Run()
-    }()
+      exec.Run()
+    }(e)

58-66: Error context on incomplete scheduler.
Include progress details to aid debugging.

-  return errors.New("scheduler did not complete")
+  return fmt.Errorf("scheduler did not complete (blockSize=%d, executors=%d)", blockSize, executors)

67-70: Consider surfacing snapshot write errors.
If WriteSnapshot ever returns an error, propagate it.

-  mvMemory.WriteSnapshot(storage)
-  return nil
+  if err := mvMemory.WriteSnapshot(storage); err != nil {
+    return fmt.Errorf("write snapshot: %w", err)
+  }
+  return nil
x/bank/keeper/keeper_test.go (1)

649-679: Virtual transfer happy-path test: LGTM (consider one extra assert).
Optional: also assert acc0 final balances to ensure refund applied as expected.

blockstm/mviterator.go (4)

56-62: Guard Value/Version with validity to avoid stale reads.

Add a validity check to prevent returning cached data after invalidation.

 func (it *MVIterator[V]) Value() V {
-	return it.value
+	it.assertValid()
+	return it.value
 }
 
 func (it *MVIterator[V]) Version() TxnVersion {
-	return it.version
+	it.assertValid()
+	return it.version
 }

64-66: Avoid exposing internal slice; return a copy of reads.

Prevents external mutation of internal state.

 func (it *MVIterator[V]) Reads() []ReadDescriptor {
-	return it.reads
+	return append([]ReadDescriptor(nil), it.reads...)
 }

72-83: Zero cached value/version when aborting due to ESTIMATE in validation.

Prevents accidental reuse of prior cached value/version after iterator invalidation.

 		if !ok {
 			// abort the iterator
 			it.valid = false
 			// signal the validation to fail
 			it.readEstimateValue = true
+			// clear cached value/version
+			var zeroV V
+			it.value = zeroV
+			it.version = 0
 			return
 		}

32-41: Optional: embed a pointer to BTreeIteratorG to avoid copying.

Storing the embedded iterator as a pointer reduces copying and clarifies ownership/lifetime of the underlying btree iterator. Non-blocking.

store/internal/btreeadaptor.go (2)

20-23: Validate constructor callbacks to prevent nil deref.

Panic fast with a clear message if isZero/valueLen are missing.

 func NewBTreeStore[V any](btree btree.BTree[V], isZero func(V) bool, valueLen func(V) int) *BTreeStore[V] {
-	return &BTreeStore[V]{btree, isZero, valueLen}
+	if isZero == nil || valueLen == nil {
+		panic("NewBTreeStore: isZero and valueLen must be non-nil")
+	}
+	return &BTreeStore[V]{btree, isZero, valueLen}
 }

30-36: Iterator creation panics on error; consider wrapping with context or propagating.

If feasible, prefer returning an invalid iterator with Error() set or panic with additional context (start/end) to aid debugging. Non-blocking.

Also applies to: 38-44

blockstm/mergeiterator.go (2)

145-156: Join close errors from both iterators.

Return both errors using errors.Join so no error is lost.

 func (iter *cacheMergeIterator[V]) Close() error {
 	if iter.onClose != nil {
 		iter.onClose(iter)
 	}
 
-	err1 := iter.cache.Close()
-	if err := iter.parent.Close(); err != nil {
-		return err
-	}
-
-	return err1
+	errC := iter.cache.Close()
+	errP := iter.parent.Close()
+	return errors.Join(errP, errC)
 }

168-170: Nit: comment is misleading.

assertValid doesn’t iterate; drop the side-effect note.

-// NOTE: May have side-effect of iterating over cache.
 func (iter *cacheMergeIterator[V]) assertValid() {
blockstm/utils.go (2)

40-43: Fix typo in comment.

-// FetchIncr increaes the atomic value by 1 and returns the old value
+// FetchIncr increases the atomic value by 1 and returns the old value

94-100: ObjIsZero/ObjLen are very loose; document constraints or narrow scope.

Returning nil/1 for arbitrary any can mislead size-based accounting or zero checks for non-pointer types. Consider documenting intended value kinds or providing typed variants.

blockstm/btreeiterator.go (1)

74-79: Clarify error string.

-		return errors.New("invalid memIterator")
+		return errors.New("invalid BTreeIteratorG")
baseapp/baseapp.go (2)

331-339: Nit: rename local var to avoid confusion.

Use objKey instead of memKey for ObjectStoreKey iteration.

- for _, key := range skeys {
-   memKey := keys[key]
-   app.MountStore(memKey, storetypes.StoreTypeObject)
+ for _, key := range skeys {
+   objKey := keys[key]
+   app.MountStore(objKey, storetypes.StoreTypeObject)

796-827: Ante events may be lost if AnteHandler returns zero Context.

You always read events from ctx; when newCtx.IsZero() the ante events live on anteCtx.

- events := ctx.EventManager().Events()
+ events := anteCtx.EventManager().Events()
   // GasMeter expected to be set in AnteHandler
   gasWanted = ctx.GasMeter().Limit()

Optionally, if !newCtx.IsZero(), overwrite events = ctx.EventManager().Events() after updating ctx.

baseapp/abci.go (1)

401-404: Typo in comment.

s/PrepareProoposal/PrepareProposal/

- // Overlap is undesirable, since `internalFinalizeBlock` and `PrepareProoposal` could share access to
+ // Overlap is undesirable, since `internalFinalizeBlock` and `PrepareProposal` could share access to
x/bank/keeper/virtual.go (2)

45-46: Fix typo in comment

Minor typo in the function documentation.

-// which are sumed up and added to the real account at the end of block.
-// Events are emiited the same as normal send.
+// which are summed up and added to the real account at the end of block.
+// Events are emitted the same as normal send.

168-170: Invalid key length handling could be more informative

The error message could provide more context about what was expected.

-return fmt.Errorf("unexpected key length: %s", hex.EncodeToString(it.Key()))
+return fmt.Errorf("unexpected key length %d (expected > 8): %s", len(it.Key()), hex.EncodeToString(it.Key()))
blockstm/status.go (1)

101-109: Missing precondition validation in SetReadyStatus

The method assumes status is StatusAborting but doesn't verify this.

Consider adding validation:

 func (s *StatusEntry) SetReadyStatus() {
     s.Lock()
+    defer s.Unlock()

+    if s.status != StatusAborting {
+        // Log warning: unexpected state transition
+    }
     s.incarnation++
-    // status must be ABORTING
     s.status = StatusReadyToExecute
-
-    s.Unlock()
 }
blockstm/types.go (1)

19-19: Consider making InvalidTxnVersion a function or unexported

Global mutable variables can lead to issues if accidentally modified. Consider making this a function or unexported constant.

-var InvalidTxnVersion = TxnVersion{-1, 0}
+func InvalidTxnVersion() TxnVersion {
+    return TxnVersion{-1, 0}
+}

Or:

-var InvalidTxnVersion = TxnVersion{-1, 0}
+var invalidTxnVersion = TxnVersion{-1, 0}
+
+func GetInvalidTxnVersion() TxnVersion {
+    return invalidTxnVersion
+}
blockstm/btree.go (2)

29-43: GetOrDefault has retry loop without backoff

The GetOrDefault method uses an infinite retry loop for CAS operations. In high contention scenarios, this could lead to excessive CPU usage without backoff.

Consider adding a maximum retry count or exponential backoff for production use:

 func (bt *BTree[T]) GetOrDefault(item T, fillDefaults func(*T)) T {
+    maxRetries := 100
+    for i := 0; i < maxRetries; i++ {
         t := bt.Load()
         result, ok := t.Get(item)
         if ok {
             return result
         }
         fillDefaults(&item)
         c := t.Copy()
         c.Set(item)
         if bt.CompareAndSwap(t, c) {
             return item
         }
+        // Optional: add small backoff here
     }
+    // Fallback: acquire a lock or panic
+    panic("failed to insert after maximum retries")
 }

45-54: Consider documenting contention behavior

The Set and Delete methods also use infinite CAS retry loops. While this is a valid lock-free approach, it would be helpful to document the behavior under high contention.

Add documentation:

// Set inserts or updates an item in the tree.
// Under high contention, this method may retry multiple times.
// Returns the previous value if it existed.
func (bt *BTree[T]) Set(item T) (prev T, ok bool) {

Also applies to: 56-65

blockstm/mvmemory.go (1)

79-86: Optimize DiffOrderedList callback to short-circuit on false returns.

The callback function in DiffOrderedList always returns true, but the implementation could benefit from early termination when wroteNewLocation is already true and no deletions are needed.

Consider optimizing the logic to avoid unnecessary processing:

 DiffOrderedList(prevLoc, newLoc, func(key Key, is_new bool) bool {
     if is_new {
         wroteNewLocation = true
+        // Continue processing to handle deletions
     } else {
         mv.data[i].Delete(key, txn)
     }
     return true
 })
blockstm/wrappers.go (1)

79-89: stmMultiStoreWrapper duplicates functionality already available through embedding.

The stmMultiStoreWrapper type embeds storetypes.MultiStore but then explicitly overrides GetStore and GetKVStore methods to simply delegate to the embedded instance, which would happen automatically without these methods.

Remove the redundant method implementations:

 type stmMultiStoreWrapper struct {
     storetypes.MultiStore
 }
-
-func (ms stmMultiStoreWrapper) GetStore(key storetypes.StoreKey) storetypes.Store {
-    return ms.MultiStore.GetStore(key)
-}
-
-func (ms stmMultiStoreWrapper) GetKVStore(key storetypes.StoreKey) storetypes.KVStore {
-    return ms.MultiStore.GetKVStore(key)
-}
blockstm/mvdata.go (2)

10-12: Consider tuning B-tree degrees based on actual usage patterns.

The B-tree degree constants are set to 4 with a comment about copy-on-write allocations. While smaller degrees do reduce allocation size during COW operations, they also increase tree depth and potentially degrade performance.

Consider making these configurable or conducting benchmarks with your specific workload to determine optimal values. For read-heavy workloads, larger degrees (8-16) might provide better cache locality and fewer traversals:

 const (
-    OuterBTreeDegree = 4 // Since we do copy-on-write a lot, smaller degree means smaller allocations
-    InnerBTreeDegree = 4
+    // DefaultOuterBTreeDegree can be tuned based on workload characteristics
+    // Smaller values (4-8) favor write-heavy COW workloads
+    // Larger values (16-32) favor read-heavy workloads with better cache locality  
+    DefaultOuterBTreeDegree = 8
+    DefaultInnerBTreeDegree = 8
 )
+
+var (
+    OuterBTreeDegree = DefaultOuterBTreeDegree
+    InnerBTreeDegree = DefaultInnerBTreeDegree
+)

26-33: MVStore constructor switches on concrete type instead of interface.

Using a type switch on concrete types limits extensibility. Consider using an interface method or store metadata to determine the appropriate store type.

Consider adding a method to the StoreKey interface to indicate its value type:

 func NewMVStore(key storetypes.StoreKey) MVStore {
-    switch key.(type) {
-    case *storetypes.ObjectStoreKey:
-        return NewGMVData(ObjIsZero, ObjLen)
-    default:
-        return NewGMVData(BytesIsZero, BytesLen)
-    }
+    // Consider adding a ValueType() method to StoreKey interface
+    // This would make the design more extensible
+    if objKey, ok := key.(interface{ IsObjectStore() bool }); ok && objKey.IsObjectStore() {
+        return NewGMVData(ObjIsZero, ObjLen)
+    }
+    return NewGMVData(BytesIsZero, BytesLen)
 }
store/internal/btree/btree.go (2)

39-41: BTree methods have value receivers but modify internal state.

The BTree methods like Set, Delete, and Clear have value receivers but modify the internal tree pointer's state. This could be confusing as it appears immutable but isn't.

Consider using pointer receivers for methods that modify state:

-func (bt BTree[V]) Set(key []byte, value V) {
+func (bt *BTree[V]) Set(key []byte, value V) {
     bt.tree.Set(newItem(key, value))
 }

-func (bt BTree[V]) Delete(key []byte) {
+func (bt *BTree[V]) Delete(key []byte) {
     var empty V
     bt.tree.Delete(newItem(key, empty))
 }

-func (bt BTree[V]) Clear() {
+func (bt *BTree[V]) Clear() {
     bt.tree.Clear()
 }

30-36: Consider making the BTree constructor return a pointer.

Since the BTree wraps a pointer to btree.BTreeG and methods potentially modify state, returning a pointer from the constructor would be more idiomatic and consistent.

-func NewBTree[V any]() BTree[V] {
-    return BTree[V]{
+func NewBTree[V any]() *BTree[V] {
+    return &BTree[V]{
         tree: btree.NewBTreeGOptions(byKeys[V], btree.Options{
             Degree:  bTreeDegree,
             NoLocks: false,
         }),
     }
 }
store/prefix/store.go (3)

89-94: Type assertion could be more robust.

The type assertion any(s).(*GStore[[]byte]) could panic if used incorrectly. While the current usage seems safe, adding a comment would improve maintainability.

Add a comment explaining the safety of this type assertion:

 func (s GStore[V]) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap {
+    // Type assertion is safe here because tracing is only supported for byte stores
+    // For other value types, we fall back to non-traced caching
     if store, ok := any(s).(*GStore[[]byte]); ok {
         return cachekv.NewGStore(tracekv.NewStore(store, w, tc), store.isZero, store.valueLen)
     }
     return s.CacheWrap()
 }

39-42: Fix typo in comment.

There's a typo in the type comment: "convinience" should be "convenience".

 // GStore is similar with cometbft/cometbft/libs/db/prefix_db
 // both gives access only to the limited subset of the store
-// for convinience or safety
+// for convenience or safety

70-76: Nil key panic could provide more context.

The panic message "nil key on Store" could be more descriptive by including the store's prefix information.

 func (s GStore[V]) key(key []byte) (res []byte) {
     if key == nil {
-        panic("nil key on Store")
+        panic(fmt.Sprintf("nil key on Store with prefix %x", s.prefix))
     }
     res = cloneAppend(s.prefix, key)
     return
 }
store/transient/store.go (2)

18-26: Docstring is stale and type name typo.

Comment says "MemDB" and "Commiter"; this now wraps BTree and the interface is "Committer".

Apply this diff:

-// GStore is a wrapper for a MemDB with Commiter implementation
+// GStore is a generic transient store backed by an in‑memory B‑Tree with Committer implementation.
 type GStore[V any] struct {
 	internal.BTreeStore[V]
 }

44-54: Typed-nil values in ObjStore are not treated as zero (surprising semantics).

isZero(v any) uses v == nil, which is false for typed-nil pointers/maps/slices held in interfaces. Deletions and Has() checks may treat typed-nil as present value.

Consider a safer zero check for common reference kinds:

 func NewObjStore() *ObjStore {
   return &ObjStore{*NewGStore(
-    func(v any) bool { return v == nil },
+    func(v any) bool {
+      if v == nil {
+        return true
+      }
+      rv := reflect.ValueOf(v)
+      switch rv.Kind() {
+      case reflect.Ptr, reflect.Map, reflect.Slice, reflect.Func, reflect.Interface, reflect.Chan:
+        return rv.IsNil()
+      default:
+        return false
+      }
+    },
     func(v any) int { return 1 }, // for value length validation
   )}
 }

Add the import:

import "reflect"
blockstm/mvview.go (4)

30-39: Harden type assertions and panic messages in NewMVView.

Direct type assertions will panic with vague messages if storage/mvData types don’t match. Prefer checked assertions and clearer panics.

Apply this diff:

 func NewMVView(store int, storage storetypes.Store, mvData MVStore, scheduler *Scheduler, txn TxnIndex) MVView {
   switch data := mvData.(type) {
   case *GMVData[any]:
-    return NewGMVMemoryView(store, storage.(storetypes.ObjKVStore), data, scheduler, txn)
+    obj, ok := storage.(storetypes.ObjKVStore)
+    if !ok {
+      panic("NewMVView: mvData is object GMVData but storage is not ObjKVStore")
+    }
+    return NewGMVMemoryView(store, obj, data, scheduler, txn)
   case *GMVData[[]byte]:
-    return NewGMVMemoryView(store, storage.(storetypes.KVStore), data, scheduler, txn)
+    kv, ok := storage.(storetypes.KVStore)
+    if !ok {
+      panic("NewMVView: mvData is bytes GMVData but storage is not KVStore")
+    }
+    return NewGMVMemoryView(store, kv, data, scheduler, txn)
   default:
     panic("unsupported value type")
   }
 }

65-79: Consider clearing writeSet after ApplyWriteSet to avoid double-apply.

If ApplyWriteSet is called twice, writes are replayed. If views can outlive ApplyWriteSet, defensive clear helps.

Option:

 func (s *GMVMemoryView[V]) ApplyWriteSet(version TxnVersion) Locations {
   if s.writeSet == nil || s.writeSet.Len() == 0 {
     return nil
   }

   newLocations := make([]Key, 0, s.writeSet.Len())
   s.writeSet.Scan(func(key Key, value V) bool {
     s.mvData.Write(key, value, version)
     newLocations = append(newLocations, key)
     return true
   })
+  // prevent accidental re-apply
+  s.writeSet = nil

   return newLocations
 }

If the view must remain writable post-apply, call s.writeSet.Clear() instead (if available).


111-121: Improve panic text in Set for debuggability.

Include store index to aid diagnostics.

 func (s *GMVMemoryView[V]) Set(key []byte, value V) {
   if s.mvData.isZero(value) {
-    panic("nil value is not allowed")
+    panic(fmt.Sprintf("nil/zero value is not allowed (store=%d)", s.store))
   }
   s.init()
   s.writeSet.OverlaySet(key, value)
 }

Add fmt import if not present.


193-199: Tracing is only applied to []byte stores; ObjKVStore goes untraced.

If tracekv supports generics, consider adding object-store tracing for parity.

Would you like a follow-up patch to add ObjKVStore tracing if tracekv exposes a generic wrapper?

store/cachemulti/store.go (3)

159-169: Tighten panic message and nil-key guard in getCacheWrapper.

Improve operator feedback when a key wasn’t registered/loaded.

- if key == nil || store == nil {
-   panic(fmt.Sprintf("kv store with key %v has not been registered in stores", key))
- }
+ if key == nil {
+   panic("kv store lookup failed: nil StoreKey")
+ }
+ if store == nil {
+   panic(fmt.Sprintf("kv store with key %q is not registered/loaded", key.Name()))
+ }

189-196: Comment nit: returns ObjKVStore, not KVStore.

-// GetObjKVStore returns an underlying KVStore by key.
+// GetObjKVStore returns an underlying ObjKVStore by key.

76-89: Trace parity: wrap ObjKVStore when tracing is enabled

initStore only type-asserts types.KVStore and wraps it with tracekv.NewStore, so types.ObjKVStore instances remain untraced. tracekv is KV-only (tracekv.Store panics in CacheWrapWithTrace), so either extend tracekv to support ObjKVStore or add a dedicated trace wrapper for ObjKVStore and update store/cachemulti/store.go → initStore to detect and wrap types.ObjKVStore as well.

store/types/store.go (2)

267-295: Iterator docs still reference []byte; fix wording/typos.

  • Replace “readonly []byte” with “readonly V”.
  • Fix “relasing” → “releasing”.
-// Value returns the value at the current position. Panics if the iterator is invalid.
-// CONTRACT: value readonly []byte
+// Value returns the value at the current position. Panics if the iterator is invalid.
+// CONTRACT: value readonly V
 func (iter *cacheMergeIterator[V]) Value() (value V)
@@
-// Close closes the iterator, relasing any allocated resources.
+// Close closes the iterator, releasing any allocated resources.

(Note: adjust signatures/comments in the appropriate files accordingly.)


473-495: ObjectStoreKey: fix doc and validate empty name (parity with KVStoreKey).

  • Doc currently says “transient stores”.
  • Add empty-name check similar to NewKVStoreKey.
-// ObjectStoreKey is used for indexing transient stores in a MultiStore
+// ObjectStoreKey is used for indexing object stores in a MultiStore
 type ObjectStoreKey struct {
   name string
 }
@@
-// NewObjectStoreKey constructs new ObjectStoreKey
+// NewObjectStoreKey constructs a new ObjectStoreKey.
 // Must return a pointer according to the ocap principle
 func NewObjectStoreKey(name string) *ObjectStoreKey {
-  return &ObjectStoreKey{
+  if name == "" {
+    panic("empty key name not allowed")
+  }
+  return &ObjectStoreKey{
     name: name,
   }
 }
store/cachekv/internal/mergeiterator.go (2)

10-16: Update comment: deletes are signaled via value (isZero), not nil keys.

Revise to avoid confusion with current semantics.

-// The cache iterator may return nil keys to signal that an item
-// had been deleted (but not deleted in the parent).
+// The cache iterator may return a value that satisfies isZero to signal that an item
+// has been deleted (but not deleted in the parent).

139-147: Close: consider preserving both parent and cache errors.

Return a joined error to avoid losing one side.

 func (iter *cacheMergeIterator[V]) Close() error {
-  err1 := iter.cache.Close()
-  if err := iter.parent.Close(); err != nil {
-    return err
-  }
-  return err1
+  errC := iter.cache.Close()
+  errP := iter.parent.Close()
+  if errC == nil {
+    return errP
+  }
+  if errP == nil {
+    return errC
+  }
+  return errors.Join(errC, errP)
 }

(Requires Go 1.20+; otherwise wrap manually.)

blockstm/memdb.go (2)

122-125: Store type may be misreported (IAVL vs in-memory).

Returning StoreTypeIAVL from an in-memory B-Tree can confuse metrics/routers. Please confirm the intended StoreType for this memory-backed store (likely StoreTypeMemory or equivalent) and adjust if needed.


199-209: Guard missing stores and unsafe type assertions.

Nil lookups or wrong StoreKey flavor will panic later with opaque errors. Fail fast with explicit checks.

Apply this diff:

 func (mmdb *MultiMemDB) GetStore(store storetypes.StoreKey) storetypes.Store {
-	return mmdb.dbs[store]
+	s, ok := mmdb.dbs[store]
+	if !ok {
+		panic("MultiMemDB: unknown store key: " + store.Name())
+	}
+	return s
 }
 
 func (mmdb *MultiMemDB) GetKVStore(store storetypes.StoreKey) storetypes.KVStore {
-	return mmdb.GetStore(store).(storetypes.KVStore)
+	s := mmdb.GetStore(store)
+	kv, ok := s.(storetypes.KVStore)
+	if !ok {
+		panic("MultiMemDB: store is not KVStore: " + store.Name())
+	}
+	return kv
 }
 
 func (mmdb *MultiMemDB) GetObjKVStore(store storetypes.StoreKey) storetypes.ObjKVStore {
-	return mmdb.GetStore(store).(storetypes.ObjKVStore)
+	s := mmdb.GetStore(store)
+	okv, ok := s.(storetypes.ObjKVStore)
+	if !ok {
+		panic("MultiMemDB: store is not ObjKVStore: " + store.Name())
+	}
+	return okv
 }
blockstm/scheduler.go (2)

176-191: Counters never increment; Stats() always reports zeros.

Increment executedTxns on FinishExecution and validatedTxns on successful FinishValidation.

 func (s *Scheduler) FinishExecution(version TxnVersion, wroteNewPath bool) (TxnVersion, TaskKind) {
 	s.txn_status[version.Index].SetExecuted()
+	s.executedTxns.Add(1)
 
 	deps := s.txn_dependency[version.Index].Swap(nil)
 	s.ResumeDependencies(deps)
 	if s.validation_idx.Load() > uint64(version.Index) { // otherwise index already small enough
 		if !wroteNewPath {
 			// schedule validation for current tx only, don't decrease num_active_tasks
 			return version, TaskKindValidation
 		}
 		// schedule validation for txn_idx and higher txns
 		s.DecreaseValidationIdx(version.Index)
 	}
 	DecrAtomic(&s.num_active_tasks)
 	return InvalidTxnVersion, 0
 }

And in FinishValidation:

 func (s *Scheduler) FinishValidation(txn TxnIndex, aborted bool) (TxnVersion, TaskKind) {
 	if aborted {
 		s.txn_status[txn].SetReadyStatus()
 		s.DecreaseValidationIdx(txn + 1)
 		if s.execution_idx.Load() > uint64(txn) {
 			return s.TryIncarnate(txn), TaskKindExecution
 		}
 	}
 
+	if !aborted {
+		s.validatedTxns.Add(1)
+	}
 	DecrAtomic(&s.num_active_tasks)
 	return InvalidTxnVersion, 0
 }

149-166: Lock ordering/risk around Suspend while holding dependency lock.

You hold entry.Lock while calling Suspend on txn_status, which may block or acquire internal locks. Verify Suspend does not call back into the dependency map or block under user control; otherwise, reorder: record dependent under lock, unlock, then call Suspend.

store/cachekv/store.go (2)

194-199: Tracing only for []byte specialization; consider parity for object stores.

CacheWrapWithTrace bypasses tracing for non-byte stores. If object-store ops should be traced, add a generic trace wrapper or equivalent ObjKV tracer.


331-345: Tiny alloc win in small unsorted path.

Pre-size the slice to avoid growth.

-	unsorted := make([]*kvPair[V], 0)
+	unsorted := make([]*kvPair[V], 0, n)
store/gaskv/store.go (2)

9-11: ObjectValueLength needs validation.

Hard-coding 16 bytes affects fee economics. Please confirm this aligns with upstream gas policy or make it configurable.


66-76: Clamp int→Gas conversions to avoid overflow surprises.

Introduce a helper to convert lengths to types.Gas safely and use it for key/value charges.

+// toGasLen converts a non-negative length to Gas.
+func toGasLen(n int) types.Gas {
+	if n <= 0 {
+		return 0
+	}
+	return types.Gas(n)
+}
@@
-	gs.gasMeter.ConsumeGas(gs.gasConfig.ReadCostPerByte*types.Gas(len(key)), types.GasReadPerByteDesc)
-	gs.gasMeter.ConsumeGas(gs.gasConfig.ReadCostPerByte*types.Gas(gs.valueLen(value)), types.GasReadPerByteDesc)
+	gs.gasMeter.ConsumeGas(gs.gasConfig.ReadCostPerByte*toGasLen(len(key)), types.GasReadPerByteDesc)
+	gs.gasMeter.ConsumeGas(gs.gasConfig.ReadCostPerByte*toGasLen(gs.valueLen(value)), types.GasReadPerByteDesc)
@@
-	gs.gasMeter.ConsumeGas(gs.gasConfig.WriteCostPerByte*types.Gas(len(key)), types.GasWritePerByteDesc)
-	gs.gasMeter.ConsumeGas(gs.gasConfig.WriteCostPerByte*types.Gas(gs.valueLen(value)), types.GasWritePerByteDesc)
+	gs.gasMeter.ConsumeGas(gs.gasConfig.WriteCostPerByte*toGasLen(len(key)), types.GasWritePerByteDesc)
+	gs.gasMeter.ConsumeGas(gs.gasConfig.WriteCostPerByte*toGasLen(gs.valueLen(value)), types.GasWritePerByteDesc)
@@
-		gi.gasMeter.ConsumeGas(gi.gasConfig.ReadCostPerByte*types.Gas(len(key)), types.GasValuePerByteDesc)
-		gi.gasMeter.ConsumeGas(gi.gasConfig.ReadCostPerByte*types.Gas(gi.valueLen(value)), types.GasValuePerByteDesc)
+		gi.gasMeter.ConsumeGas(gi.gasConfig.ReadCostPerByte*toGasLen(len(key)), types.GasValuePerByteDesc)
+		gi.gasMeter.ConsumeGas(gi.gasConfig.ReadCostPerByte*toGasLen(gi.valueLen(value)), types.GasValuePerByteDesc)

Also applies to: 78-86, 200-209

Comment on lines 981 to 983
ctx.WithMsgIndex(i)

handler := app.msgServiceRouter.Handler(msg)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Msg index not set; missing assignment.

WithMsgIndex returns a copy; current call is a no‑op.

- ctx.WithMsgIndex(i)
+ ctx = ctx.WithMsgIndex(i)
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
ctx.WithMsgIndex(i)
handler := app.msgServiceRouter.Handler(msg)
ctx = ctx.WithMsgIndex(i)
handler := app.msgServiceRouter.Handler(msg)
🤖 Prompt for AI Agents
In baseapp/baseapp.go around lines 981 to 983, the call ctx.WithMsgIndex(i) is a
no-op because WithMsgIndex returns a copy; assign the result back to ctx (e.g.,
ctx = ctx.WithMsgIndex(i)) before calling app.msgServiceRouter.Handler(msg) so
the context actually carries the message index.

"strconv"
"testing"

"github.com/test-go/testify/require"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Wrong testify import path.

Use the canonical stretchr module or builds will fail.

-	"github.com/test-go/testify/require"
+	"github.com/stretchr/testify/require"
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
"github.com/test-go/testify/require"
"github.com/stretchr/testify/require"
🤖 Prompt for AI Agents
In blockstm/bench_test.go around line 8, the import path
"github.com/test-go/testify/require" is incorrect and will break builds; replace
it with the canonical "github.com/stretchr/testify/require" in the import block
and run go mod tidy (or update go.mod) to ensure the correct module is fetched.

Comment on lines +25 to +30
func (cv *Condvar) Notify() {
cv.Lock()
cv.notified = true
cv.Unlock()
cv.cond.Signal()
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Signal only wakes one waiter; use Broadcast to avoid deadlocks.

Multiple dependents waiting on the same condvar will block forever. Also signal while holding the lock to avoid races between predicate update and wakeup.

-func (cv *Condvar) Notify() {
-	cv.Lock()
-	cv.notified = true
-	cv.Unlock()
-	cv.cond.Signal()
-}
+func (cv *Condvar) Notify() {
+	cv.Lock()
+	cv.notified = true
+	cv.cond.Broadcast()
+	cv.Unlock()
+}
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
func (cv *Condvar) Notify() {
cv.Lock()
cv.notified = true
cv.Unlock()
cv.cond.Signal()
}
func (cv *Condvar) Notify() {
cv.Lock()
cv.notified = true
cv.cond.Broadcast()
cv.Unlock()
}
🤖 Prompt for AI Agents
In blockstm/condvar.go around lines 25 to 30, the Notify implementation uses
cond.Signal and calls Signal after unlocking which can leave multiple waiters
blocked and introduces a race; change to call cv.cond.Broadcast while still
holding cv's lock (i.e., acquire cv.Lock(), set cv.notified = true, call
cv.cond.Broadcast(), then cv.Unlock()) so all waiters are woken and the
predicate update is visible before wakeups.

Comment on lines +51 to +57
func (mv *MultiMVMemoryView) ReadSet() *MultiReadSet {
rs := make(MultiReadSet, len(mv.views))
for key, view := range mv.views {
rs[mv.stores[key]] = view.ReadSet()
}
return &rs
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Slice sized by views can panic; size to all stores.
Indexing by mv.stores[key] requires capacity for every store index.

- rs := make(MultiReadSet, len(mv.views))
+ rs := make(MultiReadSet, len(mv.stores))
 for key, view := range mv.views {
   rs[mv.stores[key]] = view.ReadSet()
 }
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
func (mv *MultiMVMemoryView) ReadSet() *MultiReadSet {
rs := make(MultiReadSet, len(mv.views))
for key, view := range mv.views {
rs[mv.stores[key]] = view.ReadSet()
}
return &rs
}
func (mv *MultiMVMemoryView) ReadSet() *MultiReadSet {
rs := make(MultiReadSet, len(mv.stores))
for key, view := range mv.views {
rs[mv.stores[key]] = view.ReadSet()
}
return &rs
}
🤖 Prompt for AI Agents
In blockstm/multimvview.go around lines 51 to 57, the slice rs is created with
length len(mv.views) but later indexed by mv.stores[key], which can be larger
and cause a panic; change the allocation to make(MultiReadSet, len(mv.stores))
so the slice has capacity for every store index, and optionally add a bounds
check when assigning (e.g., ensure idx := mv.stores[key]; if idx >= 0 && idx <
len(rs) { rs[idx] = view.ReadSet() }) to avoid out-of-range panics if mappings
are unexpected.

Comment on lines +8 to +9
"github.com/test-go/testify/require"
)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Incorrect testify import path.

Use github.com/stretchr/testify/require for consistency with the rest of the repo.

Apply:

-	"github.com/test-go/testify/require"
+	"github.com/stretchr/testify/require"
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
"github.com/test-go/testify/require"
)
"github.com/stretchr/testify/require"
)
🤖 Prompt for AI Agents
In blockstm/mvdata_test.go around lines 8-9, the import path uses the incorrect
testify module ("github.com/test-go/testify/require"); update the import to the
canonical "github.com/stretchr/testify/require" so it matches the rest of the
repo and resolves module lookup errors — replace the erroneous import line with
the correct path and run go test or go mod tidy to verify imports.

Comment on lines +38 to 43
// AssertValidValueLength checks if the value length is within length limit
func AssertValidValueLength(l int) {
if l > MaxValueLength {
panic(errors.New("value is too large"))
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Inconsistent panic types may break recover-based callers.

Other panics in this file use string literals; this one panics with an error value. Unify to avoid behavior changes in tests or recover logic.

Apply:

-func AssertValidValueLength(l int) {
-	if l > MaxValueLength {
-		panic(errors.New("value is too large"))
-	}
-}
+func AssertValidValueLength(l int) {
+	if l > MaxValueLength {
+		panic("value is too large")
+	}
+}

If errors is unused after this, remove the import.

📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
// AssertValidValueLength checks if the value length is within length limit
func AssertValidValueLength(l int) {
if l > MaxValueLength {
panic(errors.New("value is too large"))
}
}
// AssertValidValueLength checks if the value length is within length limit
func AssertValidValueLength(l int) {
if l > MaxValueLength {
panic("value is too large")
}
}
🤖 Prompt for AI Agents
In store/types/validity.go around lines 38 to 43, AssertValidValueLength
currently panics with an errors.New value which is inconsistent with other
panics in this file that use string literals; change the panic to use a plain
string literal (e.g. panic("value is too large")) to match existing panic types
and restore consistent recover behavior, and if the errors package becomes
unused after this change remove its import.

Comment on lines +400 to +403
// ObjectStore fetches an object store from the MultiStore,
func (c Context) ObjectStore(key storetypes.StoreKey) storetypes.ObjKVStore {
return gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter, c.transientKVGasConfig)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

ObjectStore uses transient gas config; likely wrong.

Object stores are persistent; using transientKVGasConfig can under/over‑charge and diverge from KVStore gas policy.

-func (c Context) ObjectStore(key storetypes.StoreKey) storetypes.ObjKVStore {
-  return gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter, c.transientKVGasConfig)
+func (c Context) ObjectStore(key storetypes.StoreKey) storetypes.ObjKVStore {
+  return gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter, c.kvGasConfig)
 }
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
// ObjectStore fetches an object store from the MultiStore,
func (c Context) ObjectStore(key storetypes.StoreKey) storetypes.ObjKVStore {
return gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter, c.transientKVGasConfig)
}
// ObjectStore fetches an object store from the MultiStore,
func (c Context) ObjectStore(key storetypes.StoreKey) storetypes.ObjKVStore {
return gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter, c.kvGasConfig)
}
🤖 Prompt for AI Agents
In types/context.go around lines 400 to 403, the ObjectStore constructor is
incorrectly passing c.transientKVGasConfig (transient gas config) to
gaskv.NewObjStore for a persistent object store; change the argument to the
persistent KV gas config (e.g., c.kvGasConfig or c.persistentKVGasConfig) so the
call becomes gaskv.NewObjStore(c.ms.GetObjKVStore(key), c.gasMeter,
<persistent-gas-config>) to align gas charging with persistent KVStore policy.

Comment on lines +99 to +100
coins = value.(sdk.Coins)
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Type assertion without verification could panic

The type assertion value.(sdk.Coins) could panic if the stored value is not of the expected type. This could happen due to data corruption or programming errors.

Add type checking:

 if value != nil {
-    coins = value.(sdk.Coins)
+    var ok bool
+    coins, ok = value.(sdk.Coins)
+    if !ok {
+        return fmt.Errorf("invalid value type in object store for key %x: expected sdk.Coins, got %T", key, value)
+    }
 }

Committable suggestion skipped: line range outside the PR's diff.

🤖 Prompt for AI Agents
In x/bank/keeper/virtual.go around lines 99-100, the direct type assertion
value.(sdk.Coins) can panic if the stored value is not sdk.Coins; replace it
with a safe comma-ok assertion (coins, ok := value.(sdk.Coins)) and handle the
false case gracefully — e.g., log a warning/error about unexpected type and set
coins = sdk.Coins{} (or return an error/zero value depending on function
contract) to avoid panics and potential data-corruption crashes.

Comment on lines +121 to +122
spendable := value.(sdk.Coins)
balance, hasNeg := spendable.SafeSub(amt...)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Type assertion without verification

Similar to line 99, this type assertion could panic.

Add type checking:

-spendable := value.(sdk.Coins)
+spendable, ok := value.(sdk.Coins)
+if !ok {
+    return errorsmod.Wrapf(
+        sdkerrors.ErrInvalidType,
+        "invalid value type in object store: expected sdk.Coins, got %T",
+        value,
+    )
+}
📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
spendable := value.(sdk.Coins)
balance, hasNeg := spendable.SafeSub(amt...)
spendable, ok := value.(sdk.Coins)
if !ok {
return errorsmod.Wrapf(
sdkerrors.ErrInvalidType,
"invalid value type in object store: expected sdk.Coins, got %T",
value,
)
}
balance, hasNeg := spendable.SafeSub(amt...)
🤖 Prompt for AI Agents
In x/bank/keeper/virtual.go around lines 121 to 122, the unchecked type
assertion "value.(sdk.Coins)" can panic; change it to a safe assertion using the
comma-ok form (e.g., spendable, ok := value.(sdk.Coins)) and handle the false
case by returning an error or a zero balance as appropriate for this function's
contract; ensure any downstream code uses the validated spendable variable and
does not assume the assertion always succeeds.

toAddr = addr
}

sum.Add(it.Value().(sdk.Coins)...)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue

Type assertion without verification in iterator loop

Another unverified type assertion that could panic.

Add type checking:

-sum.Add(it.Value().(sdk.Coins)...)
+coins, ok := it.Value().(sdk.Coins)
+if !ok {
+    return fmt.Errorf("invalid value type in iterator: expected sdk.Coins, got %T", it.Value())
+}
+sum.Add(coins...)

Committable suggestion skipped: line range outside the PR's diff.

🤖 Prompt for AI Agents
In x/bank/keeper/virtual.go around line 180, the loop uses
it.Value().(sdk.Coins) without checking the type which can panic; replace the
direct assertion with the comma-ok pattern (val, ok := it.Value().(sdk.Coins))
and only call sum.Add(val...) when ok is true, otherwise handle the unexpected
type by either skipping the entry (continue) or logging an error/debug message;
ensure nil-safe handling so the iterator cannot cause a runtime panic.

"context"
"errors"
"fmt"
"runtime"

Check notice

Code scanning / CodeQL

Sensitive package import Note

Certain system packages contain functions which may be a possible source of non-determinism
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants