Commit 70bd6234 authored by Adrian Sutton's avatar Adrian Sutton Committed by GitHub

op-challenger: Fix prestate checks (#9478)

* op-challenger: Fix prestate validation

Validators were not added to players created for active games, only inactive
Both the VM and output root prestates were validated against the challenger's output root prestate

* op-challenger: Add option to ignore prestate validation failures

* op-e2e: Allow invalid prestates for devnet

Also update the op-proposer options to account for a recent rename.

* op-e2e: Revert setting cannon prestate in devnetL1.json - its actually the alphabet prestate.  Deploy script dynamically loads the cannon prestate.

* op-challenger: Fix block number included in error message.

* op-challenger: Tweak error message

* op-challenger: Only ignore prestate mismatches, not RPC failures.
parent e7ff5eb6
......@@ -443,6 +443,28 @@ func TestCannonL2Genesis(t *testing.T) {
})
}
func TestUnsafeAllowInvalidPrestate(t *testing.T) {
t.Run("DefaultsToFalse", func(t *testing.T) {
cfg := configForArgs(t, addRequiredArgsExcept(config.TraceTypeAlphabet, "--unsafe-allow-invalid-prestate"))
require.False(t, cfg.AllowInvalidPrestate)
})
t.Run("EnabledWithNoValue", func(t *testing.T) {
cfg := configForArgs(t, addRequiredArgs(config.TraceTypeCannon, "--unsafe-allow-invalid-prestate"))
require.True(t, cfg.AllowInvalidPrestate)
})
t.Run("EnabledWithTrue", func(t *testing.T) {
cfg := configForArgs(t, addRequiredArgs(config.TraceTypeCannon, "--unsafe-allow-invalid-prestate=true"))
require.True(t, cfg.AllowInvalidPrestate)
})
t.Run("DisabledWithFalse", func(t *testing.T) {
cfg := configForArgs(t, addRequiredArgs(config.TraceTypeCannon, "--unsafe-allow-invalid-prestate=false"))
require.False(t, cfg.AllowInvalidPrestate)
})
}
func verifyArgsInvalid(t *testing.T, messageContains string, cliArgs []string) {
_, _, err := dryRunWithArgs(cliArgs)
require.ErrorContains(t, err, messageContains)
......
......@@ -108,6 +108,7 @@ type Config struct {
Datadir string // Data Directory
MaxConcurrency uint // Maximum number of threads to use when progressing games
PollInterval time.Duration // Polling interval for latest-block subscription when using an HTTP RPC provider
AllowInvalidPrestate bool // Whether to allow responding to games where the prestate does not match
TraceTypes []TraceType // Type of traces supported
......
......@@ -140,6 +140,12 @@ var (
EnvVars: prefixEnvVars("GAME_WINDOW"),
Value: config.DefaultGameWindow,
}
UnsafeAllowInvalidPrestate = &cli.BoolFlag{
Name: "unsafe-allow-invalid-prestate",
Usage: "Allow responding to games where the absolute prestate is configured incorrectly. THIS IS UNSAFE!",
EnvVars: prefixEnvVars("UNSAFE_ALLOW_INVALID_PRESTATE"),
Hidden: true, // Hidden as this is an unsafe flag added only for testing purposes
}
)
// requiredFlags are checked by [CheckRequired]
......@@ -168,6 +174,7 @@ var optionalFlags = []cli.Flag{
CannonSnapshotFreqFlag,
CannonInfoFreqFlag,
GameWindowFlag,
UnsafeAllowInvalidPrestate,
}
func init() {
......@@ -299,5 +306,6 @@ func NewConfigFromCLI(ctx *cli.Context) (*config.Config, error) {
TxMgrConfig: txMgrConfig,
MetricsConfig: metricsConfig,
PprofConfig: pprofConfig,
AllowInvalidPrestate: ctx.Bool(UnsafeAllowInvalidPrestate.Name),
}, nil
}
......@@ -61,18 +61,18 @@ func (a *Agent) Act(ctx context.Context) error {
// Calculate the actions to take
actions, err := a.solver.CalculateNextActions(ctx, game)
if err != nil {
log.Error("Failed to calculate all required moves", "err", err)
a.log.Error("Failed to calculate all required moves", "err", err)
}
// Perform the actions
for _, action := range actions {
log := a.log.New("action", action.Type, "is_attack", action.IsAttack, "parent", action.ParentIdx)
actionLog := a.log.New("action", action.Type, "is_attack", action.IsAttack, "parent", action.ParentIdx)
if action.Type == types.ActionTypeStep {
containsOracleData := action.OracleData != nil
isLocal := containsOracleData && action.OracleData.IsLocal
log = log.New("prestate", common.Bytes2Hex(action.PreState), "proof", common.Bytes2Hex(action.ProofData), "containsOracleData", containsOracleData, "isLocalPreimage", isLocal)
actionLog = actionLog.New("prestate", common.Bytes2Hex(action.PreState), "proof", common.Bytes2Hex(action.ProofData), "containsOracleData", containsOracleData, "isLocalPreimage", isLocal)
} else {
log = log.New("value", action.Value)
actionLog = actionLog.New("value", action.Value)
}
switch action.Type {
......@@ -81,10 +81,10 @@ func (a *Agent) Act(ctx context.Context) error {
case types.ActionTypeStep:
a.metrics.RecordGameStep()
}
log.Info("Performing action")
actionLog.Info("Performing action")
err := a.responder.PerformAction(ctx, action)
if err != nil {
log.Error("Action failed", "err", err)
actionLog.Error("Action failed", "err", err)
}
}
return nil
......
......@@ -107,6 +107,7 @@ func NewGamePlayer(
loader: loader,
logger: logger,
status: status,
prestateValidators: validators,
}, nil
}
......
......@@ -7,6 +7,7 @@ import (
"github.com/ethereum-optimism/optimism/op-challenger/config"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/claims"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/contracts"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/trace/alphabet"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/trace/cannon"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/trace/outputs"
faultTypes "github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
......@@ -81,7 +82,7 @@ func registerAlphabet(
if err != nil {
return nil, err
}
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock)
splitDepth, err := contract.GetSplitDepth(ctx)
if err != nil {
return nil, err
......@@ -93,8 +94,8 @@ func registerAlphabet(
}
return accessor, nil
}
prestateValidator := NewPrestateValidator(contract.GetAbsolutePrestateHash, prestateProvider)
genesisValidator := NewPrestateValidator(contract.GetGenesisOutputRoot, prestateProvider)
prestateValidator := NewPrestateValidator("alphabet", contract.GetAbsolutePrestateHash, alphabet.PrestateProvider)
genesisValidator := NewPrestateValidator("output root", contract.GetGenesisOutputRoot, prestateProvider)
return NewGamePlayer(ctx, cl, logger, m, dir, game.Proxy, txSender, contract, []Validator{prestateValidator, genesisValidator}, creator)
}
oracle, err := createOracle(ctx, gameFactory, caller, faultTypes.AlphabetGameType)
......@@ -148,7 +149,7 @@ func registerCannon(
if err != nil {
return nil, err
}
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock)
creator := func(ctx context.Context, logger log.Logger, gameDepth faultTypes.Depth, dir string) (faultTypes.TraceAccessor, error) {
splitDepth, err := contract.GetSplitDepth(ctx)
if err != nil {
......@@ -160,8 +161,8 @@ func registerCannon(
}
return accessor, nil
}
prestateValidator := NewPrestateValidator(contract.GetAbsolutePrestateHash, prestateProvider)
genesisValidator := NewPrestateValidator(contract.GetGenesisOutputRoot, prestateProvider)
prestateValidator := NewPrestateValidator("cannon", contract.GetAbsolutePrestateHash, cannon.NewPrestateProvider(cfg.CannonAbsolutePreState))
genesisValidator := NewPrestateValidator("output root", contract.GetGenesisOutputRoot, prestateProvider)
return NewGamePlayer(ctx, cl, logger, m, dir, game.Proxy, txSender, contract, []Validator{prestateValidator, genesisValidator}, creator)
}
oracle, err := createOracle(ctx, gameFactory, caller, faultTypes.CannonGameType)
......
......@@ -13,13 +13,16 @@ import (
var absolutePrestate = common.FromHex("0000000000000000000000000000000000000000000000000000000000000060")
var absolutePrestateInt = new(big.Int).SetBytes(absolutePrestate)
var _ types.PrestateProvider = (*AlphabetPrestateProvider)(nil)
var _ types.PrestateProvider = (*alphabetPrestateProvider)(nil)
// AlphabetPrestateProvider is a stateless [PrestateProvider] that
// PrestateProvider provides the alphabet VM prestate
var PrestateProvider = &alphabetPrestateProvider{}
// alphabetPrestateProvider is a stateless [PrestateProvider] that
// uses a pre-determined, fixed pre-state hash.
type AlphabetPrestateProvider struct{}
type alphabetPrestateProvider struct{}
func (ap *AlphabetPrestateProvider) AbsolutePreStateCommitment(_ context.Context) (common.Hash, error) {
func (ap *alphabetPrestateProvider) AbsolutePreStateCommitment(_ context.Context) (common.Hash, error) {
hash := common.BytesToHash(crypto.Keccak256(absolutePrestate))
hash[0] = mipsevm.VMStatusUnfinished
return hash, nil
......
......@@ -9,7 +9,7 @@ import (
)
func TestAlphabetPrestateProvider_AbsolutePreStateCommitment_Succeeds(t *testing.T) {
provider := AlphabetPrestateProvider{}
provider := alphabetPrestateProvider{}
hash, err := provider.AbsolutePreStateCommitment(context.Background())
require.NoError(t, err)
expected := common.HexToHash("0x03c7ae758795765c6664a5d39bf63841c71ff191e9189522bad8ebff5d4eca98")
......
......@@ -28,7 +28,7 @@ var _ types.TraceProvider = (*AlphabetTraceProvider)(nil)
// AlphabetTraceProvider is a [TraceProvider] that monotonically increments
// the starting l2 block number as the claim value.
type AlphabetTraceProvider struct {
AlphabetPrestateProvider
alphabetPrestateProvider
startingBlockNumber *big.Int
depth types.Depth
maxLen uint64
......
......@@ -7,7 +7,6 @@ import (
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
)
var _ types.PrestateProvider = (*OutputPrestateProvider)(nil)
......@@ -17,7 +16,7 @@ type OutputPrestateProvider struct {
rollupClient OutputRollupClient
}
func NewPrestateProvider(ctx context.Context, logger log.Logger, rollupClient OutputRollupClient, prestateBlock uint64) *OutputPrestateProvider {
func NewPrestateProvider(rollupClient OutputRollupClient, prestateBlock uint64) *OutputPrestateProvider {
return &OutputPrestateProvider{
prestateBlock: prestateBlock,
rollupClient: rollupClient,
......
......@@ -6,7 +6,6 @@ import (
"fmt"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
"github.com/ethereum-optimism/optimism/op-service/dial"
"github.com/ethereum-optimism/optimism/op-service/eth"
"github.com/ethereum/go-ethereum/common"
......@@ -35,15 +34,6 @@ type OutputTraceProvider struct {
gameDepth types.Depth
}
func NewTraceProvider(ctx context.Context, logger log.Logger, rollupRpc string, gameDepth types.Depth, prestateBlock, poststateBlock uint64) (*OutputTraceProvider, error) {
rollupClient, err := dial.DialRollupClientWithTimeout(ctx, dial.DefaultDialTimeout, logger, rollupRpc)
if err != nil {
return nil, err
}
prestateProvider := NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
return NewTraceProviderFromInputs(logger, prestateProvider, rollupClient, gameDepth, prestateBlock, poststateBlock), nil
}
func NewTraceProviderFromInputs(logger log.Logger, prestateProvider types.PrestateProvider, rollupClient OutputRollupClient, gameDepth types.Depth, prestateBlock, poststateBlock uint64) *OutputTraceProvider {
return &OutputTraceProvider{
PrestateProvider: prestateProvider,
......@@ -83,7 +73,7 @@ func (o *OutputTraceProvider) GetStepData(_ context.Context, _ types.Position) (
func (o *OutputTraceProvider) outputAtBlock(ctx context.Context, block uint64) (common.Hash, error) {
output, err := o.rollupClient.OutputAtBlock(ctx, block)
if err != nil {
return common.Hash{}, fmt.Errorf("failed to fetch output at block %v: %w", o.prestateBlock, err)
return common.Hash{}, fmt.Errorf("failed to fetch output at block %v: %w", block, err)
}
return common.Hash(output.OutputRoot), nil
}
......@@ -5,6 +5,7 @@ import (
"context"
"fmt"
gameTypes "github.com/ethereum-optimism/optimism/op-challenger/game/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
......@@ -19,14 +20,16 @@ type Validator interface {
var _ Validator = (*PrestateValidator)(nil)
type PrestateValidator struct {
valueName string
load PrestateLoader
provider types.PrestateProvider
}
func NewPrestateValidator(loader PrestateLoader, provider types.PrestateProvider) *PrestateValidator {
func NewPrestateValidator(valueName string, contractProvider PrestateLoader, localProvider types.PrestateProvider) *PrestateValidator {
return &PrestateValidator{
load: loader,
provider: provider,
valueName: valueName,
load: contractProvider,
provider: localProvider,
}
}
......@@ -40,7 +43,8 @@ func (v *PrestateValidator) Validate(ctx context.Context) error {
return fmt.Errorf("failed to fetch provider's prestate hash: %w", err)
}
if !bytes.Equal(prestateCommitment[:], prestateHash[:]) {
return fmt.Errorf("provider's absolute prestate does not match contract's absolute prestate: Provider: %s | Contract: %s", prestateCommitment.Hex(), prestateHash.Hex())
return fmt.Errorf("%v %w: Provider: %s | Contract: %s",
v.valueName, gameTypes.ErrInvalidPrestate, prestateCommitment.Hex(), prestateHash.Hex())
}
return nil
}
......@@ -7,6 +7,7 @@ import (
"github.com/ethereum-optimism/optimism/cannon/mipsevm"
"github.com/ethereum-optimism/optimism/op-challenger/game/fault/types"
gameTypes "github.com/ethereum-optimism/optimism/op-challenger/game/types"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto"
"github.com/stretchr/testify/require"
......@@ -54,7 +55,7 @@ func TestValidate(t *testing.T) {
provider: newMockPrestateProvider(false, prestate),
}
err := player.Validate(context.Background())
require.Error(t, err)
require.ErrorIs(t, err, gameTypes.ErrInvalidPrestate)
})
}
......@@ -72,7 +73,7 @@ func newMockPrestateProvider(prestateErrors bool, prestate []byte) *mockPrestate
}
}
func (m *mockPrestateProvider) AbsolutePreStateCommitment(ctx context.Context) (common.Hash, error) {
func (m *mockPrestateProvider) AbsolutePreStateCommitment(_ context.Context) (common.Hash, error) {
if m.prestateErrors {
return common.Hash{}, mockProviderError
}
......
......@@ -46,6 +46,8 @@ type coordinator struct {
states map[common.Address]*gameState
disk DiskManager
allowInvalidPrestate bool
// lastScheduledBlockNum is the highest block number that the coordinator has seen and scheduled jobs.
lastScheduledBlockNum uint64
}
......@@ -133,8 +135,11 @@ func (c *coordinator) createJob(ctx context.Context, game types.GameMetadata, bl
return nil, fmt.Errorf("failed to create game player: %w", err)
}
if err := player.ValidatePrestate(ctx); err != nil {
if !c.allowInvalidPrestate || !errors.Is(err, types.ErrInvalidPrestate) {
return nil, fmt.Errorf("failed to validate prestate: %w", err)
}
c.logger.Error("Invalid prestate", "game", game.Proxy, "err", err)
}
state.player = player
state.status = player.Status()
}
......@@ -186,7 +191,7 @@ func (c *coordinator) deleteResolvedGameFiles() {
}
}
func newCoordinator(logger log.Logger, m CoordinatorMetricer, jobQueue chan<- job, resultQueue <-chan job, createPlayer PlayerCreator, disk DiskManager) *coordinator {
func newCoordinator(logger log.Logger, m CoordinatorMetricer, jobQueue chan<- job, resultQueue <-chan job, createPlayer PlayerCreator, disk DiskManager, allowInvalidPrestate bool) *coordinator {
return &coordinator{
logger: logger,
m: m,
......@@ -195,5 +200,6 @@ func newCoordinator(logger log.Logger, m CoordinatorMetricer, jobQueue chan<- jo
createPlayer: createPlayer,
disk: disk,
states: make(map[common.Address]*gameState),
allowInvalidPrestate: allowInvalidPrestate,
}
}
......@@ -15,7 +15,7 @@ import (
)
func TestScheduleNewGames(t *testing.T) {
c, workQueue, _, games, disk := setupCoordinatorTest(t, 10)
c, workQueue, _, games, disk, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -36,7 +36,7 @@ func TestScheduleNewGames(t *testing.T) {
}
func TestSkipSchedulingInflightGames(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
ctx := context.Background()
......@@ -51,7 +51,7 @@ func TestSkipSchedulingInflightGames(t *testing.T) {
func TestExitWhenContextDoneWhileSchedulingJob(t *testing.T) {
// No space in buffer to schedule a job
c, workQueue, _, _, _ := setupCoordinatorTest(t, 0)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 0)
gameAddr1 := common.Address{0xaa}
ctx, cancel := context.WithCancel(context.Background())
cancel() // Context is cancelled
......@@ -63,8 +63,8 @@ func TestExitWhenContextDoneWhileSchedulingJob(t *testing.T) {
}
func TestSchedule_PrestateValidationErrors(t *testing.T) {
c, _, _, games, _ := setupCoordinatorTest(t, 10)
games.PrestateErr = fmt.Errorf("prestate error")
c, _, _, games, _, _ := setupCoordinatorTest(t, 10)
games.PrestateErr = types.ErrInvalidPrestate
gameAddr1 := common.Address{0xaa}
ctx := context.Background()
......@@ -72,8 +72,34 @@ func TestSchedule_PrestateValidationErrors(t *testing.T) {
require.Error(t, err)
}
func TestSchedule_SkipPrestateValidationErrors(t *testing.T) {
c, _, _, games, _, logs := setupCoordinatorTest(t, 10)
c.allowInvalidPrestate = true
games.PrestateErr = types.ErrInvalidPrestate
gameAddr1 := common.Address{0xaa}
ctx := context.Background()
err := c.schedule(ctx, asGames(gameAddr1), 0)
require.NoError(t, err)
errLog := logs.FindLog(testlog.NewLevelFilter(log.LevelError), testlog.NewMessageFilter("Invalid prestate"))
require.NotNil(t, errLog)
require.Equal(t, errLog.AttrValue("game"), gameAddr1)
require.Equal(t, errLog.AttrValue("err"), games.PrestateErr)
}
func TestSchedule_PrestateValidationFailure(t *testing.T) {
c, _, _, games, _, _ := setupCoordinatorTest(t, 10)
c.allowInvalidPrestate = true
games.PrestateErr = fmt.Errorf("failed to fetch prestate")
gameAddr1 := common.Address{0xaa}
ctx := context.Background()
err := c.schedule(ctx, asGames(gameAddr1), 0)
require.ErrorIs(t, err, games.PrestateErr)
}
func TestScheduleGameAgainAfterCompletion(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
ctx := context.Background()
......@@ -94,13 +120,13 @@ func TestScheduleGameAgainAfterCompletion(t *testing.T) {
}
func TestResultForUnknownGame(t *testing.T) {
c, _, _, _, _ := setupCoordinatorTest(t, 10)
c, _, _, _, _, _ := setupCoordinatorTest(t, 10)
err := c.processResult(job{addr: common.Address{0xaa}})
require.ErrorIs(t, err, errUnknownGame)
}
func TestProcessResultsWhileJobQueueFull(t *testing.T) {
c, workQueue, resultQueue, games, disk := setupCoordinatorTest(t, 0)
c, workQueue, resultQueue, games, disk, _ := setupCoordinatorTest(t, 0)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -142,7 +168,7 @@ loop:
}
func TestDeleteDataForResolvedGames(t *testing.T) {
c, workQueue, _, _, disk := setupCoordinatorTest(t, 10)
c, workQueue, _, _, disk, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -182,7 +208,7 @@ func TestDeleteDataForResolvedGames(t *testing.T) {
}
func TestSchedule_RecordActedL1Block(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr3 := common.Address{0xcc}
ctx := context.Background()
......@@ -203,7 +229,7 @@ func TestSchedule_RecordActedL1Block(t *testing.T) {
}
func TestSchedule_RecordActedL1BlockMultipleGames(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -247,7 +273,7 @@ func TestSchedule_RecordActedL1BlockMultipleGames(t *testing.T) {
}
func TestSchedule_RecordActedL1BlockNewGame(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -276,7 +302,7 @@ func TestSchedule_RecordActedL1BlockNewGame(t *testing.T) {
}
func TestDoNotDeleteDataForGameThatFailedToCreatePlayer(t *testing.T) {
c, workQueue, _, games, disk := setupCoordinatorTest(t, 10)
c, workQueue, _, games, disk, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
ctx := context.Background()
......@@ -307,7 +333,7 @@ func TestDoNotDeleteDataForGameThatFailedToCreatePlayer(t *testing.T) {
}
func TestDropOldGameStates(t *testing.T) {
c, workQueue, _, _, _ := setupCoordinatorTest(t, 10)
c, workQueue, _, _, _, _ := setupCoordinatorTest(t, 10)
gameAddr1 := common.Address{0xaa}
gameAddr2 := common.Address{0xbb}
gameAddr3 := common.Address{0xcc}
......@@ -331,8 +357,8 @@ func TestDropOldGameStates(t *testing.T) {
require.Contains(t, c.states, gameAddr4, "should create state for game 4")
}
func setupCoordinatorTest(t *testing.T, bufferSize int) (*coordinator, <-chan job, chan job, *createdGames, *stubDiskManager) {
logger := testlog.Logger(t, log.LevelInfo)
func setupCoordinatorTest(t *testing.T, bufferSize int) (*coordinator, <-chan job, chan job, *createdGames, *stubDiskManager, *testlog.CapturingHandler) {
logger, logs := testlog.CaptureLogger(t, log.LevelInfo)
workQueue := make(chan job, bufferSize)
resultQueue := make(chan job, bufferSize)
games := &createdGames{
......@@ -340,8 +366,8 @@ func setupCoordinatorTest(t *testing.T, bufferSize int) (*coordinator, <-chan jo
created: make(map[common.Address]*test.StubGamePlayer),
}
disk := &stubDiskManager{gameDirExists: make(map[common.Address]bool)}
c := newCoordinator(logger, &stubSchedulerMetrics{}, workQueue, resultQueue, games.CreateGame, disk)
return c, workQueue, resultQueue, games, disk
c := newCoordinator(logger, &stubSchedulerMetrics{}, workQueue, resultQueue, games.CreateGame, disk, false)
return c, workQueue, resultQueue, games, disk, logs
}
type createdGames struct {
......
......@@ -39,7 +39,7 @@ type Scheduler struct {
cancel func()
}
func NewScheduler(logger log.Logger, m SchedulerMetricer, disk DiskManager, maxConcurrency uint, createPlayer PlayerCreator) *Scheduler {
func NewScheduler(logger log.Logger, m SchedulerMetricer, disk DiskManager, maxConcurrency uint, createPlayer PlayerCreator, allowInvalidPrestate bool) *Scheduler {
// Size job and results queues to be fairly small so backpressure is applied early
// but with enough capacity to keep the workers busy
jobQueue := make(chan job, maxConcurrency*2)
......@@ -52,7 +52,7 @@ func NewScheduler(logger log.Logger, m SchedulerMetricer, disk DiskManager, maxC
return &Scheduler{
logger: logger,
m: m,
coordinator: newCoordinator(logger, m, jobQueue, resultQueue, createPlayer, disk),
coordinator: newCoordinator(logger, m, jobQueue, resultQueue, createPlayer, disk, allowInvalidPrestate),
maxConcurrency: maxConcurrency,
scheduleQueue: scheduleQueue,
jobQueue: jobQueue,
......
......@@ -21,7 +21,7 @@ func TestSchedulerProcessesGames(t *testing.T) {
}
removeExceptCalls := make(chan []common.Address)
disk := &trackingDiskManager{removeExceptCalls: removeExceptCalls}
s := NewScheduler(logger, metrics.NoopMetrics, disk, 2, createPlayer)
s := NewScheduler(logger, metrics.NoopMetrics, disk, 2, createPlayer, false)
s.Start(ctx)
gameAddr1 := common.Address{0xaa}
......@@ -49,7 +49,7 @@ func TestReturnBusyWhenScheduleQueueFull(t *testing.T) {
}
removeExceptCalls := make(chan []common.Address)
disk := &trackingDiskManager{removeExceptCalls: removeExceptCalls}
s := NewScheduler(logger, metrics.NoopMetrics, disk, 2, createPlayer)
s := NewScheduler(logger, metrics.NoopMetrics, disk, 2, createPlayer, false)
// Scheduler not started - first call fills the queue
require.NoError(t, s.Schedule(asGames(common.Address{0xaa}), 0))
......
......@@ -228,7 +228,7 @@ func (s *Service) registerGameTypes(ctx context.Context, cfg *config.Config) err
func (s *Service) initScheduler(cfg *config.Config) error {
disk := newDiskManager(cfg.Datadir)
s.sched = scheduler.NewScheduler(s.logger, s.metrics, disk, cfg.MaxConcurrency, s.registry.CreatePlayer)
s.sched = scheduler.NewScheduler(s.logger, s.metrics, disk, cfg.MaxConcurrency, s.registry.CreatePlayer, cfg.AllowInvalidPrestate)
return nil
}
......
package types
import (
"errors"
"fmt"
"github.com/ethereum-optimism/optimism/op-service/txmgr"
......@@ -8,6 +9,8 @@ import (
ethtypes "github.com/ethereum/go-ethereum/core/types"
)
var ErrInvalidPrestate = errors.New("absolute prestate does not match")
type GameStatus uint8
const (
......
......@@ -136,6 +136,9 @@ func NewChallengerConfig(t *testing.T, sys EndpointProvider, options ...Option)
l1Endpoint := sys.NodeEndpoint("l1")
l1Beacon := sys.L1BeaconEndpoint()
cfg := config.NewConfig(common.Address{}, l1Endpoint, l1Beacon, t.TempDir())
// The devnet can't set the absolute prestate output root because the contracts are deployed in L1 genesis
// before the L2 genesis is known.
cfg.AllowInvalidPrestate = true
cfg.TxMgrConfig.NumConfirmations = 1
cfg.TxMgrConfig.ReceiptQueryInterval = 1 * time.Second
if cfg.MaxConcurrency > 4 {
......
......@@ -154,7 +154,7 @@ func (h *FactoryHelper) StartOutputCannonGame(ctx context.Context, l2Node string
h.require.NoError(err, "Failed to load l2 block number")
splitDepth, err := game.SplitDepth(&bind.CallOpts{Context: ctx})
h.require.NoError(err, "Failed to load split depth")
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock.Uint64())
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock.Uint64())
provider := outputs.NewTraceProviderFromInputs(logger, prestateProvider, rollupClient, faultTypes.Depth(splitDepth.Uint64()), prestateBlock.Uint64(), poststateBlock.Uint64())
return &OutputCannonGameHelper{
......@@ -206,7 +206,7 @@ func (h *FactoryHelper) StartOutputAlphabetGame(ctx context.Context, l2Node stri
h.require.NoError(err, "Failed to load l2 block number")
splitDepth, err := game.SplitDepth(&bind.CallOpts{Context: ctx})
h.require.NoError(err, "Failed to load split depth")
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock.Uint64())
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock.Uint64())
provider := outputs.NewTraceProviderFromInputs(logger, prestateProvider, rollupClient, faultTypes.Depth(splitDepth.Uint64()), prestateBlock.Uint64(), poststateBlock.Uint64())
return &OutputAlphabetGameHelper{
......
......@@ -44,7 +44,7 @@ func (g *OutputAlphabetGameHelper) CreateHonestActor(ctx context.Context, l2Node
g.require.NoError(err, "Get block range")
splitDepth := g.SplitDepth(ctx)
rollupClient := g.system.RollupClient(l2Node)
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock)
correctTrace, err := outputs.NewOutputAlphabetTraceAccessor(logger, metrics.NoopMetrics, prestateProvider, rollupClient, splitDepth, prestateBlock, poststateBlock)
g.require.NoError(err, "Create trace accessor")
return &OutputHonestHelper{
......
......@@ -63,7 +63,7 @@ func (g *OutputCannonGameHelper) CreateHonestActor(ctx context.Context, l2Node s
dir := filepath.Join(cfg.Datadir, "honest")
splitDepth := g.SplitDepth(ctx)
rollupClient := g.system.RollupClient(l2Node)
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock)
accessor, err := outputs.NewOutputCannonTraceAccessor(
logger, metrics.NoopMetrics, cfg, l2Client, contract, prestateProvider, rollupClient, dir, splitDepth, prestateBlock, poststateBlock)
g.require.NoError(err, "Failed to create output cannon trace accessor")
......@@ -229,7 +229,7 @@ func (g *OutputCannonGameHelper) createCannonTraceProvider(ctx context.Context,
prestateBlock, poststateBlock, err := contract.GetBlockRange(ctx)
g.require.NoError(err, "Failed to load block range")
rollupClient := g.system.RollupClient(l2Node)
prestateProvider := outputs.NewPrestateProvider(ctx, logger, rollupClient, prestateBlock)
prestateProvider := outputs.NewPrestateProvider(rollupClient, prestateBlock)
outputProvider := outputs.NewTraceProviderFromInputs(logger, prestateProvider, rollupClient, splitDepth, prestateBlock, poststateBlock)
selector := split.NewSplitProviderSelector(outputProvider, splitDepth, func(ctx context.Context, depth types.Depth, pre types.Claim, post types.Claim) (types.TraceProvider, error) {
......
......@@ -125,8 +125,8 @@ services:
OP_PROPOSER_MNEMONIC: test test test test test test test test test test test junk
OP_PROPOSER_L2_OUTPUT_HD_PATH: "m/44'/60'/0'/0/1"
OP_PROPOSER_L2OO_ADDRESS: "${L2OO_ADDRESS}"
OP_PROPOSER_DGF_ADDRESS: "${DGF_ADDRESS}"
OP_PROPOSER_DG_TYPE: "${DG_TYPE}"
OP_PROPOSER_GAME_FACTORY_ADDRESS: "${DGF_ADDRESS}"
OP_PROPOSER_GAME_TYPE: "${DG_TYPE}"
OP_PROPOSER_PROPOSAL_INTERVAL: "${PROPOSAL_INTERVAL}"
OP_PROPOSER_PPROF_ENABLED: "true"
OP_PROPOSER_METRICS_ENABLED: "true"
......@@ -183,6 +183,9 @@ services:
OP_CHALLENGER_ROLLUP_RPC: http://op-node:8545
OP_CHALLENGER_TRACE_TYPE: cannon
OP_CHALLENGER_GAME_FACTORY_ADDRESS: ${DGF_ADDRESS}
# The devnet can't set the absolute prestate output root because the contracts are deployed in L1 genesis
# before the L2 genesis is known.
OP_CHALLENGER_UNSAFE_ALLOW_INVALID_PRESTATE: true
OP_CHALLENGER_DATADIR: temp/challenger-data
OP_CHALLENGER_CANNON_ROLLUP_CONFIG: ./.devnet/rollup.json
OP_CHALLENGER_CANNON_L2_GENESIS: ./.devnet/genesis-l2.json
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment