Browse Source

abci: Synchronize FinalizeBlock with the updated specification (#7983)

This change set implements the most recent version of `FinalizeBlock`. 

# What does this change actually contain?

* This change set is rather large but fear not! The majority of the files touched and changes are renaming `ResponseDeliverTx` to `ExecTxResult`. This should be a pretty inoffensive change since they're effectively the same type but with a different name.
* The `execBlockOnProxyApp` was totally removed since it served as just a wrapper around the logic that is now mostly encapsulated within `FinalizeBlock`
* The `updateState` helper function has been made a public method on `State`. It was being exposed as a shim through the testing infrastructure, so this seemed innocuous.
* Tests already existed to ensure that the application received the `ByzantineValidators` and the `ValidatorUpdates`, but one was fixed up to ensure that `LastCommitInfo` was being sent across.
* Tests were removed from the `psql` indexer that seemed to search for an event in the indexer that was not being created.

# Questions for reviewers
* We store this [ABCIResponses](5721a13ab1/proto/tendermint/state/types.pb.go (L37)) type in the data base as the block results. This type has changed since v0.35 to contain the `FinalizeBlock` response. I'm wondering if we need to do any shimming to keep the old data retrieveable?
* Similarly, this change is exposed via the RPC through [ResultBlockResults](5721a13ab1/rpc/coretypes/responses.go (L69)) changing. Should we somehow shim or notify for this change? 


closes: #7658
pull/8070/head
William Banfield 2 years ago
committed by GitHub
parent
commit
0b8a62c87b
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
64 changed files with 919 additions and 1005 deletions
  1. +0
    -85
      abci/client/socket_client_test.go
  2. +20
    -17
      abci/cmd/abci-cli/abci-cli.go
  3. +4
    -4
      abci/example/example_test.go
  4. +14
    -14
      abci/example/kvstore/kvstore.go
  5. +9
    -10
      abci/example/kvstore/kvstore_test.go
  6. +1
    -1
      abci/tests/server/client.go
  7. +2
    -2
      abci/tests/test_cli/ex1.abci
  8. +3
    -3
      abci/tests/test_cli/ex2.abci
  9. +3
    -3
      abci/types/application.go
  10. +1
    -1
      abci/types/messages_test.go
  11. +10
    -0
      abci/types/result.go
  12. +453
    -400
      abci/types/types.pb.go
  13. +1
    -1
      cmd/tendermint/commands/reindex_event.go
  14. +2
    -2
      cmd/tendermint/commands/reindex_event_test.go
  15. +16
    -16
      docs/app-dev/abci-cli.md
  16. +10
    -10
      internal/consensus/mempool_test.go
  17. +1
    -1
      internal/consensus/replay_stubs.go
  18. +0
    -56
      internal/consensus/replay_test.go
  19. +73
    -0
      internal/consensus/state_test.go
  20. +2
    -2
      internal/eventbus/event_bus_test.go
  21. +1
    -1
      internal/inspect/inspect_test.go
  22. +2
    -2
      internal/mempool/mempool.go
  23. +12
    -12
      internal/mempool/mempool_test.go
  24. +1
    -1
      internal/mempool/mock/mempool.go
  25. +3
    -3
      internal/mempool/reactor_test.go
  26. +1
    -1
      internal/mempool/types.go
  27. +5
    -7
      internal/rpc/core/blocks.go
  28. +2
    -2
      internal/rpc/core/blocks_test.go
  29. +4
    -4
      internal/rpc/core/mempool.go
  30. +62
    -88
      internal/state/execution.go
  31. +48
    -58
      internal/state/execution_test.go
  32. +0
    -24
      internal/state/export_test.go
  33. +9
    -12
      internal/state/helpers_test.go
  34. +10
    -11
      internal/state/indexer/block/kv/kv.go
  35. +6
    -6
      internal/state/indexer/block/kv/kv_test.go
  36. +3
    -3
      internal/state/indexer/indexer.go
  37. +2
    -2
      internal/state/indexer/indexer_service_test.go
  38. +1
    -1
      internal/state/indexer/sink/kv/kv_test.go
  39. +2
    -22
      internal/state/indexer/sink/psql/psql_test.go
  40. +1
    -1
      internal/state/indexer/tx/kv/kv_bench_test.go
  41. +4
    -4
      internal/state/indexer/tx/kv/kv_test.go
  42. +1
    -1
      internal/state/state.go
  43. +31
    -31
      internal/state/state_test.go
  44. +4
    -4
      internal/state/store.go
  45. +3
    -3
      internal/state/store_test.go
  46. +2
    -2
      light/detector.go
  47. +1
    -1
      light/rpc/client.go
  48. +2
    -2
      node/node.go
  49. +2
    -2
      proto/tendermint/abci/types.proto
  50. +16
    -15
      proto/tendermint/abci/types.proto.intermediate
  51. +1
    -1
      rpc/client/examples_test.go
  52. +1
    -1
      rpc/client/interface.go
  53. +1
    -1
      rpc/client/mock/abci.go
  54. +5
    -5
      rpc/client/mock/abci_test.go
  55. +2
    -2
      rpc/client/rpc_test.go
  56. +16
    -16
      rpc/coretypes/responses.go
  57. +1
    -1
      spec/abci++/abci++_basic_concepts_002_draft.md
  58. +2
    -2
      spec/abci++/abci++_methods_002_draft.md
  59. +5
    -5
      test/e2e/app/app.go
  60. +1
    -0
      tools/tools.go
  61. +3
    -4
      types/events.go
  62. +6
    -6
      types/results.go
  63. +6
    -6
      types/results_test.go
  64. +3
    -1
      types/validation.go

+ 0
- 85
abci/client/socket_client_test.go View File

@ -1,85 +0,0 @@
package abciclient_test
import (
"context"
"fmt"
"testing"
"time"
"math/rand"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abciclient "github.com/tendermint/tendermint/abci/client"
"github.com/tendermint/tendermint/abci/server"
"github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
)
func TestProperSyncCalls(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
app := slowApp{}
logger := log.NewNopLogger()
_, c := setupClientServer(ctx, t, logger, app)
resp := make(chan error, 1)
go func() {
rsp, err := c.FinalizeBlock(ctx, types.RequestFinalizeBlock{})
assert.NoError(t, err)
assert.NoError(t, c.Flush(ctx))
assert.NotNil(t, rsp)
select {
case <-ctx.Done():
case resp <- c.Error():
}
}()
select {
case <-time.After(time.Second):
require.Fail(t, "No response arrived")
case err, ok := <-resp:
require.True(t, ok, "Must not close channel")
assert.NoError(t, err, "This should return success")
}
}
func setupClientServer(
ctx context.Context,
t *testing.T,
logger log.Logger,
app types.Application,
) (service.Service, abciclient.Client) {
t.Helper()
// some port between 20k and 30k
port := 20000 + rand.Int31()%10000
addr := fmt.Sprintf("localhost:%d", port)
s, err := server.NewServer(logger, addr, "socket", app)
require.NoError(t, err)
require.NoError(t, s.Start(ctx))
t.Cleanup(s.Wait)
c := abciclient.NewSocketClient(logger, addr, true)
require.NoError(t, c.Start(ctx))
t.Cleanup(c.Wait)
require.True(t, s.IsRunning())
require.True(t, c.IsRunning())
return s, c
}
type slowApp struct {
types.BaseApplication
}
func (slowApp) FinalizeBlock(req types.RequestFinalizeBlock) types.ResponseFinalizeBlock {
time.Sleep(200 * time.Millisecond)
return types.ResponseFinalizeBlock{}
}

+ 20
- 17
abci/cmd/abci-cli/abci-cli.go View File

@ -125,7 +125,7 @@ func addCommands(cmd *cobra.Command, logger log.Logger) {
cmd.AddCommand(consoleCmd)
cmd.AddCommand(echoCmd)
cmd.AddCommand(infoCmd)
cmd.AddCommand(deliverTxCmd)
cmd.AddCommand(finalizeBlockCmd)
cmd.AddCommand(checkTxCmd)
cmd.AddCommand(commitCmd)
cmd.AddCommand(versionCmd)
@ -150,10 +150,9 @@ where example.file looks something like:
check_tx 0x00
check_tx 0xff
deliver_tx 0x00
finalize_block 0x00
check_tx 0x00
deliver_tx 0x01
deliver_tx 0x04
finalize_block 0x01 0x04 0xff
info
`,
Args: cobra.ExactArgs(0),
@ -169,7 +168,7 @@ This command opens an interactive console for running any of the other commands
without opening a new connection each time
`,
Args: cobra.ExactArgs(0),
ValidArgs: []string{"echo", "info", "deliver_tx", "check_tx", "commit", "query"},
ValidArgs: []string{"echo", "info", "finalize_block", "check_tx", "commit", "query"},
RunE: cmdConsole,
}
@ -188,11 +187,11 @@ var infoCmd = &cobra.Command{
RunE: cmdInfo,
}
var deliverTxCmd = &cobra.Command{
Use: "deliver_tx",
Short: "deliver a new transaction to the application",
Long: "deliver a new transaction to the application",
Args: cobra.ExactArgs(1),
var finalizeBlockCmd = &cobra.Command{
Use: "finalize_block",
Short: "deliver a block of transactions to the application",
Long: "deliver a block of transactions to the application",
Args: cobra.MinimumNArgs(1),
RunE: cmdFinalizeBlock,
}
@ -426,7 +425,7 @@ func muxOnCommands(cmd *cobra.Command, pArgs []string) error {
return cmdCheckTx(cmd, actualArgs)
case "commit":
return cmdCommit(cmd, actualArgs)
case "deliver_tx":
case "finalize_block":
return cmdFinalizeBlock(cmd, actualArgs)
case "echo":
return cmdEcho(cmd, actualArgs)
@ -500,19 +499,23 @@ func cmdFinalizeBlock(cmd *cobra.Command, args []string) error {
if len(args) == 0 {
printResponse(cmd, args, response{
Code: codeBad,
Log: "want the tx",
Log: "Must provide at least one transaction",
})
return nil
}
txBytes, err := stringOrHexToBytes(args[0])
if err != nil {
return err
txs := make([][]byte, len(args))
for i, arg := range args {
txBytes, err := stringOrHexToBytes(arg)
if err != nil {
return err
}
txs[i] = txBytes
}
res, err := client.FinalizeBlock(cmd.Context(), types.RequestFinalizeBlock{Txs: [][]byte{txBytes}})
res, err := client.FinalizeBlock(cmd.Context(), types.RequestFinalizeBlock{Txs: txs})
if err != nil {
return err
}
for _, tx := range res.Txs {
for _, tx := range res.TxResults {
printResponse(cmd, args, response{
Code: tx.Code,
Data: tx.Data,


+ 4
- 4
abci/example/example_test.go View File

@ -84,8 +84,8 @@ func testBulk(ctx context.Context, t *testing.T, logger log.Logger, app types.Ap
// Send bulk request
res, err := client.FinalizeBlock(ctx, rfb)
require.NoError(t, err)
require.Equal(t, numDeliverTxs, len(res.Txs), "Number of txs doesn't match")
for _, tx := range res.Txs {
require.Equal(t, numDeliverTxs, len(res.TxResults), "Number of txs doesn't match")
for _, tx := range res.TxResults {
require.Equal(t, tx.Code, code.CodeTypeOK, "Tx failed")
}
@ -138,8 +138,8 @@ func testGRPCSync(ctx context.Context, t *testing.T, logger log.Logger, app type
// Send request
response, err := client.FinalizeBlock(ctx, &rfb)
require.NoError(t, err, "Error in GRPC FinalizeBlock")
require.Equal(t, numDeliverTxs, len(response.Txs), "Number of txs returned via GRPC doesn't match")
for _, tx := range response.Txs {
require.Equal(t, numDeliverTxs, len(response.TxResults), "Number of txs returned via GRPC doesn't match")
for _, tx := range response.TxResults {
require.Equal(t, tx.Code, code.CodeTypeOK, "Tx failed")
}
}

+ 14
- 14
abci/example/kvstore/kvstore.go View File

@ -117,7 +117,7 @@ func (app *Application) Info(req types.RequestInfo) types.ResponseInfo {
}
// tx is either "val:pubkey!power" or "key=value" or just arbitrary bytes
func (app *Application) handleTx(tx []byte) *types.ResponseDeliverTx {
func (app *Application) handleTx(tx []byte) *types.ExecTxResult {
// if it starts with "val:", update the validator set
// format is "val:pubkey!power"
if isValidatorTx(tx) {
@ -156,7 +156,7 @@ func (app *Application) handleTx(tx []byte) *types.ResponseDeliverTx {
},
}
return &types.ResponseDeliverTx{Code: code.CodeTypeOK, Events: events}
return &types.ExecTxResult{Code: code.CodeTypeOK, Events: events}
}
func (app *Application) Close() error {
@ -190,12 +190,12 @@ func (app *Application) FinalizeBlock(req types.RequestFinalizeBlock) types.Resp
}
}
respTxs := make([]*types.ResponseDeliverTx, len(req.Txs))
respTxs := make([]*types.ExecTxResult, len(req.Txs))
for i, tx := range req.Txs {
respTxs[i] = app.handleTx(tx)
}
return types.ResponseFinalizeBlock{Txs: respTxs, ValidatorUpdates: app.ValUpdates}
return types.ResponseFinalizeBlock{TxResults: respTxs, ValidatorUpdates: app.ValUpdates}
}
func (*Application) CheckTx(req types.RequestCheckTx) types.ResponseCheckTx {
@ -338,13 +338,13 @@ func isValidatorTx(tx []byte) bool {
// format is "val:pubkey!power"
// pubkey is a base64-encoded 32-byte ed25519 key
func (app *Application) execValidatorTx(tx []byte) *types.ResponseDeliverTx {
func (app *Application) execValidatorTx(tx []byte) *types.ExecTxResult {
tx = tx[len(ValidatorSetChangePrefix):]
// get the pubkey and power
pubKeyAndPower := strings.Split(string(tx), "!")
if len(pubKeyAndPower) != 2 {
return &types.ResponseDeliverTx{
return &types.ExecTxResult{
Code: code.CodeTypeEncodingError,
Log: fmt.Sprintf("Expected 'pubkey!power'. Got %v", pubKeyAndPower)}
}
@ -353,7 +353,7 @@ func (app *Application) execValidatorTx(tx []byte) *types.ResponseDeliverTx {
// decode the pubkey
pubkey, err := base64.StdEncoding.DecodeString(pubkeyS)
if err != nil {
return &types.ResponseDeliverTx{
return &types.ExecTxResult{
Code: code.CodeTypeEncodingError,
Log: fmt.Sprintf("Pubkey (%s) is invalid base64", pubkeyS)}
}
@ -361,7 +361,7 @@ func (app *Application) execValidatorTx(tx []byte) *types.ResponseDeliverTx {
// decode the power
power, err := strconv.ParseInt(powerS, 10, 64)
if err != nil {
return &types.ResponseDeliverTx{
return &types.ExecTxResult{
Code: code.CodeTypeEncodingError,
Log: fmt.Sprintf("Power (%s) is not an int", powerS)}
}
@ -371,7 +371,7 @@ func (app *Application) execValidatorTx(tx []byte) *types.ResponseDeliverTx {
}
// add, update, or remove a validator
func (app *Application) updateValidator(v types.ValidatorUpdate) *types.ResponseDeliverTx {
func (app *Application) updateValidator(v types.ValidatorUpdate) *types.ExecTxResult {
pubkey, err := encoding.PubKeyFromProto(v.PubKey)
if err != nil {
panic(fmt.Errorf("can't decode public key: %w", err))
@ -386,7 +386,7 @@ func (app *Application) updateValidator(v types.ValidatorUpdate) *types.Response
}
if !hasKey {
pubStr := base64.StdEncoding.EncodeToString(pubkey.Bytes())
return &types.ResponseDeliverTx{
return &types.ExecTxResult{
Code: code.CodeTypeUnauthorized,
Log: fmt.Sprintf("Cannot remove non-existent validator %s", pubStr)}
}
@ -398,7 +398,7 @@ func (app *Application) updateValidator(v types.ValidatorUpdate) *types.Response
// add or update validator
value := bytes.NewBuffer(make([]byte, 0))
if err := types.WriteMessage(&v, value); err != nil {
return &types.ResponseDeliverTx{
return &types.ExecTxResult{
Code: code.CodeTypeEncodingError,
Log: fmt.Sprintf("error encoding validator: %v", err)}
}
@ -411,7 +411,7 @@ func (app *Application) updateValidator(v types.ValidatorUpdate) *types.Response
// we only update the changes array if we successfully updated the tree
app.ValUpdates = append(app.ValUpdates, v)
return &types.ResponseDeliverTx{Code: code.CodeTypeOK}
return &types.ExecTxResult{Code: code.CodeTypeOK}
}
// -----------------------------
@ -425,9 +425,9 @@ func isPrepareTx(tx []byte) bool {
// execPrepareTx is noop. tx data is considered as placeholder
// and is substitute at the PrepareProposal.
func (app *Application) execPrepareTx(tx []byte) *types.ResponseDeliverTx {
func (app *Application) execPrepareTx(tx []byte) *types.ExecTxResult {
// noop
return &types.ResponseDeliverTx{}
return &types.ExecTxResult{}
}
// substPrepareTx subst all the preparetx in the blockdata


+ 9
- 10
abci/example/kvstore/kvstore_test.go View File

@ -27,12 +27,12 @@ const (
func testKVStore(t *testing.T, app types.Application, tx []byte, key, value string) {
req := types.RequestFinalizeBlock{Txs: [][]byte{tx}}
ar := app.FinalizeBlock(req)
require.Equal(t, 1, len(ar.Txs))
require.False(t, ar.Txs[0].IsErr())
require.Equal(t, 1, len(ar.TxResults))
require.False(t, ar.TxResults[0].IsErr())
// repeating tx doesn't raise error
ar = app.FinalizeBlock(req)
require.Equal(t, 1, len(ar.Txs))
require.False(t, ar.Txs[0].IsErr())
require.Equal(t, 1, len(ar.TxResults))
require.False(t, ar.TxResults[0].IsErr())
// commit
app.Commit()
@ -107,7 +107,7 @@ func TestPersistentKVStoreInfo(t *testing.T) {
header := tmproto.Header{
Height: height,
}
kvstore.FinalizeBlock(types.RequestFinalizeBlock{Hash: hash, Header: header, Height: height})
kvstore.FinalizeBlock(types.RequestFinalizeBlock{Hash: hash, Header: header})
kvstore.Commit()
resInfo = kvstore.Info(types.RequestInfo{})
@ -196,7 +196,6 @@ func makeApplyBlock(
resFinalizeBlock := kvstore.FinalizeBlock(types.RequestFinalizeBlock{
Hash: hash,
Header: header,
Height: height,
Txs: txs,
})
@ -326,13 +325,13 @@ func runClientTests(ctx context.Context, t *testing.T, client abciclient.Client)
func testClient(ctx context.Context, t *testing.T, app abciclient.Client, tx []byte, key, value string) {
ar, err := app.FinalizeBlock(ctx, types.RequestFinalizeBlock{Txs: [][]byte{tx}})
require.NoError(t, err)
require.Equal(t, 1, len(ar.Txs))
require.False(t, ar.Txs[0].IsErr())
require.Equal(t, 1, len(ar.TxResults))
require.False(t, ar.TxResults[0].IsErr())
// repeating FinalizeBlock doesn't raise error
ar, err = app.FinalizeBlock(ctx, types.RequestFinalizeBlock{Txs: [][]byte{tx}})
require.NoError(t, err)
require.Equal(t, 1, len(ar.Txs))
require.False(t, ar.Txs[0].IsErr())
require.Equal(t, 1, len(ar.TxResults))
require.False(t, ar.TxResults[0].IsErr())
// commit
_, err = app.Commit(ctx)
require.NoError(t, err)


+ 1
- 1
abci/tests/server/client.go View File

@ -51,7 +51,7 @@ func Commit(ctx context.Context, client abciclient.Client, hashExp []byte) error
func FinalizeBlock(ctx context.Context, client abciclient.Client, txBytes [][]byte, codeExp []uint32, dataExp []byte) error {
res, _ := client.FinalizeBlock(ctx, types.RequestFinalizeBlock{Txs: txBytes})
for i, tx := range res.Txs {
for i, tx := range res.TxResults {
code, data, log := tx.Code, tx.Data, tx.Log
if code != codeExp[i] {
fmt.Println("Failed test: FinalizeBlock")


+ 2
- 2
abci/tests/test_cli/ex1.abci View File

@ -1,10 +1,10 @@
echo hello
info
commit
deliver_tx "abc"
finalize_block "abc"
info
commit
query "abc"
deliver_tx "def=xyz"
finalize_block "def=xyz" "ghi=123"
commit
query "def"

+ 3
- 3
abci/tests/test_cli/ex2.abci View File

@ -1,7 +1,7 @@
check_tx 0x00
check_tx 0xff
deliver_tx 0x00
finalize_block 0x00
check_tx 0x00
deliver_tx 0x01
deliver_tx 0x04
finalize_block 0x01
finalize_block 0x04
info

+ 3
- 3
abci/types/application.go View File

@ -103,12 +103,12 @@ func (BaseApplication) ProcessProposal(req RequestProcessProposal) ResponseProce
}
func (BaseApplication) FinalizeBlock(req RequestFinalizeBlock) ResponseFinalizeBlock {
txs := make([]*ResponseDeliverTx, len(req.Txs))
txs := make([]*ExecTxResult, len(req.Txs))
for i := range req.Txs {
txs[i] = &ResponseDeliverTx{Code: CodeTypeOK}
txs[i] = &ExecTxResult{Code: CodeTypeOK}
}
return ResponseFinalizeBlock{
Txs: txs,
TxResults: txs,
}
}


+ 1
- 1
abci/types/messages_test.go View File

@ -13,7 +13,7 @@ import (
)
func TestMarshalJSON(t *testing.T) {
b, err := json.Marshal(&ResponseDeliverTx{})
b, err := json.Marshal(&ExecTxResult{Code: 1})
assert.NoError(t, err)
// include empty fields.
assert.True(t, strings.Contains(string(b), "code"))


+ 10
- 0
abci/types/result.go View File

@ -33,6 +33,16 @@ func (r ResponseDeliverTx) IsErr() bool {
return r.Code != CodeTypeOK
}
// IsOK returns true if Code is OK.
func (r ExecTxResult) IsOK() bool {
return r.Code == CodeTypeOK
}
// IsErr returns true if Code is something other than OK.
func (r ExecTxResult) IsErr() bool {
return r.Code != CodeTypeOK
}
// IsOK returns true if Code is OK.
func (r ResponseQuery) IsOK() bool {
return r.Code == CodeTypeOK


+ 453
- 400
abci/types/types.pb.go
File diff suppressed because it is too large
View File


+ 1
- 1
cmd/tendermint/commands/reindex_event.go View File

@ -213,7 +213,7 @@ func eventReIndex(cmd *cobra.Command, args eventReIndexArgs) error {
Height: b.Height,
Index: uint32(i),
Tx: b.Data.Txs[i],
Result: *(r.FinalizeBlock.Txs[i]),
Result: *(r.FinalizeBlock.TxResults[i]),
}
_ = batch.Add(&tr)


+ 2
- 2
cmd/tendermint/commands/reindex_event_test.go View File

@ -153,10 +153,10 @@ func TestReIndexEvent(t *testing.T) {
On("IndexTxEvents", mock.AnythingOfType("[]*types.TxResult")).Return(errors.New("")).Once().
On("IndexTxEvents", mock.AnythingOfType("[]*types.TxResult")).Return(nil)
dtx := abcitypes.ResponseDeliverTx{}
dtx := abcitypes.ExecTxResult{}
abciResp := &prototmstate.ABCIResponses{
FinalizeBlock: &abcitypes.ResponseFinalizeBlock{
Txs: []*abcitypes.ResponseDeliverTx{&dtx},
TxResults: []*abcitypes.ExecTxResult{&dtx},
},
}


+ 16
- 16
docs/app-dev/abci-cli.md View File

@ -27,17 +27,17 @@ Usage:
abci-cli [command]
Available Commands:
batch Run a batch of abci commands against an application
check_tx Validate a tx
commit Commit the application state and return the Merkle root hash
console Start an interactive abci console for multiple commands
deliver_tx Deliver a new tx to the application
kvstore ABCI demo example
echo Have the application echo a message
help Help about any command
info Get some info about the application
query Query the application state
set_option Set an options on the application
batch Run a batch of abci commands against an application
check_tx Validate a tx
commit Commit the application state and return the Merkle root hash
console Start an interactive abci console for multiple commands
finalize_block Send a set of transactions to the application
kvstore ABCI demo example
echo Have the application echo a message
help Help about any command
info Get some info about the application
query Query the application state
set_option Set an options on the application
Flags:
--abci string socket or grpc (default "socket")
@ -53,7 +53,7 @@ Use "abci-cli [command] --help" for more information about a command.
The `abci-cli` tool lets us send ABCI messages to our application, to
help build and debug them.
The most important messages are `deliver_tx`, `check_tx`, and `commit`,
The most important messages are `finalize_block`, `check_tx`, and `commit`,
but there are others for convenience, configuration, and information
purposes.
@ -173,7 +173,7 @@ Try running these commands:
-> code: OK
-> data.hex: 0x0000000000000000
> deliver_tx "abc"
> finalize_block "abc"
-> code: OK
> info
@ -192,7 +192,7 @@ Try running these commands:
-> value: abc
-> value.hex: 616263
> deliver_tx "def=xyz"
> finalize_block "def=xyz"
-> code: OK
> commit
@ -207,8 +207,8 @@ Try running these commands:
-> value.hex: 78797A
```
Note that if we do `deliver_tx "abc"` it will store `(abc, abc)`, but if
we do `deliver_tx "abc=efg"` it will store `(abc, efg)`.
Note that if we do `finalize_block "abc"` it will store `(abc, abc)`, but if
we do `finalize_block "abc=efg"` it will store `(abc, efg)`.
Similarly, you could put the commands in a file and run
`abci-cli --verbose batch < myfile`.


+ 10
- 10
internal/consensus/mempool_test.go View File

@ -51,7 +51,7 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
ensureNewEventOnChannel(t, newBlockCh) // first block gets committed
ensureNoNewEventOnChannel(t, newBlockCh)
deliverTxsRange(ctx, t, cs, 0, 1)
checkTxsRange(ctx, t, cs, 0, 1)
ensureNewEventOnChannel(t, newBlockCh) // commit txs
ensureNewEventOnChannel(t, newBlockCh) // commit updated app hash
ensureNoNewEventOnChannel(t, newBlockCh)
@ -118,7 +118,7 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
round = 0
ensureNewRound(t, newRoundCh, height, round) // first round at next height
deliverTxsRange(ctx, t, cs, 0, 1) // we deliver txs, but dont set a proposal so we get the next round
checkTxsRange(ctx, t, cs, 0, 1) // we deliver txs, but don't set a proposal so we get the next round
ensureNewTimeout(t, timeoutCh, height, round, cs.config.TimeoutPropose.Nanoseconds())
round++ // moving to the next round
@ -126,7 +126,7 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
ensureNewEventOnChannel(t, newBlockCh) // now we can commit the block
}
func deliverTxsRange(ctx context.Context, t *testing.T, cs *State, start, end int) {
func checkTxsRange(ctx context.Context, t *testing.T, cs *State, start, end int) {
t.Helper()
// Deliver some txs.
for i := start; i < end; i++ {
@ -159,7 +159,7 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
newBlockHeaderCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlockHeader)
const numTxs int64 = 3000
go deliverTxsRange(ctx, t, cs, 0, int(numTxs))
go checkTxsRange(ctx, t, cs, 0, int(numTxs))
startTestRound(ctx, cs, cs.Height, cs.Round)
for n := int64(0); n < numTxs; {
@ -192,8 +192,8 @@ func TestMempoolRmBadTx(t *testing.T) {
txBytes := make([]byte, 8)
binary.BigEndian.PutUint64(txBytes, uint64(0))
resDeliver := app.FinalizeBlock(abci.RequestFinalizeBlock{Txs: [][]byte{txBytes}})
assert.False(t, resDeliver.Txs[0].IsErr(), fmt.Sprintf("expected no error. got %v", resDeliver))
resFinalize := app.FinalizeBlock(abci.RequestFinalizeBlock{Txs: [][]byte{txBytes}})
assert.False(t, resFinalize.TxResults[0].IsErr(), fmt.Sprintf("expected no error. got %v", resFinalize))
resCommit := app.Commit()
assert.True(t, len(resCommit.Data) > 0)
@ -265,20 +265,20 @@ func (app *CounterApplication) Info(req abci.RequestInfo) abci.ResponseInfo {
}
func (app *CounterApplication) FinalizeBlock(req abci.RequestFinalizeBlock) abci.ResponseFinalizeBlock {
respTxs := make([]*abci.ResponseDeliverTx, len(req.Txs))
respTxs := make([]*abci.ExecTxResult, len(req.Txs))
for i, tx := range req.Txs {
txValue := txAsUint64(tx)
if txValue != uint64(app.txCount) {
respTxs[i] = &abci.ResponseDeliverTx{
respTxs[i] = &abci.ExecTxResult{
Code: code.CodeTypeBadNonce,
Log: fmt.Sprintf("Invalid nonce. Expected %d, got %d", app.txCount, txValue),
}
continue
}
app.txCount++
respTxs[i] = &abci.ResponseDeliverTx{Code: code.CodeTypeOK}
respTxs[i] = &abci.ExecTxResult{Code: code.CodeTypeOK}
}
return abci.ResponseFinalizeBlock{Txs: respTxs}
return abci.ResponseFinalizeBlock{TxResults: respTxs}
}
func (app *CounterApplication) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx {


+ 1
- 1
internal/consensus/replay_stubs.go View File

@ -32,7 +32,7 @@ func (emptyMempool) Update(
_ context.Context,
_ int64,
_ types.Txs,
_ []*abci.ResponseDeliverTx,
_ []*abci.ExecTxResult,
_ mempool.PreCheckFunc,
_ mempool.PostCheckFunc,
) error {


+ 0
- 56
internal/consensus/replay_test.go View File

@ -35,7 +35,6 @@ import (
"github.com/tendermint/tendermint/libs/log"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/privval"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
)
@ -652,61 +651,6 @@ func TestHandshakeReplayNone(t *testing.T) {
}
}
// Test mockProxyApp should not panic when app return ABCIResponses with some empty ResponseDeliverTx
func TestMockProxyApp(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
sim := setupSimulator(ctx, t) // setup config and simulator
cfg := sim.Config
assert.NotNil(t, cfg)
logger := log.TestingLogger()
var validTxs, invalidTxs = 0, 0
txCount := 0
assert.NotPanics(t, func() {
abciResWithEmptyDeliverTx := new(tmstate.ABCIResponses)
abciResWithEmptyDeliverTx.FinalizeBlock = new(abci.ResponseFinalizeBlock)
abciResWithEmptyDeliverTx.FinalizeBlock.Txs = make([]*abci.ResponseDeliverTx, 0)
abciResWithEmptyDeliverTx.FinalizeBlock.Txs = append(abciResWithEmptyDeliverTx.FinalizeBlock.Txs, &abci.ResponseDeliverTx{})
// called when saveABCIResponses:
bytes, err := proto.Marshal(abciResWithEmptyDeliverTx)
require.NoError(t, err)
loadedAbciRes := new(tmstate.ABCIResponses)
// this also happens sm.LoadABCIResponses
err = proto.Unmarshal(bytes, loadedAbciRes)
require.NoError(t, err)
mock, err := newMockProxyApp(ctx, logger, []byte("mock_hash"), loadedAbciRes)
require.NoError(t, err)
abciRes := new(tmstate.ABCIResponses)
abciRes.FinalizeBlock = new(abci.ResponseFinalizeBlock)
abciRes.FinalizeBlock.Txs = make([]*abci.ResponseDeliverTx, len(loadedAbciRes.FinalizeBlock.Txs))
someTx := []byte("tx")
resp, err := mock.FinalizeBlock(ctx, abci.RequestFinalizeBlock{Txs: [][]byte{someTx}})
require.NoError(t, err)
// TODO: make use of res.Log
// TODO: make use of this info
// Blocks may include invalid txs.
for _, tx := range resp.Txs {
if tx.Code == abci.CodeTypeOK {
validTxs++
} else {
invalidTxs++
}
txCount++
}
})
require.Equal(t, 1, txCount)
require.Equal(t, 1, validTxs)
require.Zero(t, invalidTxs)
}
func tempWALWithData(t *testing.T, data []byte) string {
t.Helper()


+ 73
- 0
internal/consensus/state_test.go View File

@ -1965,6 +1965,79 @@ func TestProcessProposalAccept(t *testing.T) {
}
}
func TestFinalizeBlockCalled(t *testing.T) {
for _, testCase := range []struct {
name string
voteNil bool
expectCalled bool
}{
{
name: "finalze block called when block committed",
voteNil: false,
expectCalled: true,
},
{
name: "not called when block not committed",
voteNil: true,
expectCalled: false,
},
} {
t.Run(testCase.name, func(t *testing.T) {
config := configSetup(t)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
m := abcimocks.NewBaseMock()
m.On("ProcessProposal", mock.Anything).Return(abcitypes.ResponseProcessProposal{Accept: true})
m.On("VerifyVoteExtension", mock.Anything).Return(abcitypes.ResponseVerifyVoteExtension{
Result: abcitypes.ResponseVerifyVoteExtension_ACCEPT,
})
m.On("FinalizeBlock", mock.Anything).Return(abcitypes.ResponseFinalizeBlock{}).Maybe()
cs1, vss := makeState(ctx, t, makeStateArgs{config: config, application: m})
height, round := cs1.Height, cs1.Round
proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal)
newRoundCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryNewRound)
pv1, err := cs1.privValidator.GetPubKey(ctx)
require.NoError(t, err)
addr := pv1.Address()
voteCh := subscribeToVoter(ctx, t, cs1, addr)
startTestRound(ctx, cs1, cs1.Height, round)
ensureNewRound(t, newRoundCh, height, round)
ensureNewProposal(t, proposalCh, height, round)
rs := cs1.GetRoundState()
blockID := types.BlockID{}
nextRound := round + 1
nextHeight := height
if !testCase.voteNil {
nextRound = 0
nextHeight = height + 1
blockID = types.BlockID{
Hash: rs.ProposalBlock.Hash(),
PartSetHeader: rs.ProposalBlockParts.Header(),
}
}
signAddVotes(ctx, t, cs1, tmproto.PrevoteType, config.ChainID(), blockID, vss[1:]...)
ensurePrevoteMatch(t, voteCh, height, round, rs.ProposalBlock.Hash())
signAddVotes(ctx, t, cs1, tmproto.PrecommitType, config.ChainID(), blockID, vss[1:]...)
ensurePrecommit(t, voteCh, height, round)
ensureNewRound(t, newRoundCh, nextHeight, nextRound)
m.AssertExpectations(t)
if !testCase.expectCalled {
m.AssertNotCalled(t, "FinalizeBlock", mock.Anything)
} else {
m.AssertCalled(t, "FinalizeBlock", mock.Anything)
}
})
}
}
// 4 vals, 3 Nil Precommits at P0
// What we want:
// P0 waits for timeoutPrecommit before starting next round


+ 2
- 2
internal/eventbus/event_bus_test.go View File

@ -27,7 +27,7 @@ func TestEventBusPublishEventTx(t *testing.T) {
require.NoError(t, err)
tx := types.Tx("foo")
result := abci.ResponseDeliverTx{
result := abci.ExecTxResult{
Data: []byte("bar"),
Events: []abci.Event{
{Type: "testType", Attributes: []abci.EventAttribute{{Key: "baz", Value: "1"}}},
@ -134,7 +134,7 @@ func TestEventBusPublishEventTxDuplicateKeys(t *testing.T) {
require.NoError(t, err)
tx := types.Tx("foo")
result := abci.ResponseDeliverTx{
result := abci.ExecTxResult{
Data: []byte("bar"),
Events: []abci.Event{
{


+ 1
- 1
internal/inspect/inspect_test.go View File

@ -265,7 +265,7 @@ func TestBlockResults(t *testing.T) {
// tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
stateStoreMock.On("LoadABCIResponses", testHeight).Return(&state.ABCIResponses{
FinalizeBlock: &abcitypes.ResponseFinalizeBlock{
Txs: []*abcitypes.ResponseDeliverTx{
TxResults: []*abcitypes.ExecTxResult{
{
GasUsed: testGasUsed,
},


+ 2
- 2
internal/mempool/mempool.go View File

@ -417,7 +417,7 @@ func (txmp *TxMempool) Update(
ctx context.Context,
blockHeight int64,
blockTxs types.Txs,
deliverTxResponses []*abci.ResponseDeliverTx,
execTxResult []*abci.ExecTxResult,
newPreFn PreCheckFunc,
newPostFn PostCheckFunc,
) error {
@ -433,7 +433,7 @@ func (txmp *TxMempool) Update(
}
for i, tx := range blockTxs {
if deliverTxResponses[i].Code == abci.CodeTypeOK {
if execTxResult[i].Code == abci.CodeTypeOK {
// add the valid committed transaction to the cache (if missing)
_ = txmp.cache.Push(tx)
} else if !txmp.config.KeepInvalidTxsInCache {


+ 12
- 12
internal/mempool/mempool_test.go View File

@ -172,9 +172,9 @@ func TestTxMempool_TxsAvailable(t *testing.T) {
rawTxs[i] = tx.tx
}
responses := make([]*abci.ResponseDeliverTx, len(rawTxs[:50]))
responses := make([]*abci.ExecTxResult, len(rawTxs[:50]))
for i := 0; i < len(responses); i++ {
responses[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
responses[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
}
// commit half the transactions and ensure we fire an event
@ -204,9 +204,9 @@ func TestTxMempool_Size(t *testing.T) {
rawTxs[i] = tx.tx
}
responses := make([]*abci.ResponseDeliverTx, len(rawTxs[:50]))
responses := make([]*abci.ExecTxResult, len(rawTxs[:50]))
for i := 0; i < len(responses); i++ {
responses[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
responses[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
}
txmp.Lock()
@ -231,9 +231,9 @@ func TestTxMempool_Flush(t *testing.T) {
rawTxs[i] = tx.tx
}
responses := make([]*abci.ResponseDeliverTx, len(rawTxs[:50]))
responses := make([]*abci.ExecTxResult, len(rawTxs[:50]))
for i := 0; i < len(responses); i++ {
responses[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
responses[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
}
txmp.Lock()
@ -446,7 +446,7 @@ func TestTxMempool_ConcurrentTxs(t *testing.T) {
for range ticker.C {
reapedTxs := txmp.ReapMaxTxs(200)
if len(reapedTxs) > 0 {
responses := make([]*abci.ResponseDeliverTx, len(reapedTxs))
responses := make([]*abci.ExecTxResult, len(reapedTxs))
for i := 0; i < len(responses); i++ {
var code uint32
@ -456,7 +456,7 @@ func TestTxMempool_ConcurrentTxs(t *testing.T) {
code = abci.CodeTypeOK
}
responses[i] = &abci.ResponseDeliverTx{Code: code}
responses[i] = &abci.ExecTxResult{Code: code}
}
txmp.Lock()
@ -494,9 +494,9 @@ func TestTxMempool_ExpiredTxs_NumBlocks(t *testing.T) {
// reap 5 txs at the next height -- no txs should expire
reapedTxs := txmp.ReapMaxTxs(5)
responses := make([]*abci.ResponseDeliverTx, len(reapedTxs))
responses := make([]*abci.ExecTxResult, len(reapedTxs))
for i := 0; i < len(responses); i++ {
responses[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
responses[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
}
txmp.Lock()
@ -520,9 +520,9 @@ func TestTxMempool_ExpiredTxs_NumBlocks(t *testing.T) {
// removed. However, we do know that that at most 95 txs can be expired and
// removed.
reapedTxs = txmp.ReapMaxTxs(5)
responses = make([]*abci.ResponseDeliverTx, len(reapedTxs))
responses = make([]*abci.ExecTxResult, len(reapedTxs))
for i := 0; i < len(responses); i++ {
responses[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
responses[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
}
txmp.Lock()


+ 1
- 1
internal/mempool/mock/mempool.go View File

@ -27,7 +27,7 @@ func (Mempool) Update(
_ context.Context,
_ int64,
_ types.Txs,
_ []*abci.ResponseDeliverTx,
_ []*abci.ExecTxResult,
_ mempool.PreCheckFunc,
_ mempool.PostCheckFunc,
) error {


+ 3
- 3
internal/mempool/reactor_test.go View File

@ -242,9 +242,9 @@ func TestReactorConcurrency(t *testing.T) {
mempool.Lock()
defer mempool.Unlock()
deliverTxResponses := make([]*abci.ResponseDeliverTx, len(txs))
deliverTxResponses := make([]*abci.ExecTxResult, len(txs))
for i := range txs {
deliverTxResponses[i] = &abci.ResponseDeliverTx{Code: 0}
deliverTxResponses[i] = &abci.ExecTxResult{Code: 0}
}
require.NoError(t, mempool.Update(ctx, 1, convertTex(txs), deliverTxResponses, nil, nil))
@ -261,7 +261,7 @@ func TestReactorConcurrency(t *testing.T) {
mempool.Lock()
defer mempool.Unlock()
err := mempool.Update(ctx, 1, []types.Tx{}, make([]*abci.ResponseDeliverTx, 0), nil, nil)
err := mempool.Update(ctx, 1, []types.Tx{}, make([]*abci.ExecTxResult, 0), nil, nil)
require.NoError(t, err)
}()
}


+ 1
- 1
internal/mempool/types.go View File

@ -66,7 +66,7 @@ type Mempool interface {
ctx context.Context,
blockHeight int64,
blockTxs types.Txs,
deliverTxResponses []*abci.ResponseDeliverTx,
txResults []*abci.ExecTxResult,
newPreFn PreCheckFunc,
newPostFn PostCheckFunc,
) error


+ 5
- 7
internal/rpc/core/blocks.go View File

@ -193,8 +193,6 @@ func (env *Environment) Commit(ctx context.Context, heightPtr *int64) (*coretype
// If no height is provided, it will fetch results for the latest block.
//
// Results are for the height of the block containing the txs.
// Thus response.results.deliver_tx[5] is the results of executing
// getBlock(h).Txs[5]
// More: https://docs.tendermint.com/master/rpc/#/Info/block_results
func (env *Environment) BlockResults(ctx context.Context, heightPtr *int64) (*coretypes.ResultBlockResults, error) {
height, err := env.getHeight(env.BlockStore.Height(), heightPtr)
@ -208,13 +206,13 @@ func (env *Environment) BlockResults(ctx context.Context, heightPtr *int64) (*co
}
var totalGasUsed int64
for _, tx := range results.FinalizeBlock.GetTxs() {
totalGasUsed += tx.GetGasUsed()
for _, res := range results.FinalizeBlock.GetTxResults() {
totalGasUsed += res.GetGasUsed()
}
return &coretypes.ResultBlockResults{
Height: height,
TxsResults: results.FinalizeBlock.Txs,
TxsResults: results.FinalizeBlock.TxResults,
TotalGasUsed: totalGasUsed,
FinalizeBlockEvents: results.FinalizeBlock.Events,
ValidatorUpdates: results.FinalizeBlock.ValidatorUpdates,
@ -222,8 +220,8 @@ func (env *Environment) BlockResults(ctx context.Context, heightPtr *int64) (*co
}, nil
}
// BlockSearch searches for a paginated set of blocks matching BeginBlock and
// EndBlock event search criteria.
// BlockSearch searches for a paginated set of blocks matching the provided
// query.
func (env *Environment) BlockSearch(
ctx context.Context,
query string,


+ 2
- 2
internal/rpc/core/blocks_test.go View File

@ -72,7 +72,7 @@ func TestBlockchainInfo(t *testing.T) {
func TestBlockResults(t *testing.T) {
results := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
Txs: []*abci.ResponseDeliverTx{
TxResults: []*abci.ExecTxResult{
{Code: 0, Data: []byte{0x01}, Log: "ok", GasUsed: 10},
{Code: 0, Data: []byte{0x02}, Log: "ok", GasUsed: 5},
{Code: 1, Log: "not ok", GasUsed: 0},
@ -99,7 +99,7 @@ func TestBlockResults(t *testing.T) {
{101, true, nil},
{100, false, &coretypes.ResultBlockResults{
Height: 100,
TxsResults: results.FinalizeBlock.Txs,
TxsResults: results.FinalizeBlock.TxResults,
TotalGasUsed: 15,
FinalizeBlockEvents: results.FinalizeBlock.Events,
ValidatorUpdates: results.FinalizeBlock.ValidatorUpdates,


+ 4
- 4
internal/rpc/core/mempool.go View File

@ -114,10 +114,10 @@ func (env *Environment) BroadcastTxCommit(ctx context.Context, tx types.Tx) (*co
}
return &coretypes.ResultBroadcastTxCommit{
CheckTx: *r,
DeliverTx: txres.TxResult,
Hash: tx.Hash(),
Height: txres.Height,
CheckTx: *r,
TxResult: txres.TxResult,
Hash: tx.Hash(),
Height: txres.Height,
}, nil
}
}


+ 62
- 88
internal/state/execution.go View File

@ -2,7 +2,6 @@ package state
import (
"context"
"errors"
"fmt"
"time"
@ -13,6 +12,7 @@ import (
"github.com/tendermint/tendermint/internal/proxy"
"github.com/tendermint/tendermint/libs/log"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
tmtypes "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
)
@ -162,7 +162,7 @@ func (blockExec *BlockExecutor) ProcessProposal(
Hash: block.Header.Hash(),
Header: *block.Header.ToProto(),
Txs: block.Data.Txs.ToSliceOfBytes(),
LastCommitInfo: buildLastCommitInfo(block, blockExec.store, state.InitialHeight),
ProposedLastCommit: buildLastCommitInfo(block, blockExec.store, state.InitialHeight),
ByzantineValidators: block.Evidence.ToABCI(),
}
@ -207,18 +207,22 @@ func (blockExec *BlockExecutor) ValidateBlock(ctx context.Context, state State,
func (blockExec *BlockExecutor) ApplyBlock(
ctx context.Context,
state State,
blockID types.BlockID,
block *types.Block,
) (State, error) {
blockID types.BlockID, block *types.Block) (State, error) {
// validate the block if we haven't already
if err := blockExec.ValidateBlock(ctx, state, block); err != nil {
return state, ErrInvalidBlock(err)
}
startTime := time.Now().UnixNano()
abciResponses, err := execBlockOnProxyApp(ctx,
blockExec.logger, blockExec.proxyApp, block, blockExec.store, state.InitialHeight,
pbh := block.Header.ToProto()
finalizeBlockResponse, err := blockExec.proxyApp.FinalizeBlock(
ctx,
abci.RequestFinalizeBlock{
Hash: block.Hash(),
Header: *pbh,
Txs: block.Txs.ToSliceOfBytes(),
DecidedLastCommit: buildLastCommitInfo(block, blockExec.store, state.InitialHeight),
ByzantineValidators: block.Evidence.ToABCI(),
},
)
endTime := time.Now().UnixNano()
blockExec.metrics.BlockProcessingTime.Observe(float64(endTime-startTime) / 1000000)
@ -226,19 +230,22 @@ func (blockExec *BlockExecutor) ApplyBlock(
return state, ErrProxyAppConn(err)
}
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: finalizeBlockResponse,
}
// Save the results before we commit.
if err := blockExec.store.SaveABCIResponses(block.Height, abciResponses); err != nil {
return state, err
}
// validate the validator updates and convert to tendermint types
abciValUpdates := abciResponses.FinalizeBlock.ValidatorUpdates
err = validateValidatorUpdates(abciValUpdates, state.ConsensusParams.Validator)
err = validateValidatorUpdates(finalizeBlockResponse.ValidatorUpdates, state.ConsensusParams.Validator)
if err != nil {
return state, fmt.Errorf("error in validator updates: %w", err)
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciValUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(finalizeBlockResponse.ValidatorUpdates)
if err != nil {
return state, err
}
@ -247,13 +254,13 @@ func (blockExec *BlockExecutor) ApplyBlock(
}
// Update the state with the block and responses.
state, err = updateState(state, blockID, &block.Header, abciResponses, validatorUpdates)
state, err = state.Update(blockID, &block.Header, ABCIResponsesResultsHash(abciResponses), finalizeBlockResponse.ConsensusParamUpdates, validatorUpdates)
if err != nil {
return state, fmt.Errorf("commit failed for application: %w", err)
}
// Lock mempool, commit app state, update mempoool.
appHash, retainHeight, err := blockExec.Commit(ctx, state, block, abciResponses.FinalizeBlock.Txs)
appHash, retainHeight, err := blockExec.Commit(ctx, state, block, finalizeBlockResponse.TxResults)
if err != nil {
return state, fmt.Errorf("commit failed for application: %w", err)
}
@ -282,7 +289,7 @@ func (blockExec *BlockExecutor) ApplyBlock(
// Events are fired after everything else.
// NOTE: if we crash between Commit and Save, events wont be fired during replay
fireEvents(ctx, blockExec.logger, blockExec.eventBus, block, blockID, abciResponses, validatorUpdates)
fireEvents(ctx, blockExec.logger, blockExec.eventBus, block, blockID, finalizeBlockResponse, validatorUpdates)
return state, nil
}
@ -326,7 +333,7 @@ func (blockExec *BlockExecutor) Commit(
ctx context.Context,
state State,
block *types.Block,
deliverTxResponses []*abci.ResponseDeliverTx,
txResults []*abci.ExecTxResult,
) ([]byte, int64, error) {
blockExec.mempool.Lock()
defer blockExec.mempool.Unlock()
@ -359,7 +366,7 @@ func (blockExec *BlockExecutor) Commit(
ctx,
block.Height,
block.Txs,
deliverTxResponses,
txResults,
TxPreCheckForState(state),
TxPostCheckForState(state),
)
@ -367,55 +374,11 @@ func (blockExec *BlockExecutor) Commit(
return res.Data, res.RetainHeight, err
}
//---------------------------------------------------------
// Helper functions for executing blocks and updating state
// Executes block's transactions on proxyAppConn.
// Returns a list of transaction results and updates to the validator set
func execBlockOnProxyApp(
ctx context.Context,
logger log.Logger,
proxyAppConn proxy.AppConnConsensus,
block *types.Block,
store Store,
initialHeight int64,
) (*tmstate.ABCIResponses, error) {
abciResponses := new(tmstate.ABCIResponses)
abciResponses.FinalizeBlock = &abci.ResponseFinalizeBlock{}
dtxs := make([]*abci.ResponseDeliverTx, len(block.Txs))
abciResponses.FinalizeBlock.Txs = dtxs
// Begin block
var err error
pbh := block.Header.ToProto()
if pbh == nil {
return nil, errors.New("nil header")
}
abciResponses.FinalizeBlock, err = proxyAppConn.FinalizeBlock(
ctx,
abci.RequestFinalizeBlock{
Txs: block.Txs.ToSliceOfBytes(),
Hash: block.Hash(),
Header: *pbh,
Height: block.Height,
LastCommitInfo: buildLastCommitInfo(block, store, initialHeight),
ByzantineValidators: block.Evidence.ToABCI(),
},
)
if err != nil {
logger.Error("error in proxyAppConn.FinalizeBlock", "err", err)
return nil, err
}
logger.Info("executed block", "height", block.Height)
return abciResponses, nil
}
func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) abci.LastCommitInfo {
func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) abci.CommitInfo {
if block.Height == initialHeight {
// there is no last commmit for the initial height.
// return an empty value.
return abci.LastCommitInfo{}
return abci.CommitInfo{}
}
lastValSet, err := store.LoadValidators(block.Height - 1)
@ -446,7 +409,7 @@ func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) a
}
}
return abci.LastCommitInfo{
return abci.CommitInfo{
Round: block.LastCommit.Round,
Votes: votes,
}
@ -477,16 +440,16 @@ func validateValidatorUpdates(abciUpdates []abci.ValidatorUpdate,
return nil
}
// updateState returns a new State updated according to the header and responses.
func updateState(
state State,
// Update returns a copy of state with the fields set using the arguments passed in.
func (state State) Update(
blockID types.BlockID,
header *types.Header,
abciResponses *tmstate.ABCIResponses,
resultsHash []byte,
consensusParamUpdates *tmtypes.ConsensusParams,
validatorUpdates []*types.Validator,
) (State, error) {
// Copy the valset so we can apply changes from EndBlock
// Copy the valset so we can apply changes from FinalizeBlock
// and update s.LastValidators and s.Validators.
nValSet := state.NextValidators.Copy()
@ -507,9 +470,9 @@ func updateState(
// Update the params with the latest abciResponses.
nextParams := state.ConsensusParams
lastHeightParamsChanged := state.LastHeightConsensusParamsChanged
if abciResponses.FinalizeBlock.ConsensusParamUpdates != nil {
// NOTE: must not mutate s.ConsensusParams
nextParams = state.ConsensusParams.UpdateConsensusParams(abciResponses.FinalizeBlock.ConsensusParamUpdates)
if consensusParamUpdates != nil {
// NOTE: must not mutate state.ConsensusParams
nextParams = state.ConsensusParams.UpdateConsensusParams(consensusParamUpdates)
err := nextParams.ValidateConsensusParams()
if err != nil {
return state, fmt.Errorf("error updating consensus params: %w", err)
@ -538,7 +501,7 @@ func updateState(
LastHeightValidatorsChanged: lastHeightValsChanged,
ConsensusParams: nextParams,
LastHeightConsensusParamsChanged: lastHeightParamsChanged,
LastResultsHash: ABCIResponsesResultsHash(abciResponses),
LastResultsHash: resultsHash,
AppHash: nil,
}, nil
}
@ -552,13 +515,13 @@ func fireEvents(
eventBus types.BlockEventPublisher,
block *types.Block,
blockID types.BlockID,
abciResponses *tmstate.ABCIResponses,
finalizeBlockResponse *abci.ResponseFinalizeBlock,
validatorUpdates []*types.Validator,
) {
if err := eventBus.PublishEventNewBlock(ctx, types.EventDataNewBlock{
Block: block,
BlockID: blockID,
ResultFinalizeBlock: *abciResponses.FinalizeBlock,
ResultFinalizeBlock: *finalizeBlockResponse,
}); err != nil {
logger.Error("failed publishing new block", "err", err)
}
@ -566,7 +529,7 @@ func fireEvents(
if err := eventBus.PublishEventNewBlockHeader(ctx, types.EventDataNewBlockHeader{
Header: block.Header,
NumTxs: int64(len(block.Txs)),
ResultFinalizeBlock: *abciResponses.FinalizeBlock,
ResultFinalizeBlock: *finalizeBlockResponse,
}); err != nil {
logger.Error("failed publishing new block header", "err", err)
}
@ -583,9 +546,9 @@ func fireEvents(
}
// sanity check
if len(abciResponses.FinalizeBlock.Txs) != len(block.Data.Txs) {
if len(finalizeBlockResponse.TxResults) != len(block.Data.Txs) {
panic(fmt.Sprintf("number of TXs (%d) and ABCI TX responses (%d) do not match",
len(block.Data.Txs), len(abciResponses.FinalizeBlock.Txs)))
len(block.Data.Txs), len(finalizeBlockResponse.TxResults)))
}
for i, tx := range block.Data.Txs {
@ -594,14 +557,14 @@ func fireEvents(
Height: block.Height,
Index: uint32(i),
Tx: tx,
Result: *(abciResponses.FinalizeBlock.Txs[i]),
Result: *(finalizeBlockResponse.TxResults[i]),
},
}); err != nil {
logger.Error("failed publishing event TX", "err", err)
}
}
if len(validatorUpdates) > 0 {
if len(finalizeBlockResponse.ValidatorUpdates) > 0 {
if err := eventBus.PublishEventValidatorSetUpdates(ctx,
types.EventDataValidatorSetUpdates{ValidatorUpdates: validatorUpdates}); err != nil {
logger.Error("failed publishing event", "err", err)
@ -624,23 +587,34 @@ func ExecCommitBlock(
initialHeight int64,
s State,
) ([]byte, error) {
abciResponses, err := execBlockOnProxyApp(ctx, logger, appConnConsensus, block, store, initialHeight)
pbh := block.Header.ToProto()
finalizeBlockResponse, err := appConnConsensus.FinalizeBlock(
ctx,
abci.RequestFinalizeBlock{
Hash: block.Hash(),
Header: *pbh,
Txs: block.Txs.ToSliceOfBytes(),
DecidedLastCommit: buildLastCommitInfo(block, store, initialHeight),
ByzantineValidators: block.Evidence.ToABCI(),
},
)
if err != nil {
logger.Error("failed executing block on proxy app", "height", block.Height, "err", err)
logger.Error("executing block", "err", err)
return nil, err
}
logger.Info("executed block", "height", block.Height)
// the BlockExecutor condition is using for the final block replay process.
if be != nil {
abciValUpdates := abciResponses.FinalizeBlock.ValidatorUpdates
err = validateValidatorUpdates(abciValUpdates, s.ConsensusParams.Validator)
err = validateValidatorUpdates(finalizeBlockResponse.ValidatorUpdates, s.ConsensusParams.Validator)
if err != nil {
logger.Error("err", err)
logger.Error("validating validator updates", "err", err)
return nil, err
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciValUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(finalizeBlockResponse.ValidatorUpdates)
if err != nil {
logger.Error("err", err)
logger.Error("converting validator updates to native types", "err", err)
return nil, err
}
@ -650,7 +624,7 @@ func ExecCommitBlock(
}
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
fireEvents(ctx, be.logger, be.eventBus, block, blockID, abciResponses, validatorUpdates)
fireEvents(ctx, be.logger, be.eventBus, block, blockID, finalizeBlockResponse, validatorUpdates)
}
// Commit block, get hash back


+ 48
- 58
internal/state/execution_test.go View File

@ -27,7 +27,6 @@ import (
"github.com/tendermint/tendermint/internal/store"
"github.com/tendermint/tendermint/internal/test/factory"
"github.com/tendermint/tendermint/libs/log"
tmtime "github.com/tendermint/tendermint/libs/time"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
)
@ -68,8 +67,10 @@ func TestApplyBlock(t *testing.T) {
assert.EqualValues(t, 1, state.Version.Consensus.App, "App version wasn't updated")
}
// TestBeginBlockValidators ensures we send absent validators list.
func TestBeginBlockValidators(t *testing.T) {
// TestFinalizeBlockDecidedLastCommit ensures we correctly send the DecidedLastCommit to the
// application. The test ensures that the DecidedLastCommit properly reflects
// which validators signed the preceding block.
func TestFinalizeBlockDecidedLastCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -80,67 +81,56 @@ func TestBeginBlockValidators(t *testing.T) {
err := proxyApp.Start(ctx)
require.NoError(t, err)
state, stateDB, _ := makeState(t, 2, 2)
state, stateDB, privVals := makeState(t, 7, 1)
stateStore := sm.NewStore(stateDB)
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
prevBlockID := types.BlockID{Hash: prevHash, PartSetHeader: prevParts}
var (
now = tmtime.Now()
commitSig0 = types.NewCommitSigForBlock(
[]byte("Signature1"),
state.Validators.Validators[0].Address,
now,
types.VoteExtensionToSign{},
)
commitSig1 = types.NewCommitSigForBlock(
[]byte("Signature2"),
state.Validators.Validators[1].Address,
now,
types.VoteExtensionToSign{},
)
absentSig = types.NewCommitSigAbsent()
)
absentSig := types.NewCommitSigAbsent()
testCases := []struct {
desc string
lastCommitSigs []types.CommitSig
expectedAbsentValidators []int
name string
absentCommitSigs map[int]bool
}{
{"none absent", []types.CommitSig{commitSig0, commitSig1}, []int{}},
{"one absent", []types.CommitSig{commitSig0, absentSig}, []int{1}},
{"multiple absent", []types.CommitSig{absentSig, absentSig}, []int{0, 1}},
{"none absent", map[int]bool{}},
{"one absent", map[int]bool{1: true}},
{"multiple absent", map[int]bool{1: true, 3: true}},
}
for _, tc := range testCases {
lastCommit := types.NewCommit(1, 0, prevBlockID, tc.lastCommitSigs)
// block for height 2
block, err := sf.MakeBlock(state, 2, lastCommit)
require.NoError(t, err)
_, err = sm.ExecCommitBlock(ctx, nil, proxyApp.Consensus(), block, log.TestingLogger(), stateStore, 1, state)
require.NoError(t, err, tc.desc)
// -> app receives a list of validators with a bool indicating if they signed
ctr := 0
for i, v := range app.CommitVotes {
if ctr < len(tc.expectedAbsentValidators) &&
tc.expectedAbsentValidators[ctr] == i {
t.Run(tc.name, func(t *testing.T) {
blockStore := store.NewBlockStore(dbm.NewMemDB())
evpool := &mocks.EvidencePool{}
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, 0)
evpool.On("Update", ctx, mock.Anything, mock.Anything).Return()
evpool.On("CheckEvidence", ctx, mock.Anything).Return(nil)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(), mmock.Mempool{}, evpool, blockStore)
state, _, lastCommit := makeAndCommitGoodBlock(ctx, t, state, 1, new(types.Commit), state.NextValidators.Validators[0].Address, blockExec, privVals, nil)
for idx, isAbsent := range tc.absentCommitSigs {
if isAbsent {
lastCommit.Signatures[idx] = absentSig
}
}
assert.False(t, v.SignedLastBlock)
ctr++
} else {
assert.True(t, v.SignedLastBlock)
// block for height 2
block, err := sf.MakeBlock(state, 2, lastCommit)
require.NoError(t, err)
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
_, err = blockExec.ApplyBlock(ctx, state, blockID, block)
require.NoError(t, err)
// -> app receives a list of validators with a bool indicating if they signed
for i, v := range app.CommitVotes {
_, absent := tc.absentCommitSigs[i]
assert.Equal(t, !absent, v.SignedLastBlock)
}
}
})
}
}
// TestBeginBlockByzantineValidators ensures we send byzantine validators list.
func TestBeginBlockByzantineValidators(t *testing.T) {
// TestFinalizeBlockByzantineValidators ensures we send byzantine validators list.
func TestFinalizeBlockByzantineValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -301,7 +291,7 @@ func TestProcessProposal(t *testing.T) {
Header: *block1.Header.ToProto(),
Txs: block1.Txs.ToSliceOfBytes(),
ByzantineValidators: block1.Evidence.ToABCI(),
LastCommitInfo: abci.LastCommitInfo{
ProposedLastCommit: abci.CommitInfo{
Round: 0,
Votes: voteInfos,
},
@ -445,8 +435,8 @@ func TestUpdateValidators(t *testing.T) {
}
}
// TestEndBlockValidatorUpdates ensures we update validator set and send an event.
func TestEndBlockValidatorUpdates(t *testing.T) {
// TestFinalizeBlockValidatorUpdates ensures we update validator set and send an event.
func TestFinalizeBlockValidatorUpdates(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -478,7 +468,7 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
blockExec.SetEventBus(eventBus)
updatesSub, err := eventBus.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "TestEndBlockValidatorUpdates",
ClientID: "TestFinalizeBlockValidatorUpdates",
Query: types.EventQueryValidatorSetUpdates,
})
require.NoError(t, err)
@ -519,9 +509,9 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
}
}
// TestEndBlockValidatorUpdatesResultingInEmptySet checks that processing validator updates that
// TestFinalizeBlockValidatorUpdatesResultingInEmptySet checks that processing validator updates that
// would result in empty set causes no panic, an error is raised and NextValidators is not updated
func TestEndBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
func TestFinalizeBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()


+ 0
- 24
internal/state/export_test.go View File

@ -2,33 +2,9 @@ package state
import (
abci "github.com/tendermint/tendermint/abci/types"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
"github.com/tendermint/tendermint/types"
)
//
// TODO: Remove dependence on all entities exported from this file.
//
// Every entity exported here is dependent on a private entity from the `state`
// package. Currently, these functions are only made available to tests in the
// `state_test` package, but we should not be relying on them for our testing.
// Instead, we should be exclusively relying on exported entities for our
// testing, and should be refactoring exported entities to make them more
// easily testable from outside of the package.
//
// UpdateState is an alias for updateState exported from execution.go,
// exclusively and explicitly for testing.
func UpdateState(
state State,
blockID types.BlockID,
header *types.Header,
abciResponses *tmstate.ABCIResponses,
validatorUpdates []*types.Validator,
) (State, error) {
return updateState(state, blockID, header, abciResponses, validatorUpdates)
}
// ValidateValidatorUpdates is an alias for validateValidatorUpdates exported
// from execution.go, exclusively and explicitly for testing.
func ValidateValidatorUpdates(abciUpdates []abci.ValidatorUpdate, params types.ValidatorParams) error {


+ 9
- 12
internal/state/helpers_test.go View File

@ -155,9 +155,7 @@ func makeHeaderPartsResponsesValPubKeyChange(
block, err := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{ValidatorUpdates: nil},
}
abciResponses := &tmstate.ABCIResponses{}
// If the pubkey is new, remove the old and add the new.
_, val := state.NextValidators.GetByIndex(0)
if !bytes.Equal(pubkey.Bytes(), val.PubKey.Bytes()) {
@ -187,10 +185,9 @@ func makeHeaderPartsResponsesValPowerChange(
block, err := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{ValidatorUpdates: nil},
}
abciResponses := &tmstate.ABCIResponses{}
abciResponses.FinalizeBlock = &abci.ResponseFinalizeBlock{}
// If the pubkey is new, remove the old and add the new.
_, val := state.NextValidators.GetByIndex(0)
if val.VotingPower != power {
@ -296,15 +293,15 @@ func (app *testApp) Info(req abci.RequestInfo) (resInfo abci.ResponseInfo) {
}
func (app *testApp) FinalizeBlock(req abci.RequestFinalizeBlock) abci.ResponseFinalizeBlock {
app.CommitVotes = req.LastCommitInfo.Votes
app.CommitVotes = req.DecidedLastCommit.Votes
app.ByzantineValidators = req.ByzantineValidators
resTxs := make([]*abci.ResponseDeliverTx, len(req.Txs))
resTxs := make([]*abci.ExecTxResult, len(req.Txs))
for i, tx := range req.Txs {
if len(tx) > 0 {
resTxs[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK}
resTxs[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK}
} else {
resTxs[i] = &abci.ResponseDeliverTx{Code: abci.CodeTypeOK + 10} // error
resTxs[i] = &abci.ExecTxResult{Code: abci.CodeTypeOK + 10} // error
}
}
@ -315,8 +312,8 @@ func (app *testApp) FinalizeBlock(req abci.RequestFinalizeBlock) abci.ResponseFi
AppVersion: 1,
},
},
Events: []abci.Event{},
Txs: resTxs,
Events: []abci.Event{},
TxResults: resTxs,
}
}


+ 10
- 11
internal/state/indexer/block/kv/kv.go View File

@ -20,7 +20,7 @@ import (
var _ indexer.BlockIndexer = (*BlockerIndexer)(nil)
// BlockerIndexer implements a block indexer, indexing BeginBlock and EndBlock
// BlockerIndexer implements a block indexer, indexing FinalizeBlock
// events with an underlying KV store. Block events are indexed by their height,
// such that matching search criteria returns the respective block height(s).
type BlockerIndexer struct {
@ -44,12 +44,11 @@ func (idx *BlockerIndexer) Has(height int64) (bool, error) {
return idx.store.Has(key)
}
// Index indexes BeginBlock and EndBlock events for a given block by its height.
// Index indexes FinalizeBlock events for a given block by its height.
// The following is indexed:
//
// primary key: encode(block.height | height) => encode(height)
// BeginBlock events: encode(eventType.eventAttr|eventValue|height|begin_block) => encode(height)
// EndBlock events: encode(eventType.eventAttr|eventValue|height|end_block) => encode(height)
// FinalizeBlock events: encode(eventType.eventAttr|eventValue|height|finalize_block) => encode(height)
func (idx *BlockerIndexer) Index(bh types.EventDataNewBlockHeader) error {
batch := idx.store.NewBatch()
defer batch.Close()
@ -65,19 +64,19 @@ func (idx *BlockerIndexer) Index(bh types.EventDataNewBlockHeader) error {
return err
}
// 2. index BeginBlock events
if err := idx.indexEvents(batch, bh.ResultFinalizeBlock.Events, "finalize_block", height); err != nil {
// 2. index FinalizeBlock events
if err := idx.indexEvents(batch, bh.ResultFinalizeBlock.Events, types.EventTypeFinalizeBlock, height); err != nil {
return fmt.Errorf("failed to index FinalizeBlock events: %w", err)
}
return batch.WriteSync()
}
// Search performs a query for block heights that match a given BeginBlock
// and Endblock event search criteria. The given query can match against zero,
// one or more block heights. In the case of height queries, i.e. block.height=H,
// if the height is indexed, that height alone will be returned. An error and
// nil slice is returned. Otherwise, a non-nil slice and nil error is returned.
// Search performs a query for block heights that match a given FinalizeBlock
// The given query can match against zero or more block heights. In the case
// of height queries, i.e. block.height=H, if the height is indexed, that height
// alone will be returned. An error and nil slice is returned. Otherwise, a
// non-nil slice and nil error is returned.
func (idx *BlockerIndexer) Search(ctx context.Context, q *query.Query) ([]int64, error) {
results := make([]int64, 0)
select {


+ 6
- 6
internal/state/indexer/block/kv/kv_test.go View File

@ -92,19 +92,19 @@ func TestBlockIndexer(t *testing.T) {
q: query.MustCompile(`block.height = 5`),
results: []int64{5},
},
"begin_event.key1 = 'value1'": {
"finalize_event.key1 = 'value1'": {
q: query.MustCompile(`finalize_event1.key1 = 'value1'`),
results: []int64{},
},
"begin_event.proposer = 'FCAA001'": {
"finalize_event.proposer = 'FCAA001'": {
q: query.MustCompile(`finalize_event1.proposer = 'FCAA001'`),
results: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11},
},
"end_event.foo <= 5": {
"finalize_event.foo <= 5": {
q: query.MustCompile(`finalize_event2.foo <= 5`),
results: []int64{2, 4},
},
"end_event.foo >= 100": {
"finalize_event.foo >= 100": {
q: query.MustCompile(`finalize_event2.foo >= 100`),
results: []int64{1},
},
@ -112,11 +112,11 @@ func TestBlockIndexer(t *testing.T) {
q: query.MustCompile(`block.height > 2 AND finalize_event2.foo <= 8`),
results: []int64{4, 6, 8},
},
"begin_event.proposer CONTAINS 'FFFFFFF'": {
"finalize_event.proposer CONTAINS 'FFFFFFF'": {
q: query.MustCompile(`finalize_event1.proposer CONTAINS 'FFFFFFF'`),
results: []int64{},
},
"begin_event.proposer CONTAINS 'FCAA001'": {
"finalize_event.proposer CONTAINS 'FCAA001'": {
q: query.MustCompile(`finalize_event1.proposer CONTAINS 'FCAA001'`),
results: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11},
},


+ 3
- 3
internal/state/indexer/indexer.go View File

@ -30,11 +30,11 @@ type BlockIndexer interface {
// upon database query failure.
Has(height int64) (bool, error)
// Index indexes BeginBlock and EndBlock events for a given block by its height.
// Index indexes FinalizeBlock events for a given block by its height.
Index(types.EventDataNewBlockHeader) error
// Search performs a query for block heights that match a given BeginBlock
// and Endblock event search criteria.
// Search performs a query for block heights that match a given FinalizeBlock
// event search criteria.
Search(ctx context.Context, q *query.Query) ([]int64, error)
}


+ 2
- 2
internal/state/indexer/indexer_service_test.go View File

@ -80,7 +80,7 @@ func TestIndexerServiceIndexesBlocks(t *testing.T) {
Height: 1,
Index: uint32(0),
Tx: types.Tx("foo"),
Result: abci.ResponseDeliverTx{Code: 0},
Result: abci.ExecTxResult{Code: 0},
}
err = eventBus.PublishEventTx(ctx, types.EventDataTx{TxResult: *txResult1})
require.NoError(t, err)
@ -88,7 +88,7 @@ func TestIndexerServiceIndexesBlocks(t *testing.T) {
Height: 1,
Index: uint32(1),
Tx: types.Tx("bar"),
Result: abci.ResponseDeliverTx{Code: 0},
Result: abci.ExecTxResult{Code: 0},
}
err = eventBus.PublishEventTx(ctx, types.EventDataTx{TxResult: *txResult2})
require.NoError(t, err)


+ 1
- 1
internal/state/indexer/sink/kv/kv_test.go View File

@ -338,7 +338,7 @@ func txResultWithEvents(events []abci.Event) *abci.TxResult {
Height: 1,
Index: 0,
Tx: tx,
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK,
Log: "",


+ 2
- 22
internal/state/indexer/sink/psql/psql_test.go View File

@ -46,8 +46,7 @@ const (
dbName = "postgres"
chainID = "test-chainID"
viewBlockEvents = "block_events"
viewTxEvents = "tx_events"
viewTxEvents = "tx_events"
)
func TestMain(m *testing.M) {
@ -266,7 +265,7 @@ func txResultWithEvents(events []abci.Event) *abci.TxResult {
Height: 1,
Index: 0,
Tx: types.Tx("HELLO WORLD"),
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK,
Log: "",
@ -309,25 +308,6 @@ SELECT height FROM `+tableBlocks+` WHERE height = $1;
} else if err != nil {
t.Fatalf("Database query failed: %v", err)
}
// Verify the presence of begin_block and end_block events.
if err := testDB().QueryRow(`
SELECT type, height, chain_id FROM `+viewBlockEvents+`
WHERE height = $1 AND type = $2 AND chain_id = $3;
`, height, types.EventTypeBeginBlock, chainID).Err(); err == sql.ErrNoRows {
t.Errorf("No %q event found for height=%d", types.EventTypeBeginBlock, height)
} else if err != nil {
t.Fatalf("Database query failed: %c", err)
}
if err := testDB().QueryRow(`
SELECT type, height, chain_id FROM `+viewBlockEvents+`
WHERE height = $1 AND type = $2 AND chain_id = $3;
`, height, types.EventTypeEndBlock, chainID).Err(); err == sql.ErrNoRows {
t.Errorf("No %q event found for height=%d", types.EventTypeEndBlock, height)
} else if err != nil {
t.Fatalf("Database query failed: %v", err)
}
}
// verifyNotImplemented calls f and verifies that it returns both a


+ 1
- 1
internal/state/indexer/tx/kv/kv_bench_test.go View File

@ -43,7 +43,7 @@ func BenchmarkTxSearch(b *testing.B) {
Height: int64(i),
Index: 0,
Tx: types.Tx(string(txBz)),
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK,
Log: "",


+ 4
- 4
internal/state/indexer/tx/kv/kv_test.go View File

@ -25,7 +25,7 @@ func TestTxIndex(t *testing.T) {
Height: 1,
Index: 0,
Tx: tx,
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK, Log: "", Events: nil,
},
@ -48,7 +48,7 @@ func TestTxIndex(t *testing.T) {
Height: 1,
Index: 0,
Tx: tx2,
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK, Log: "", Events: nil,
},
@ -322,7 +322,7 @@ func txResultWithEvents(events []abci.Event) *abci.TxResult {
Height: 1,
Index: 0,
Tx: tx,
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK,
Log: "",
@ -346,7 +346,7 @@ func benchmarkTxIndex(txsCount int64, b *testing.B) {
Height: 1,
Index: txIndex,
Tx: tx,
Result: abci.ResponseDeliverTx{
Result: abci.ExecTxResult{
Data: []byte{0},
Code: abci.CodeTypeOK,
Log: "",


+ 1
- 1
internal/state/state.go View File

@ -91,7 +91,7 @@ type State struct {
LastHeightValidatorsChanged int64
// Consensus parameters used for validating blocks.
// Changes returned by EndBlock and updated after Commit.
// Changes returned by FinalizeBlock and updated after Commit.
ConsensusParams types.ConsensusParams
LastHeightConsensusParamsChanged int64


+ 31
- 31
internal/state/state_test.go View File

@ -107,12 +107,12 @@ func TestABCIResponsesSaveLoad1(t *testing.T) {
require.NoError(t, err)
abciResponses := new(tmstate.ABCIResponses)
dtxs := make([]*abci.ResponseDeliverTx, 2)
dtxs := make([]*abci.ExecTxResult, 2)
abciResponses.FinalizeBlock = new(abci.ResponseFinalizeBlock)
abciResponses.FinalizeBlock.Txs = dtxs
abciResponses.FinalizeBlock.TxResults = dtxs
abciResponses.FinalizeBlock.Txs[0] = &abci.ResponseDeliverTx{Data: []byte("foo"), Events: nil}
abciResponses.FinalizeBlock.Txs[1] = &abci.ResponseDeliverTx{Data: []byte("bar"), Log: "ok", Events: nil}
abciResponses.FinalizeBlock.TxResults[0] = &abci.ExecTxResult{Data: []byte("foo"), Events: nil}
abciResponses.FinalizeBlock.TxResults[1] = &abci.ExecTxResult{Data: []byte("bar"), Log: "ok", Events: nil}
pbpk, err := encoding.PubKeyToProto(ed25519.GenPrivKey().PubKey())
require.NoError(t, err)
abciResponses.FinalizeBlock.ValidatorUpdates = []abci.ValidatorUpdate{{PubKey: pbpk, Power: 10}}
@ -136,23 +136,23 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
cases := [...]struct {
// Height is implied to equal index+2,
// as block 1 is created from genesis.
added []*abci.ResponseDeliverTx
expected []*abci.ResponseDeliverTx
added []*abci.ExecTxResult
expected []*abci.ExecTxResult
}{
0: {
nil,
nil,
},
1: {
[]*abci.ResponseDeliverTx{
[]*abci.ExecTxResult{
{Code: 32, Data: []byte("Hello"), Log: "Huh?"},
},
[]*abci.ResponseDeliverTx{
[]*abci.ExecTxResult{
{Code: 32, Data: []byte("Hello")},
},
},
2: {
[]*abci.ResponseDeliverTx{
[]*abci.ExecTxResult{
{Code: 383},
{
Data: []byte("Gotcha!"),
@ -162,7 +162,7 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
},
},
},
[]*abci.ResponseDeliverTx{
[]*abci.ExecTxResult{
{Code: 383, Data: nil},
{Code: 0, Data: []byte("Gotcha!"), Events: []abci.Event{
{Type: "type1", Attributes: []abci.EventAttribute{{Key: "a", Value: "1"}}},
@ -175,7 +175,7 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
nil,
},
4: {
[]*abci.ResponseDeliverTx{nil},
[]*abci.ExecTxResult{nil},
nil,
},
}
@ -192,7 +192,7 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
h := int64(i + 1) // last block height, one below what we save
responses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
Txs: tc.added,
TxResults: tc.added,
},
}
err := stateStore.SaveABCIResponses(h, responses)
@ -207,7 +207,7 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
t.Log(res)
responses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
Txs: tc.expected,
TxResults: tc.expected,
},
}
sm.ABCIResponsesResultsHash(res)
@ -278,7 +278,7 @@ func TestOneValidatorChangesSaveLoad(t *testing.T) {
header, blockID, responses := makeHeaderPartsResponsesValPowerChange(t, state, power)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = sm.UpdateState(state, blockID, &header, responses, validatorUpdates)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
err := stateStore.Save(state)
require.NoError(t, err)
@ -463,7 +463,7 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
updatedState, err := sm.UpdateState(state, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err := state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
curTotal := val1VotingPower
// one increment step and one validator: 0 + power - total_power == 0
@ -478,7 +478,7 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
updateAddVal := abci.ValidatorUpdate{PubKey: fvp, Power: val2VotingPower}
validatorUpdates, err = types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{updateAddVal})
assert.NoError(t, err)
updatedState2, err := sm.UpdateState(updatedState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState2, err := updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState2.NextValidators.Validators), 2)
@ -517,7 +517,7 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
// this will cause the diff of priorities (77)
// to be larger than threshold == 2*totalVotingPower (22):
updatedState3, err := sm.UpdateState(updatedState2, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState3, err := updatedState2.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState3.NextValidators.Validators), 2)
@ -583,7 +583,7 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
updatedState, err := sm.UpdateState(state, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err := state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
// 0 + 10 (initial prio) - 10 (avg) - 10 (mostest - total) = -10
@ -600,7 +600,7 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err = types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{updateAddVal})
assert.NoError(t, err)
updatedState2, err := sm.UpdateState(updatedState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState2, err := updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState2.NextValidators.Validators), 2)
@ -643,7 +643,7 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
updatedState3, err := sm.UpdateState(updatedState2, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState3, err := updatedState2.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
assert.Equal(t, updatedState3.Validators.Proposer.Address, updatedState3.NextValidators.Proposer.Address)
@ -687,7 +687,7 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
oldState, err = sm.UpdateState(oldState, blockID, &block.Header, abciResponses, validatorUpdates)
oldState, err = oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
expectedVal1Prio2 = 1
expectedVal2Prio2 = -1
@ -704,7 +704,7 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
updatedState, err := sm.UpdateState(oldState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
// alternate (and cyclic priorities):
assert.NotEqual(
@ -769,7 +769,7 @@ func TestLargeGenesisValidator(t *testing.T) {
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err := sm.UpdateState(oldState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
// no changes in voting power (ProposerPrio += VotingPower == Voting in 1st round; than shiftByAvg == 0,
// than -Total == -Voting)
@ -803,7 +803,7 @@ func TestLargeGenesisValidator(t *testing.T) {
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err := sm.UpdateState(oldState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
lastState := updatedState
@ -825,7 +825,7 @@ func TestLargeGenesisValidator(t *testing.T) {
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedStateInner, err := sm.UpdateState(lastState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedStateInner, err := lastState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
lastState = updatedStateInner
}
@ -862,7 +862,7 @@ func TestLargeGenesisValidator(t *testing.T) {
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
state, err = sm.UpdateState(state, blockID, &block.Header, abciResponses, validatorUpdates)
state, err = state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
}
require.Equal(t, 10+2, len(state.NextValidators.Validators))
@ -886,7 +886,7 @@ func TestLargeGenesisValidator(t *testing.T) {
blockID = types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
updatedState, err = sm.UpdateState(state, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err = state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
// only the first added val (not the genesis val) should be left
assert.Equal(t, 11, len(updatedState.NextValidators.Validators))
@ -911,7 +911,7 @@ func TestLargeGenesisValidator(t *testing.T) {
require.NoError(t, err)
blockID = types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
curState, err = sm.UpdateState(curState, blockID, &block.Header, abciResponses, validatorUpdates)
curState, err = curState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
if !bytes.Equal(curState.Validators.Proposer.Address, curState.NextValidators.Proposer.Address) {
isProposerUnchanged = false
@ -943,7 +943,7 @@ func TestLargeGenesisValidator(t *testing.T) {
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err = sm.UpdateState(updatedState, blockID, &block.Header, abciResponses, validatorUpdates)
updatedState, err = updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
if i > numVals { // expect proposers to cycle through after the first iteration (of numVals blocks):
if proposers[i%numVals] == nil {
@ -1002,7 +1002,7 @@ func TestManyValidatorChangesSaveLoad(t *testing.T) {
var validatorUpdates []*types.Validator
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = sm.UpdateState(state, blockID, &header, responses, validatorUpdates)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
nextHeight := state.LastBlockHeight + 1
err = stateStore.Save(state)
@ -1080,7 +1080,7 @@ func TestConsensusParamsChangesSaveLoad(t *testing.T) {
header, blockID, responses := makeHeaderPartsResponsesParams(t, state, &cp)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = sm.UpdateState(state, blockID, &header, responses, validatorUpdates)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
err := stateStore.Save(state)


+ 4
- 4
internal/state/store.go View File

@ -401,7 +401,7 @@ func (store dbStore) reverseBatchDelete(batch dbm.Batch, start, end []byte) ([]b
//
// See merkle.SimpleHashFromByteSlices
func ABCIResponsesResultsHash(ar *tmstate.ABCIResponses) []byte {
return types.NewResults(ar.FinalizeBlock.Txs).Hash()
return types.NewResults(ar.FinalizeBlock.TxResults).Hash()
}
// LoadABCIResponses loads the ABCIResponses for the given height from the
@ -442,15 +442,15 @@ func (store dbStore) SaveABCIResponses(height int64, abciResponses *tmstate.ABCI
}
func (store dbStore) saveABCIResponses(height int64, abciResponses *tmstate.ABCIResponses) error {
var dtxs []*abci.ResponseDeliverTx
var dtxs []*abci.ExecTxResult
// strip nil values,
for _, tx := range abciResponses.FinalizeBlock.Txs {
for _, tx := range abciResponses.FinalizeBlock.TxResults {
if tx != nil {
dtxs = append(dtxs, tx)
}
}
abciResponses.FinalizeBlock.Txs = dtxs
abciResponses.FinalizeBlock.TxResults = dtxs
bz, err := abciResponses.Marshal()
if err != nil {


+ 3
- 3
internal/state/store_test.go View File

@ -239,7 +239,7 @@ func TestPruneStates(t *testing.T) {
err = stateStore.SaveABCIResponses(h, &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
Txs: []*abci.ResponseDeliverTx{
TxResults: []*abci.ExecTxResult{
{Data: []byte{1}},
{Data: []byte{2}},
{Data: []byte{3}},
@ -303,7 +303,7 @@ func TestPruneStates(t *testing.T) {
func TestABCIResponsesResultsHash(t *testing.T) {
responses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
Txs: []*abci.ResponseDeliverTx{
TxResults: []*abci.ExecTxResult{
{Code: 32, Data: []byte("Hello"), Log: "Huh?"},
},
},
@ -312,7 +312,7 @@ func TestABCIResponsesResultsHash(t *testing.T) {
root := sm.ABCIResponsesResultsHash(responses)
// root should be Merkle tree root of FinalizeBlock tx responses
results := types.NewResults(responses.FinalizeBlock.Txs)
results := types.NewResults(responses.FinalizeBlock.TxResults)
assert.Equal(t, root, results.Hash())
// test we can prove first tx in FinalizeBlock


+ 2
- 2
light/detector.go View File

@ -39,8 +39,8 @@ func (c *Client) detectDivergence(ctx context.Context, primaryTrace []*types.Lig
lastVerifiedHeader = primaryTrace[len(primaryTrace)-1].SignedHeader
witnessesToRemove = make([]int, 0)
)
c.logger.Debug("running detector against trace", "endBlockHeight", lastVerifiedHeader.Height,
"endBlockHash", lastVerifiedHeader.Hash, "length", len(primaryTrace))
c.logger.Debug("running detector against trace", "finalizeBlockHeight", lastVerifiedHeader.Height,
"finalizeBlockHash", lastVerifiedHeader.Hash, "length", len(primaryTrace))
// launch one goroutine per witness to retrieve the light block of the target height
// and compare it with the header from the primary


+ 1
- 1
light/rpc/client.go View File

@ -461,7 +461,7 @@ func (c *Client) BlockResults(ctx context.Context, height *int64) (*coretypes.Re
// Build a Merkle tree of proto-encoded FinalizeBlock tx results and get a hash.
results := types.NewResults(res.TxsResults)
// Build a Merkle tree out of the above 3 binary slices.
// Build a Merkle tree out of the slice.
rH := merkle.HashFromByteSlices([][]byte{bbeBytes, results.Hash()})
// Verify block results.


+ 2
- 2
node/node.go View File

@ -162,8 +162,8 @@ func makeNode(
// EventBus and IndexerService must be started before the handshake because
// we might need to index the txs of the replayed block as this might not have happened
// when the node stopped last time (i.e. the node stopped after it saved the block
// but before it indexed the txs, or, endblocker panicked)
// when the node stopped last time (i.e. the node stopped or crashed after it saved the block
// but before it indexed the txs)
eventBus := eventbus.NewDefault(logger.With("module", "events"))
if err := eventBus.Start(ctx); err != nil {
return nil, combineCloseError(err, makeCloser(closers))


+ 2
- 2
proto/tendermint/abci/types.proto View File

@ -342,7 +342,7 @@ message ResponseVerifyVoteExtension {
}
message ResponseFinalizeBlock {
repeated Event block_events = 1
repeated Event events = 1
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"];
repeated ExecTxResult tx_results = 2;
repeated ValidatorUpdate validator_updates = 3;
@ -389,7 +389,7 @@ message ExecTxResult {
string info = 4; // nondeterministic
int64 gas_wanted = 5;
int64 gas_used = 6;
repeated Event tx_events = 7
repeated Event events = 7
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"]; // nondeterministic
string codespace = 8;
}


+ 16
- 15
proto/tendermint/abci/types.proto.intermediate View File

@ -77,7 +77,7 @@ message RequestQuery {
message RequestBeginBlock {
bytes hash = 1;
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
LastCommitInfo last_commit_info = 3 [(gogoproto.nullable) = false];
CommitInfo last_commit_info = 3 [(gogoproto.nullable) = false];
repeated Evidence byzantine_validators = 4 [(gogoproto.nullable) = false];
}
@ -151,17 +151,16 @@ message RequestProcessProposal {
bytes hash = 1;
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
repeated bytes txs = 3;
LastCommitInfo last_commit_info = 4 [(gogoproto.nullable) = false];
CommitInfo proposed_last_commit = 4 [(gogoproto.nullable) = false];
repeated Evidence byzantine_validators = 5 [(gogoproto.nullable) = false];
}
message RequestFinalizeBlock {
repeated bytes txs = 1;
bytes hash = 2;
int64 height = 3;
tendermint.types.Header header = 4 [(gogoproto.nullable) = false];
LastCommitInfo last_commit_info = 5 [(gogoproto.nullable) = false];
repeated Evidence byzantine_validators = 6 [(gogoproto.nullable) = false];
bytes hash = 1;
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
repeated bytes txs = 3;
CommitInfo decided_last_commit = 4 [(gogoproto.nullable) = false];
repeated Evidence byzantine_validators = 5 [(gogoproto.nullable) = false];
}
//----------------------------------------
@ -343,17 +342,19 @@ message ResponseProcessProposal {
}
message ResponseFinalizeBlock {
repeated ResponseDeliverTx txs = 1;
repeated ValidatorUpdate validator_updates = 2 [(gogoproto.nullable) = false];
tendermint.types.ConsensusParams consensus_param_updates = 3;
repeated Event events = 4
repeated Event events = 1
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"];
repeated ExecTxResult tx_results = 2;
repeated ValidatorUpdate validator_updates = 3 [(gogoproto.nullable) = false];
tendermint.types.ConsensusParams consensus_param_updates = 4;
bytes app_hash = 5;
int64 retain_height = 6;
}
//----------------------------------------
// Misc.
message LastCommitInfo {
message CommitInfo {
int32 round = 1;
repeated VoteInfo votes = 2 [(gogoproto.nullable) = false];
}
@ -383,7 +384,7 @@ message ExecTxResult {
string info = 4; // nondeterministic
int64 gas_wanted = 5;
int64 gas_used = 6;
repeated Event tx_events = 7
repeated Event events = 7
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"]; // nondeterministic
string codespace = 8;
}
@ -395,7 +396,7 @@ message TxResult {
int64 height = 1;
uint32 index = 2;
bytes tx = 3;
ResponseDeliverTx result = 4 [(gogoproto.nullable) = false];
ExecTxResult result = 4 [(gogoproto.nullable) = false];
}
//----------------------------------------


+ 1
- 1
rpc/client/examples_test.go View File

@ -49,7 +49,7 @@ func TestHTTPSimple(t *testing.T) {
if err != nil {
log.Fatal(err)
}
if bres.CheckTx.IsErr() || bres.DeliverTx.IsErr() {
if bres.CheckTx.IsErr() || bres.TxResult.IsErr() {
log.Fatal("BroadcastTxCommit transaction failed")
}


+ 1
- 1
rpc/client/interface.go View File

@ -90,7 +90,7 @@ type SignClient interface {
) (*coretypes.ResultTxSearch, error)
// BlockSearch defines a method to search for a paginated set of blocks by
// BeginBlock and EndBlock event search criteria.
// FinalizeBlock event search criteria.
BlockSearch(
ctx context.Context,
query string,


+ 1
- 1
rpc/client/mock/abci.go View File

@ -56,7 +56,7 @@ func (a ABCIApp) BroadcastTxCommit(ctx context.Context, tx types.Tx) (*coretypes
return &res, nil
}
fb := a.App.FinalizeBlock(abci.RequestFinalizeBlock{Txs: [][]byte{tx}})
res.DeliverTx = *fb.Txs[0]
res.TxResult = *fb.TxResults[0]
res.Height = -1 // TODO
return &res, nil
}


+ 5
- 5
rpc/client/mock/abci_test.go View File

@ -38,8 +38,8 @@ func TestABCIMock(t *testing.T) {
BroadcastCommit: mock.Call{
Args: goodTx,
Response: &coretypes.ResultBroadcastTxCommit{
CheckTx: abci.ResponseCheckTx{Data: bytes.HexBytes("stand")},
DeliverTx: abci.ResponseDeliverTx{Data: bytes.HexBytes("deliver")},
CheckTx: abci.ResponseCheckTx{Data: bytes.HexBytes("stand")},
TxResult: abci.ExecTxResult{Data: bytes.HexBytes("deliver")},
},
Error: errors.New("bad tx"),
},
@ -76,7 +76,7 @@ func TestABCIMock(t *testing.T) {
require.NoError(t, err, "%+v", err)
assert.EqualValues(t, 0, bres.CheckTx.Code)
assert.EqualValues(t, "stand", bres.CheckTx.Data)
assert.EqualValues(t, "deliver", bres.DeliverTx.Data)
assert.EqualValues(t, "deliver", bres.TxResult.Data)
}
func TestABCIRecorder(t *testing.T) {
@ -179,8 +179,8 @@ func TestABCIApp(t *testing.T) {
res, err := m.BroadcastTxCommit(ctx, types.Tx(tx))
require.NoError(t, err)
assert.True(t, res.CheckTx.IsOK())
require.NotNil(t, res.DeliverTx)
assert.True(t, res.DeliverTx.IsOK())
require.NotNil(t, res.TxResult)
assert.True(t, res.TxResult.IsOK())
// commit
// TODO: This may not be necessary in the future


+ 2
- 2
rpc/client/rpc_test.go View File

@ -324,7 +324,7 @@ func TestClientMethodCalls(t *testing.T) {
k, v, tx := MakeTxKV()
bres, err := c.BroadcastTxCommit(ctx, tx)
require.NoError(t, err)
require.True(t, bres.DeliverTx.IsOK())
require.True(t, bres.TxResult.IsOK())
txh := bres.Height
apph := txh + 1 // this is where the tx will be applied to the state
@ -443,7 +443,7 @@ func TestClientMethodCalls(t *testing.T) {
bres, err := c.BroadcastTxCommit(ctx, tx)
require.NoError(t, err, "%d: %+v", i, err)
require.True(t, bres.CheckTx.IsOK())
require.True(t, bres.DeliverTx.IsOK())
require.True(t, bres.TxResult.IsOK())
require.Equal(t, 0, pool.Size())
})


+ 16
- 16
rpc/coretypes/responses.go View File

@ -65,12 +65,12 @@ type ResultCommit struct {
// ABCI results from a block
type ResultBlockResults struct {
Height int64 `json:"height,string"`
TxsResults []*abci.ResponseDeliverTx `json:"txs_results"`
TotalGasUsed int64 `json:"total_gas_used,string"`
FinalizeBlockEvents []abci.Event `json:"finalize_block_events"`
ValidatorUpdates []abci.ValidatorUpdate `json:"validator_updates"`
ConsensusParamUpdates *tmproto.ConsensusParams `json:"consensus_param_updates"`
Height int64 `json:"height,string"`
TxsResults []*abci.ExecTxResult `json:"txs_results"`
TotalGasUsed int64 `json:"total_gas_used,string"`
FinalizeBlockEvents []abci.Event `json:"finalize_block_events"`
ValidatorUpdates []abci.ValidatorUpdate `json:"validator_updates"`
ConsensusParamUpdates *tmproto.ConsensusParams `json:"consensus_param_updates"`
}
// NewResultCommit is a helper to initialize the ResultCommit with
@ -241,10 +241,10 @@ type ResultBroadcastTx struct {
// CheckTx and DeliverTx results
type ResultBroadcastTxCommit struct {
CheckTx abci.ResponseCheckTx `json:"check_tx"`
DeliverTx abci.ResponseDeliverTx `json:"deliver_tx"`
Hash bytes.HexBytes `json:"hash"`
Height int64 `json:"height,string"`
CheckTx abci.ResponseCheckTx `json:"check_tx"`
TxResult abci.ExecTxResult `json:"tx_result"`
Hash bytes.HexBytes `json:"hash"`
Height int64 `json:"height,string"`
}
// ResultCheckTx wraps abci.ResponseCheckTx.
@ -254,12 +254,12 @@ type ResultCheckTx struct {
// Result of querying for a tx
type ResultTx struct {
Hash bytes.HexBytes `json:"hash"`
Height int64 `json:"height,string"`
Index uint32 `json:"index"`
TxResult abci.ResponseDeliverTx `json:"tx_result"`
Tx types.Tx `json:"tx"`
Proof types.TxProof `json:"proof,omitempty"`
Hash bytes.HexBytes `json:"hash"`
Height int64 `json:"height,string"`
Index uint32 `json:"index"`
TxResult abci.ExecTxResult `json:"tx_result"`
Tx types.Tx `json:"tx"`
Proof types.TxProof `json:"proof,omitempty"`
}
// Result of searching for txs


+ 1
- 1
spec/abci++/abci++_basic_concepts_002_draft.md View File

@ -99,7 +99,7 @@ returned directly to the client that initiated the query.
Method `CheckTx` includes an `Events` field in its `Response*`.
Method `FinalizeBlock` includes an `Events` field at the top level in its
`Response*`, and one `tx_events` field per transaction included in the block.
`Response*`, and one `events` field per transaction included in the block.
Applications may respond to these ABCI++ methods with a set of events.
Events allow applications to associate metadata about ABCI++ method execution with the
transactions and blocks this metadata relates to.


+ 2
- 2
spec/abci++/abci++_methods_002_draft.md View File

@ -588,7 +588,7 @@ from this condition, but not sure), and _p_ receives a Precommit message for rou
| Name | Type | Description | Field Number |
|-------------------------|-------------------------------------------------------------|----------------------------------------------------------------------------------|--------------|
| block_events | repeated [Event](abci++_basic_concepts_002_draft.md#events) | Type & Key-Value events for indexing | 1 |
| events | repeated [Event](abci++_basic_concepts_002_draft.md#events) | Type & Key-Value events for indexing | 1 |
| tx_results | repeated [ExecTxResult](#txresult) | List of structures containing the data resulting from executing the transactions | 2 |
| validator_updates | repeated [ValidatorUpdate](#validatorupdate) | Changes to validator set (set voting power to 0 to remove). | 3 |
| consensus_param_updates | [ConsensusParams](#consensusparams) | Changes to consensus-critical gas, size, and other parameters. | 4 |
@ -832,7 +832,7 @@ Most of the data structures used in ABCI are shared [common data structures](../
| info | string | Additional information. **May be non-deterministic.** | 4 |
| gas_wanted | int64 | Amount of gas requested for transaction. | 5 |
| gas_used | int64 | Amount of gas consumed by transaction. | 6 |
| tx_events | repeated [Event](abci++_basic_concepts_002_draft.md#events) | Type & Key-Value events for indexing transactions (e.g. by account). | 7 |
| events | repeated [Event](abci++_basic_concepts_002_draft.md#events) | Type & Key-Value events for indexing transactions (e.g. by account). | 7 |
| codespace | string | Namespace for the `code`. | 8 |
### TxAction


+ 5
- 5
test/e2e/app/app.go View File

@ -155,7 +155,7 @@ func (app *Application) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx {
// FinalizeBlock implements ABCI.
func (app *Application) FinalizeBlock(req abci.RequestFinalizeBlock) abci.ResponseFinalizeBlock {
var txs = make([]*abci.ResponseDeliverTx, len(req.Txs))
var txs = make([]*abci.ExecTxResult, len(req.Txs))
app.mu.Lock()
defer app.mu.Unlock()
@ -167,16 +167,16 @@ func (app *Application) FinalizeBlock(req abci.RequestFinalizeBlock) abci.Respon
}
app.state.Set(key, value)
txs[i] = &abci.ResponseDeliverTx{Code: code.CodeTypeOK}
txs[i] = &abci.ExecTxResult{Code: code.CodeTypeOK}
}
valUpdates, err := app.validatorUpdates(uint64(req.Height))
valUpdates, err := app.validatorUpdates(uint64(req.Header.Height))
if err != nil {
panic(err)
}
return abci.ResponseFinalizeBlock{
Txs: txs,
TxResults: txs,
ValidatorUpdates: valUpdates,
Events: []abci.Event{
{
@ -188,7 +188,7 @@ func (app *Application) FinalizeBlock(req abci.RequestFinalizeBlock) abci.Respon
},
{
Key: "height",
Value: strconv.Itoa(int(req.Height)),
Value: strconv.Itoa(int(req.Header.Height)),
},
},
},


+ 1
- 0
tools/tools.go View File

@ -1,3 +1,4 @@
//go:build tools
// +build tools
// This file uses the recommended method for tracking developer tools in a go module.


+ 3
- 4
types/events.go View File

@ -270,12 +270,11 @@ const (
// see EventBus#PublishEventTx
TxHeightKey = "tx.height"
// BlockHeightKey is a reserved key used for indexing BeginBlock and Endblock
// events.
// BlockHeightKey is a reserved key used for indexing FinalizeBlock events.
BlockHeightKey = "block.height"
EventTypeBeginBlock = "begin_block"
EventTypeEndBlock = "end_block"
// EventTypeFinalizeBlock is a reserved key used for indexing FinalizeBlock events.
EventTypeFinalizeBlock = "finalize_block"
)
var (


+ 6
- 6
types/results.go View File

@ -6,14 +6,14 @@ import (
)
// ABCIResults wraps the deliver tx results to return a proof.
type ABCIResults []*abci.ResponseDeliverTx
type ABCIResults []*abci.ExecTxResult
// NewResults strips non-deterministic fields from ResponseDeliverTx responses
// and returns ABCIResults.
func NewResults(responses []*abci.ResponseDeliverTx) ABCIResults {
func NewResults(responses []*abci.ExecTxResult) ABCIResults {
res := make(ABCIResults, len(responses))
for i, d := range responses {
res[i] = deterministicResponseDeliverTx(d)
res[i] = deterministicExecTxResult(d)
}
return res
}
@ -42,10 +42,10 @@ func (a ABCIResults) toByteSlices() [][]byte {
return bzs
}
// deterministicResponseDeliverTx strips non-deterministic fields from
// deterministicExecTxResult strips non-deterministic fields from
// ResponseDeliverTx and returns another ResponseDeliverTx.
func deterministicResponseDeliverTx(response *abci.ResponseDeliverTx) *abci.ResponseDeliverTx {
return &abci.ResponseDeliverTx{
func deterministicExecTxResult(response *abci.ExecTxResult) *abci.ExecTxResult {
return &abci.ExecTxResult{
Code: response.Code,
Data: response.Data,
GasWanted: response.GasWanted,


+ 6
- 6
types/results_test.go View File

@ -10,12 +10,12 @@ import (
)
func TestABCIResults(t *testing.T) {
a := &abci.ResponseDeliverTx{Code: 0, Data: nil}
b := &abci.ResponseDeliverTx{Code: 0, Data: []byte{}}
c := &abci.ResponseDeliverTx{Code: 0, Data: []byte("one")}
d := &abci.ResponseDeliverTx{Code: 14, Data: nil}
e := &abci.ResponseDeliverTx{Code: 14, Data: []byte("foo")}
f := &abci.ResponseDeliverTx{Code: 14, Data: []byte("bar")}
a := &abci.ExecTxResult{Code: 0, Data: nil}
b := &abci.ExecTxResult{Code: 0, Data: []byte{}}
c := &abci.ExecTxResult{Code: 0, Data: []byte("one")}
d := &abci.ExecTxResult{Code: 14, Data: nil}
e := &abci.ExecTxResult{Code: 14, Data: []byte("foo")}
f := &abci.ExecTxResult{Code: 14, Data: []byte("bar")}
// Nil and []byte{} should produce the same bytes
bzA, err := a.Marshal()


+ 3
- 1
types/validation.go View File

@ -15,11 +15,13 @@ func shouldBatchVerify(vals *ValidatorSet, commit *Commit) bool {
return len(commit.Signatures) >= batchVerifyThreshold && batch.SupportsBatchVerifier(vals.GetProposer().PubKey)
}
// TODO(wbanfield): determine if the following comment is still true regarding Gaia.
// VerifyCommit verifies +2/3 of the set had signed the given commit.
//
// It checks all the signatures! While it's safe to exit as soon as we have
// 2/3+ signatures, doing so would impact incentivization logic in the ABCI
// application that depends on the LastCommitInfo sent in BeginBlock, which
// application that depends on the LastCommitInfo sent in FinalizeBlock, which
// includes which validators signed. For instance, Gaia incentivizes proposers
// with a bonus for including more than +2/3 of the signatures.
func VerifyCommit(chainID string, vals *ValidatorSet, blockID BlockID,


Loading…
Cancel
Save