diff --git a/cmd/cartesi-rollups-cli/root/app/add/add.go b/cmd/cartesi-rollups-cli/root/app/add/add.go index 0545c88b9..1d622e3de 100644 --- a/cmd/cartesi-rollups-cli/root/app/add/add.go +++ b/cmd/cartesi-rollups-cli/root/app/add/add.go @@ -118,7 +118,7 @@ func run(cmd *cobra.Command, args []string) { IConsensusAddress: common.HexToAddress(iConsensusAddress), } - err := cmdcommom.Database.InsertApplication(ctx, &application) + _, err := cmdcommom.Database.InsertApplication(ctx, &application) cobra.CheckErr(err) fmt.Printf("Application %v successfully added\n", application.ContractAddress) } diff --git a/internal/node/model/models.go b/internal/node/model/models.go index b0bd46c3d..aeeb53d96 100644 --- a/internal/node/model/models.go +++ b/internal/node/model/models.go @@ -69,6 +69,13 @@ type Application struct { LastProcessedBlock uint64 Status ApplicationStatus IConsensusAddress Address + // Temporary ------------------------------------ + MachineIncCycles uint64 + MachineMaxCycles uint64 + MachineAdvanceTimeout uint32 + MachineInspectTimeout uint32 + MachineMaxConcurrentInspects uint32 + // ---------------------------------------------- } type Epoch struct { diff --git a/internal/repository/base.go b/internal/repository/base.go index c544e5b55..6a005de70 100644 --- a/internal/repository/base.go +++ b/internal/repository/base.go @@ -19,7 +19,14 @@ type Database struct { db *pgxpool.Pool } -var ErrInsertRow = errors.New("unable to insert row") +var ( + ErrInsertRow = errors.New("unable to insert row") + ErrUpdateRow = errors.New("unable to update row") + ErrCopyFrom = errors.New("unable to COPY FROM") + + ErrBeginTx = errors.New("unable to begin transaction") + ErrCommitTx = errors.New("unable to commit transaction") +) func Connect( ctx context.Context, @@ -84,35 +91,57 @@ func (pg *Database) InsertNodeConfig( func (pg *Database) InsertApplication( ctx context.Context, app *Application, -) error { +) (uint64, error) { query := ` INSERT INTO application (contract_address, template_hash, + template_uri, last_processed_block, status, - iconsensus_address) + iconsensus_address, + machine_inc_cycles, + machine_max_cycles, + machine_advance_timeout, + machine_inspect_timeout, + machine_max_concurrent_inspects) VALUES (@contractAddress, @templateHash, + @templateUri, @lastProcessedBlock, @status, - @iConsensusAddress)` + @iConsensusAddress, + @machineIncCycles, + @machineMaxCycles, + @machineAdvanceTimeout, + @machineInspectTimeout, + @machineMaxConcurrentInspects) + RETURNING + id + ` args := pgx.NamedArgs{ - "contractAddress": app.ContractAddress, - "templateHash": app.TemplateHash, - "lastProcessedBlock": app.LastProcessedBlock, - "status": app.Status, - "iConsensusAddress": app.IConsensusAddress, + "contractAddress": app.ContractAddress, + "templateHash": app.TemplateHash, + "templateUri": app.TemplateUri, + "lastProcessedBlock": app.LastProcessedBlock, + "status": app.Status, + "iConsensusAddress": app.IConsensusAddress, + "machineIncCycles": app.MachineIncCycles, + "machineMaxCycles": app.MachineMaxCycles, + "machineAdvanceTimeout": app.MachineAdvanceTimeout, + "machineInspectTimeout": app.MachineInspectTimeout, + "machineMaxConcurrentInspects": app.MachineMaxConcurrentInspects, } - _, err := pg.db.Exec(ctx, query, args) + var id uint64 + err := pg.db.QueryRow(ctx, query, args).Scan(&id) if err != nil { - return fmt.Errorf("%w: %w", ErrInsertRow, err) + return 0, fmt.Errorf("%w: %w", ErrInsertRow, err) } - return nil + return id, nil } func (pg *Database) InsertEpoch( @@ -140,7 +169,8 @@ func (pg *Database) InsertEpoch( @status, @applicationAddress) RETURNING - id` + id + ` args := pgx.NamedArgs{ "index": epoch.Index, @@ -286,7 +316,9 @@ func (pg *Database) InsertSnapshot( (@inputId, @appAddress, @uri) - RETURNING id` + RETURNING + id + ` args := pgx.NamedArgs{ "inputId": snapshot.InputId, diff --git a/internal/repository/base_test.go b/internal/repository/base_test.go index 3caa6943c..36d9c32b3 100644 --- a/internal/repository/base_test.go +++ b/internal/repository/base_test.go @@ -84,10 +84,10 @@ func (s *RepositorySuite) SetupDatabase() { Status: ApplicationStatusNotRunning, } - err = s.database.InsertApplication(s.ctx, &app) + _, err = s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) - err = s.database.InsertApplication(s.ctx, &app2) + _, err = s.database.InsertApplication(s.ctx, &app2) s.Require().Nil(err) genericHash := common.HexToHash("deadbeef") @@ -256,7 +256,7 @@ func (s *RepositorySuite) TestApplicationFailsDuplicateRow() { Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, &app) + _, err := s.database.InsertApplication(s.ctx, &app) s.Require().ErrorContains(err, "duplicate key value") } diff --git a/internal/repository/machine.go b/internal/repository/machine.go new file mode 100644 index 000000000..73dda60ed --- /dev/null +++ b/internal/repository/machine.go @@ -0,0 +1,311 @@ +// (c) Cartesi and individual authors (see AUTHORS) +// SPDX-License-Identifier: Apache-2.0 (see LICENSE) + +package repository + +import ( + "context" + "errors" + "fmt" + "strings" + + . "github.com/cartesi/rollups-node/internal/node/model" + "github.com/cartesi/rollups-node/internal/nodemachine" + + "github.com/cartesi/rollups-node/internal/node/advancer/machines" + + "github.com/jackc/pgx/v5" +) + +var ErrAdvancerRepository = errors.New("advancer repository error") + +type MachineRepository struct{ *Database } + +func (repo *MachineRepository) GetMachineConfigurations( + ctx context.Context, +) ([]*machines.MachineConfig, error) { + query := ` + SELECT DISTINCT ON (address) + a.contract_address AS address, + i.index, + a.machine_inc_cycles, + a.machine_max_cycles, + a.machine_advance_timeout, + a.machine_inspect_timeout, + a.machine_max_concurrent_inspects, + COALESCE(s.uri, a.template_uri) + FROM application AS a + LEFT JOIN snapshot AS s ON (a.contract_address = s.application_address) + LEFT JOIN input AS i ON (s.input_id = i.id) + WHERE a.status = 'RUNNING' + ORDER BY address, i.index DESC + ` + rows, err := repo.db.Query(ctx, query) + if err != nil { + return nil, fmt.Errorf("%w (failed querying applications): %w", ErrAdvancerRepository, err) + } + + // TODO: missing machine config fields + res := []*machines.MachineConfig{} + var row machines.MachineConfig + + scans := []any{ + &row.AppAddress, + &row.SnapshotInputIndex, + &row.IncCycles, + &row.MaxCycles, + &row.AdvanceTimeout, + &row.InspectTimeout, + &row.MaxConcurrentInspects, + &row.SnapshotPath, + } + _, err = pgx.ForEachRow(rows, scans, func() error { + row := row + res = append(res, &row) + return nil + }) + if err != nil { + return nil, fmt.Errorf("%w (failed reading rows): %w", ErrAdvancerRepository, err) + } + + return res, nil +} + +func (repo *MachineRepository) GetProcessedInputs( + ctx context.Context, + app Address, + index uint64, +) ([]*Input, error) { + query := fmt.Sprintf(` + SELECT id, index, status, raw_data + FROM input + WHERE application_address = @applicationAddress + AND index >= @index + AND status != 'NONE' + ORDER BY index ASC + `) + args := pgx.NamedArgs{ + "applicationAddress": app, + "index": index, + } + rows, err := repo.db.Query(ctx, query, args) + if err != nil { + return nil, fmt.Errorf("%w (failed querying inputs): %w", ErrAdvancerRepository, err) + } + + res := []*Input{} + var input Input + scans := []any{&input.Id, &input.Index, &input.CompletionStatus, &input.RawData} + _, err = pgx.ForEachRow(rows, scans, func() error { + input := input + res = append(res, &input) + return nil + }) + if err != nil { + return nil, fmt.Errorf("%w (failed reading input rows): %w", ErrAdvancerRepository, err) + } + + return res, nil +} + +func (repo *MachineRepository) GetUnprocessedInputs( + ctx context.Context, + apps []Address, +) (map[Address][]*Input, error) { + result := map[Address][]*Input{} + if len(apps) == 0 { + return result, nil + } + + query := fmt.Sprintf(` + SELECT id, application_address, raw_data + FROM input + WHERE status = 'NONE' + AND application_address IN %s + ORDER BY index ASC, application_address + `, toSqlIn(apps)) // NOTE: not sanitized + rows, err := repo.db.Query(ctx, query) + if err != nil { + return nil, fmt.Errorf("%w (failed querying inputs): %w", ErrAdvancerRepository, err) + } + + var input Input + scans := []any{&input.Id, &input.AppAddress, &input.RawData} + _, err = pgx.ForEachRow(rows, scans, func() error { + input := input + if _, ok := result[input.AppAddress]; ok { //nolint:gosimple + result[input.AppAddress] = append(result[input.AppAddress], &input) + } else { + result[input.AppAddress] = []*Input{&input} + } + return nil + }) + if err != nil { + return nil, fmt.Errorf("%w (failed reading input rows): %w", ErrAdvancerRepository, err) + } + + return result, nil +} + +func (repo *MachineRepository) StoreAdvanceResult( + ctx context.Context, + input *Input, + res *nodemachine.AdvanceResult, +) error { + tx, err := repo.db.Begin(ctx) + if err != nil { + return errors.Join(ErrBeginTx, err) + } + + // Inserts the outputs. + nextOutputIndex, err := repo.getNextIndex(ctx, tx, "output", input.AppAddress) + if err != nil { + return err + } + err = repo.insert(ctx, tx, "output", res.Outputs, input.Id, nextOutputIndex) + if err != nil { + return err + } + + // Inserts the reports. + nextReportIndex, err := repo.getNextIndex(ctx, tx, "report", input.AppAddress) + if err != nil { + return err + } + err = repo.insert(ctx, tx, "report", res.Reports, input.Id, nextReportIndex) + if err != nil { + return err + } + + // Updates the input's status. + err = repo.updateInput(ctx, tx, input.Id, res.Status, res.OutputsHash, res.MachineHash) + if err != nil { + return err + } + + err = tx.Commit(ctx) + if err != nil { + return errors.Join(ErrCommitTx, err, tx.Rollback(ctx)) + } + + return nil +} + +func (repo *MachineRepository) UpdateEpochs(ctx context.Context, app Address) error { + query := ` + UPDATE epoch + SET status = 'PROCESSED_ALL_INPUTS' + WHERE id IN (( + SELECT DISTINCT epoch.id + FROM epoch INNER JOIN input ON (epoch.id = input.epoch_id) + WHERE epoch.application_address = @applicationAddress + AND epoch.status = 'CLOSED' + AND input.status != 'NONE' + ) EXCEPT ( + SELECT DISTINCT epoch.id + FROM epoch INNER JOIN input ON (epoch.id = input.epoch_id) + WHERE epoch.application_address = @applicationAddress + AND epoch.status = 'CLOSED' + AND input.status = 'NONE')) + ` + args := pgx.NamedArgs{"applicationAddress": app} + _, err := repo.db.Exec(ctx, query, args) + if err != nil { + return errors.Join(ErrUpdateRow, err) + } + return nil +} + +// ------------------------------------------------------------------------------------------------ + +func (_ *MachineRepository) getNextIndex( + ctx context.Context, + tx pgx.Tx, + tableName string, + appAddress Address, +) (uint64, error) { + var nextIndex uint64 + query := fmt.Sprintf(` + SELECT COALESCE(MAX(%s.index) + 1, 0) + FROM input INNER JOIN %s ON input.id = %s.input_id + WHERE input.status = 'ACCEPTED' + AND input.application_address = $1 + `, tableName, tableName, tableName) + err := tx.QueryRow(ctx, query, appAddress).Scan(&nextIndex) + if err != nil { + err = fmt.Errorf("failed to get the next %s index: %w", tableName, err) + return 0, errors.Join(err, tx.Rollback(ctx)) + } + return nextIndex, nil +} + +func (_ *MachineRepository) insert( + ctx context.Context, + tx pgx.Tx, + tableName string, + dataArray [][]byte, + inputId uint64, + nextIndex uint64, +) error { + lenOutputs := int64(len(dataArray)) + if lenOutputs < 1 { + return nil + } + + rows := [][]any{} + for i, data := range dataArray { + rows = append(rows, []any{inputId, nextIndex + uint64(i), data}) + } + + count, err := tx.CopyFrom( + ctx, + pgx.Identifier{tableName}, + []string{"input_id", "index", "raw_data"}, + pgx.CopyFromRows(rows), + ) + if err != nil { + return errors.Join(ErrCopyFrom, err, tx.Rollback(ctx)) + } + if lenOutputs != count { + err := fmt.Errorf("not all %ss were inserted (%d != %d)", tableName, lenOutputs, count) + return errors.Join(err, tx.Rollback(ctx)) + } + + return nil +} + +func (_ *MachineRepository) updateInput( + ctx context.Context, + tx pgx.Tx, + inputId uint64, + status InputCompletionStatus, + outputsHash Hash, + machineHash *Hash, +) error { + query := ` + UPDATE input + SET (status, outputs_hash, machine_hash) = (@status, @outputsHash, @machineHash) + WHERE id = @id + ` + args := pgx.NamedArgs{ + "status": status, + "outputsHash": outputsHash, + "machineHash": machineHash, + "id": inputId, + } + _, err := tx.Exec(ctx, query, args) + if err != nil { + return errors.Join(ErrUpdateRow, err, tx.Rollback(ctx)) + } + return nil +} + +// ------------------------------------------------------------------------------------------------ + +func toSqlIn[T fmt.Stringer](a []T) string { + s := []string{} + for _, x := range a { + s = append(s, fmt.Sprintf("'\\x%s'", x.String()[2:])) + } + return fmt.Sprintf("(%s)", strings.Join(s, ", ")) +} diff --git a/internal/repository/machine_test.go b/internal/repository/machine_test.go new file mode 100644 index 000000000..fc54d5e0e --- /dev/null +++ b/internal/repository/machine_test.go @@ -0,0 +1,315 @@ +// (c) Cartesi and individual authors (see AUTHORS) +// SPDX-License-Identifier: Apache-2.0 (see LICENSE) + +package repository + +import ( + "context" + "testing" + + . "github.com/cartesi/rollups-node/internal/node/model" + + "github.com/cartesi/rollups-node/pkg/rollupsmachine" + "github.com/ethereum/go-ethereum/common" + + "github.com/cartesi/rollups-node/test/tooling/db" + "github.com/stretchr/testify/require" +) + +func TestMachineRepository(t *testing.T) { + ctx := context.Background() + + t.Run("GetMachineConfigurations", func(t *testing.T) { + require := require.New(t) + + endpoint, err := db.Setup(ctx) + require.Nil(err) + + database, err := Connect(ctx, endpoint) + require.Nil(err) + require.NotNil(database) + + apps, _, _, _, err := populate2(database) + require.Nil(err) + repository := &MachineRepository{Database: database} + + res, err := repository.GetMachineConfigurations(ctx) + require.Nil(err) + require.Len(res, 2) + + config1, config2 := res[0], res[1] + + require.Equal(apps[1].ContractAddress, config2.AppAddress) + require.Equal(uint64(6), *config2.SnapshotInputIndex) + require.Equal("path/to/snapshot/2", config2.SnapshotPath) + + require.Equal(apps[2].ContractAddress, config1.AppAddress) + require.Nil(config1.SnapshotInputIndex) + require.Equal("path/to/template/uri/2", config1.SnapshotPath) + }) + + t.Run("GetProcessedInputs", func(t *testing.T) { + t.Skip("TODO") + }) + + t.Run("GetUnprocessedInputs", func(t *testing.T) { + t.Skip("TODO") + }) + + t.Run("StoreAdvanceResult", func(t *testing.T) { + t.Skip("TODO") + }) + + t.Run("UpdateEpochs", func(t *testing.T) { + require := require.New(t) + + endpoint, err := db.Setup(ctx) + require.Nil(err) + + database, err := Connect(ctx, endpoint) + require.Nil(err) + require.NotNil(database) + + app, _, _, err := populate1(database) + require.Nil(err) + repository := &MachineRepository{Database: database} + + err = repository.UpdateEpochs(ctx, app.ContractAddress) + require.Nil(err) + + epoch0, err := repository.GetEpoch(ctx, 0, app.ContractAddress) + require.Nil(err) + require.NotNil(epoch0) + + epoch1, err := repository.GetEpoch(ctx, 1, app.ContractAddress) + require.Nil(err) + require.NotNil(epoch1) + + epoch2, err := repository.GetEpoch(ctx, 2, app.ContractAddress) + require.Nil(err) + require.NotNil(epoch2) + + epoch3, err := repository.GetEpoch(ctx, 3, app.ContractAddress) + require.Nil(err) + require.NotNil(epoch3) + + require.Equal(EpochStatusProcessedAllInputs, epoch0.Status) + require.Equal(EpochStatusProcessedAllInputs, epoch1.Status) + require.Equal(EpochStatusClosed, epoch2.Status) + require.Equal(EpochStatusOpen, epoch3.Status) + }) +} + +// ------------------------------------------------------------------------------------------------ + +func populate1(database *Database) (*Application, []*Epoch, []*Input, error) { + ctx := context.Background() + + app := &Application{ + ContractAddress: common.HexToAddress("deadbeef"), + IConsensusAddress: common.HexToAddress("beefdead"), + TemplateHash: [32]byte{}, + LastProcessedBlock: 0, + Status: "RUNNING", + } + + _, err := database.InsertApplication(ctx, app) + if err != nil { + return nil, nil, nil, err + } + + epochs := []*Epoch{{ + FirstBlock: 0, + LastBlock: 1, + Status: EpochStatusClosed, + }, { + FirstBlock: 2, + LastBlock: 3, + Status: EpochStatusClosed, + }, { + FirstBlock: 4, + LastBlock: 5, + Status: EpochStatusClosed, + }, { + FirstBlock: 6, + LastBlock: 7, + Status: EpochStatusOpen, + }} + + for i, epoch := range epochs { + epoch.Index = uint64(i) + epoch.AppAddress = app.ContractAddress + epoch.Id, err = database.InsertEpoch(ctx, epoch) + if err != nil { + return nil, nil, nil, err + } + } + + inputs := []*Input{{ + EpochId: epochs[0].Id, + CompletionStatus: InputStatusAccepted, + RawData: []byte("first input"), + }, { + EpochId: epochs[0].Id, + CompletionStatus: InputStatusRejected, + RawData: []byte("second input"), + }, { + EpochId: epochs[1].Id, + CompletionStatus: InputStatusException, + RawData: []byte("third input"), + }, { + EpochId: epochs[1].Id, + CompletionStatus: InputStatusAccepted, + RawData: []byte("fourth input"), + }, { + EpochId: epochs[2].Id, + CompletionStatus: InputStatusAccepted, + RawData: []byte("fifth input"), + }, { + EpochId: epochs[2].Id, + CompletionStatus: InputStatusNone, + RawData: []byte("sixth input"), + }, { + EpochId: epochs[3].Id, + CompletionStatus: InputStatusNone, + RawData: []byte("seventh input"), + }} + + for i, input := range inputs { + input.Index = uint64(i) + input.BlockNumber = uint64(i) + input.AppAddress = app.ContractAddress + + input.RawData, err = rollupsmachine.Input{Data: input.RawData}.Encode() + if err != nil { + return nil, nil, nil, err + } + + input.Id, err = database.InsertInput(ctx, input) + if err != nil { + return nil, nil, nil, err + } + } + + return app, epochs, inputs, nil +} + +// ------------------------------------------------------------------------------------------------ + +func populate2(database *Database) ([]*Application, []*Epoch, []*Input, []*Snapshot, error) { + ctx := context.Background() + + apps := []*Application{{ + ContractAddress: common.HexToAddress("dead"), + TemplateUri: "path/to/template/uri/0", + Status: ApplicationStatusNotRunning, + }, { + ContractAddress: common.HexToAddress("beef"), + TemplateUri: "path/to/template/uri/1", + Status: ApplicationStatusRunning, + }, { + ContractAddress: common.HexToAddress("bead"), + TemplateUri: "path/to/template/uri/2", + Status: ApplicationStatusRunning, + }} + if err := database.InsertApps(ctx, apps); err != nil { + return nil, nil, nil, nil, err + } + + epochs := []*Epoch{{ + Index: 0, + Status: EpochStatusClosed, + AppAddress: apps[1].ContractAddress, + }, { + Index: 1, + Status: EpochStatusClosed, + AppAddress: apps[1].ContractAddress, + }, { + Status: EpochStatusClosed, + AppAddress: apps[2].ContractAddress, + }} + err := database.InsertEpochs(ctx, epochs) + if err != nil { + return nil, nil, nil, nil, err + } + + inputs := []*Input{{ + Index: 0, + CompletionStatus: InputStatusAccepted, + RawData: []byte("first"), + AppAddress: apps[1].ContractAddress, + EpochId: epochs[0].Id, + }, { + Index: 6, + CompletionStatus: InputStatusAccepted, + RawData: []byte("second"), + AppAddress: apps[1].ContractAddress, + EpochId: epochs[1].Id, + }} + err = database.InsertInputs(ctx, inputs) + if err != nil { + return nil, nil, nil, nil, err + } + + snapshots := []*Snapshot{{ + URI: "path/to/snapshot/1", + InputId: inputs[0].Id, + AppAddress: apps[1].ContractAddress, + }, { + URI: "path/to/snapshot/2", + InputId: inputs[1].Id, + AppAddress: apps[1].ContractAddress, + }} + err = database.InsertSnapshots(ctx, snapshots) + if err != nil { + return nil, nil, nil, nil, err + } + + return apps, epochs, inputs, snapshots, nil +} + +// ------------------------------------------------------------------------------------------------ + +func (pg *Database) InsertApps(ctx context.Context, apps []*Application) error { + var err error + for _, app := range apps { + app.Id, err = pg.InsertApplication(ctx, app) + if err != nil { + return err + } + } + return nil +} + +func (pg *Database) InsertEpochs(ctx context.Context, epochs []*Epoch) error { + var err error + for _, epoch := range epochs { + epoch.Id, err = pg.InsertEpoch(ctx, epoch) + if err != nil { + return err + } + } + return nil +} + +func (pg *Database) InsertInputs(ctx context.Context, inputs []*Input) error { + var err error + for _, input := range inputs { + input.Id, err = pg.InsertInput(ctx, input) + if err != nil { + return err + } + } + return nil +} + +func (pg *Database) InsertSnapshots(ctx context.Context, snapshots []*Snapshot) error { + var err error + for _, snapshot := range snapshots { + snapshot.Id, err = pg.InsertSnapshot(ctx, snapshot) + if err != nil { + return err + } + } + return nil +} diff --git a/internal/repository/schema/migrations/000001_create_application_input_claim_output_report_nodeconfig.up.sql b/internal/repository/schema/migrations/000001_create_application_input_claim_output_report_nodeconfig.up.sql index a4c6483fb..674a7bded 100644 --- a/internal/repository/schema/migrations/000001_create_application_input_claim_output_report_nodeconfig.up.sql +++ b/internal/repository/schema/migrations/000001_create_application_input_claim_output_report_nodeconfig.up.sql @@ -34,9 +34,17 @@ CREATE TABLE "application" "id" SERIAL, "contract_address" BYTEA NOT NULL, "template_hash" BYTEA NOT NULL, + "template_uri" VARCHAR(4096) NOT NULL, "last_processed_block" NUMERIC(20,0) NOT NULL CHECK ("last_processed_block" >= 0 AND "last_processed_block" <= f_maxuint64()), "status" "ApplicationStatus" NOT NULL, "iconsensus_address" BYTEA NOT NULL, + -- Temporary ------------------------------------- + "machine_inc_cycles" BIGINT NOT NULL, + "machine_max_cycles" BIGINT NOT NULL, + "machine_advance_timeout" INT NOT NULL, + "machine_inspect_timeout" INT NOT NULL, + "machine_max_concurrent_inspects" INT NOT NULL, + -------------------------------------------------- CONSTRAINT "application_pkey" PRIMARY KEY ("id"), UNIQUE("contract_address") ); diff --git a/internal/repository/validator_test.go b/internal/repository/validator_test.go index 92cf98bf5..676deddc2 100644 --- a/internal/repository/validator_test.go +++ b/internal/repository/validator_test.go @@ -24,7 +24,7 @@ func (s *RepositorySuite) TestGetOutputsProducedInBlockRange() { TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err = s.database.InsertApplication(s.ctx, &app) + _, err = s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) epoch := Epoch{ @@ -77,7 +77,7 @@ func (s *RepositorySuite) TestGetProcessedEpochs() { TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, &app) + _, err := s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) // no epochs, should return nothing @@ -123,7 +123,7 @@ func (s *RepositorySuite) TestGetLastInputOutputHash() { TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, &app) + _, err := s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) epoch := Epoch{ @@ -193,7 +193,7 @@ func (s *RepositorySuite) TestGetPreviousEpoch() { TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, app) + _, err := s.database.InsertApplication(s.ctx, app) s.Require().Nil(err) epoch := Epoch{ @@ -234,7 +234,7 @@ func (s *RepositorySuite) TestSetEpochClaimAndInsertProofsTransaction() { TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, &app) + _, err := s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) epoch := Epoch{ @@ -321,7 +321,7 @@ func (s *RepositorySuite) TestSetEpochClaimAndInsertProofsTransactionRollback() TemplateHash: common.BytesToHash([]byte("template")), Status: ApplicationStatusRunning, } - err := s.database.InsertApplication(s.ctx, &app) + _, err := s.database.InsertApplication(s.ctx, &app) s.Require().Nil(err) epoch := Epoch{