Skip to content

Commit

Permalink
Merge pull request #2755 from jorgemmsilva/refactor/vm-panic-catches
Browse files Browse the repository at this point in the history
refactor(vm): simplify request panic catching
  • Loading branch information
jorgemmsilva authored Jul 27, 2023
2 parents 4cd6071 + d278a42 commit 7ab50c4
Show file tree
Hide file tree
Showing 4 changed files with 110 additions and 83 deletions.
41 changes: 41 additions & 0 deletions packages/vm/core/testcore/accounts_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/iotaledger/wasp/packages/isc"
"github.com/iotaledger/wasp/packages/kv/codec"
"github.com/iotaledger/wasp/packages/kv/dict"
"github.com/iotaledger/wasp/packages/origin"
"github.com/iotaledger/wasp/packages/parameters"
"github.com/iotaledger/wasp/packages/solo"
"github.com/iotaledger/wasp/packages/testutil/testmisc"
Expand Down Expand Up @@ -780,6 +781,46 @@ func TestWithdrawDepositNativeTokens(t *testing.T) {
v.ch.AssertL2NativeTokens(v.userAgentID, v.nativeTokenID, 0)
v.env.AssertL1NativeTokens(v.userAddr, v.nativeTokenID, 50)
})

t.Run("accounting UTXOs and pruning", func(t *testing.T) {
// mint 100 tokens from chain 1 and withdraw those to L1
v := initWithdrawTest(t, 2*isc.Million)
{
allSenderAssets := v.ch.L2Assets(v.userAgentID)
v.req.AddAllowance(allSenderAssets)
v.req.AddBaseTokens(BaseTokensDepositFee)
_, err := v.ch.PostRequestSync(v.req, v.user)
require.NoError(t, err)
v.env.AssertL1NativeTokens(v.userAddr, v.nativeTokenID, 100)
v.ch.AssertL2NativeTokens(v.userAgentID, v.nativeTokenID, 0)
}

// create a new chain (ch2) with active state pruning set to keep only 1 block
blockKeepAmount := int32(1)
ch2, _ := v.env.NewChainExt(nil, 0, "evmchain", dict.Dict{
origin.ParamBlockKeepAmount: codec.EncodeInt32(blockKeepAmount),
})

// deposit 1 native token from L1 into ch2
err := ch2.DepositAssetsToL2(isc.NewAssets(1*isc.Million, iotago.NativeTokens{
{ID: v.nativeTokenID, Amount: big.NewInt(1)},
}), v.user)
require.NoError(t, err)

// make the chain produce 2 blocks (prune the previous block with the initial deposit info)
for i := 0; i < 2; i++ {
_, err = ch2.PostRequestSync(solo.NewCallParams("contract", "func"), nil)
require.Error(t, err) // dummy request, so an error is expected
require.NotNil(t, ch2.LastReceipt().Error) // but it produced a receipt, thus make the state progress
}

// deposit 1 more after the initial deposit block has been prunned
err = ch2.DepositAssetsToL2(isc.NewAssets(1*isc.Million, iotago.NativeTokens{
{ID: v.nativeTokenID, Amount: big.NewInt(1)},
}), v.user)
require.Error(t, err) // TODO this check needs to be changed after the "accounting UTXO pruning fix"
testmisc.RequireErrorToBe(t, err, "request has been skipped")
})
}

func TestTransferAndCheckBaseTokens(t *testing.T) {
Expand Down
10 changes: 7 additions & 3 deletions packages/vm/vmexceptions/exceptions.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package vmexceptions

import (
"errors"
"fmt"

iotago "github.com/iotaledger/iota.go/v3"
Expand Down Expand Up @@ -41,7 +42,10 @@ func (m *skipRequestException) Error() string {
return m.msg
}

func IsSkipRequestException(e interface{}) bool {
_, ok := e.(*skipRequestException)
return ok
func IsSkipRequestException(e interface{}) error {
s, ok := e.(*skipRequestException)
if !ok {
return nil
}
return errors.New(s.msg)
}
140 changes: 61 additions & 79 deletions packages/vm/vmimpl/runreq.go
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
package vmimpl

import (
"fmt"
"math"
"runtime/debug"
"time"
Expand All @@ -21,7 +20,6 @@ import (
"github.com/iotaledger/wasp/packages/util/panicutil"
"github.com/iotaledger/wasp/packages/vm"
"github.com/iotaledger/wasp/packages/vm/core/accounts"
"github.com/iotaledger/wasp/packages/vm/core/blocklog"
"github.com/iotaledger/wasp/packages/vm/core/errors/coreerrors"
"github.com/iotaledger/wasp/packages/vm/core/governance"
"github.com/iotaledger/wasp/packages/vm/core/root"
Expand Down Expand Up @@ -65,29 +63,15 @@ func (vmctx *vmContext) runRequest(req isc.Request, requestIndex uint16, mainten
// so far there were no panics except optimistic reader
txsnapshot := vmctx.createTxBuilderSnapshot()

var result *vm.RequestResult
err = reqctx.catchRequestPanic(
func() {
// transfer all attached assets to the sender's account
reqctx.creditAssetsToChain()
// load gas and fee policy, calculate and set gas budget
reqctx.prepareGasBudget()
// run the contract program
receipt, callRet := reqctx.callTheContract()
vmctx.mustCheckTransactionSize()
result = &vm.RequestResult{
Request: req,
Receipt: receipt,
Return: callRet,
}
},
)
result, err := reqctx.callTheContract()
if err == nil {
err = vmctx.checkTransactionSize()
}
if err != nil {
// protocol exception triggered. Skipping the request. Rollback
// skip the request / rollback tx builder (no need to rollback the state, because the mutations will never be applied)
vmctx.restoreTxBuilderSnapshot(txsnapshot)
vmctx.blockGas.burned = initialGasBurnedTotal
vmctx.blockGas.feeCharged = initialGasFeeChargedTotal

return nil, nil, err
}

Expand Down Expand Up @@ -147,26 +131,6 @@ func (reqctx *requestContext) creditAssetsToChain() {
}
}

func (reqctx *requestContext) catchRequestPanic(f func()) error {
err := panicutil.CatchPanic(f)
if err == nil {
return nil
}
// catches protocol exception error which is not the request or contract fault
// If it occurs, the request is just skipped
if vmexceptions.IsSkipRequestException(err) {
return err
}
// panic again with more information about the error
panic(fmt.Errorf(
"panic when running request #%d ID:%s, requestbytes:%s err:%w",
reqctx.requestIndex,
reqctx.req.ID(),
iotago.EncodeHex(reqctx.req.Bytes()),
err,
))
}

// checkAllowance ensure there are enough funds to cover the specified allowance
// panics if not enough funds
func (reqctx *requestContext) checkAllowance() {
Expand All @@ -192,26 +156,47 @@ func (reqctx *requestContext) prepareGasBudget() {
reqctx.gasSetBudget(reqctx.calculateAffordableGasBudget())
}

// callTheContract runs the contract. It catches and processes all panics except the one which cancel the whole block
func (reqctx *requestContext) callTheContract() (receipt *blocklog.RequestReceipt, callRet dict.Dict) {
// callTheContract runs the contract. if an error is returned, the request will be skipped
func (reqctx *requestContext) callTheContract() (*vm.RequestResult, error) {
// TODO: do not mutate vmContext's txbuilder
txSnapshot := reqctx.vm.createTxBuilderSnapshot()

// pre execution ---------------------------------------------------------------
err := panicutil.CatchPanic(func() {
// transfer all attached assets to the sender's account
reqctx.creditAssetsToChain()
// load gas and fee policy, calculate and set gas budget
reqctx.prepareGasBudget()
// run the contract program
})
if err != nil {
// this should never happen. something is wrong here, SKIP the request
reqctx.vm.task.Log.Errorf("panic before request execution (reqid: %s): %v", reqctx.req.ID(), err)
return nil, err
}

// execution ---------------------------------------------------------------

result := &vm.RequestResult{Request: reqctx.req}

txSnapshot := reqctx.vm.createTxBuilderSnapshot() // take the txbuilder snapshot **after** the request has been consumed (in `creditAssetsToChain`)
stateSnapshot := reqctx.uncommittedState.Clone()

rollback := func() {
reqctx.vm.restoreTxBuilderSnapshot(txSnapshot)
reqctx.uncommittedState = stateSnapshot
}

var callErr *isc.VMError
var executionErr *isc.VMError
var skipRequestErr error
func() {
defer func() {
panicErr := checkVMPluginPanic(recover())
if panicErr == nil {
r := recover()
if r == nil {
return
}
callErr = panicErr
reqctx.Debugf("recovered panic from contract call: %v", panicErr)
skipRequestErr = vmexceptions.IsSkipRequestException(r)
executionErr = recoverFromExecutionError(r)
reqctx.Debugf("recovered panic from contract call: %v", executionErr)
if reqctx.vm.task.WillProduceBlock() {
reqctx.Debugf(string(debug.Stack()))
}
Expand All @@ -220,54 +205,50 @@ func (reqctx *requestContext) callTheContract() (receipt *blocklog.RequestReceip
reqctx.checkAllowance()

reqctx.GasBurnEnable(true)
callRet = reqctx.callFromRequest()
result.Return = reqctx.callFromRequest()
// ensure at least the minimum amount of gas is charged
reqctx.GasBurn(gas.BurnCodeMinimumGasPerRequest1P, reqctx.GasBurned())
}()
reqctx.GasBurnEnable(false)
if skipRequestErr != nil {
return nil, skipRequestErr
}

// post execution ---------------------------------------------------------------

// execution over, save receipt, update nonces, etc
// if anything goes wrong here, state must be rolled back and the request must be skipped
func() {
defer func() {
if r := recover(); r != nil {
rollback()
callErrStr := ""
if callErr != nil {
callErrStr = callErr.Error()
}
reqctx.vm.task.Log.Errorf("panic after request execution (reqid: %s, executionErr: %s): %v", reqctx.req.ID(), callErrStr, r)
reqctx.vm.task.Log.Debug(string(debug.Stack()))
panic(vmexceptions.ErrPostExecutionPanic)
}
}()
if callErr != nil {
err = panicutil.CatchPanic(func() {
if executionErr != nil {
// panic happened during VM plugin call. Restore the state
rollback()
}
// charge gas fee no matter what
reqctx.chargeGasFee()

// write receipt no matter what
receipt = reqctx.writeReceiptToBlockLog(callErr)
result.Receipt = reqctx.writeReceiptToBlockLog(executionErr)

if reqctx.req.IsOffLedger() {
reqctx.updateOffLedgerRequestNonce()
}
}()
})

return receipt, callRet
if err != nil {
rollback()
callErrStr := ""
if executionErr != nil {
callErrStr = executionErr.Error()
}
reqctx.vm.task.Log.Errorf("panic after request execution (reqid: %s, executionErr: %s): %v", reqctx.req.ID(), callErrStr, err)
reqctx.vm.task.Log.Debug(string(debug.Stack()))
return nil, err
}

return result, nil
}

func checkVMPluginPanic(r interface{}) *isc.VMError {
if r == nil {
return nil
}
// re-panic-ing if error it not user nor VM plugin fault.
if vmexceptions.IsSkipRequestException(r) {
panic(r)
}
// Otherwise, the panic is wrapped into the returned error, including gas-related panic
func recoverFromExecutionError(r interface{}) *isc.VMError {
switch err := r.(type) {
case *isc.VMError:
return r.(*isc.VMError)
Expand Down Expand Up @@ -452,11 +433,12 @@ func (vmctx *vmContext) loadChainConfig() {
vmctx.chainInfo = governance.NewStateAccess(vmctx.stateDraft).ChainInfo(vmctx.ChainID())
}

// mustCheckTransactionSize panics with ErrMaxTransactionSizeExceeded if the estimated transaction size exceeds the limit
func (vmctx *vmContext) mustCheckTransactionSize() {
// checkTransactionSize panics with ErrMaxTransactionSizeExceeded if the estimated transaction size exceeds the limit
func (vmctx *vmContext) checkTransactionSize() error {
essence, _ := vmctx.BuildTransactionEssence(state.L1CommitmentNil, false)
tx := transaction.MakeAnchorTransaction(essence, &iotago.Ed25519Signature{})
if tx.Size() > parameters.L1().MaxPayloadSize {
panic(vmexceptions.ErrMaxTransactionSizeExceeded)
return vmexceptions.ErrMaxTransactionSizeExceeded
}
return nil
}
2 changes: 1 addition & 1 deletion tools/cluster/tests/pruning_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func TestPruning(t *testing.T) {
},
})

// set blockKeepAmount to 10 as well
// set blockKeepAmount (active state pruning) to 10 as well
chain, err := clu.DeployChainWithDKG(clu.Config.AllNodes(), clu.Config.AllNodes(), 4, int32(blockKeepAmount))
require.NoError(t, err)
env := newChainEnv(t, clu, chain)
Expand Down

0 comments on commit 7ab50c4

Please sign in to comment.