From 17d56b83e2de1a40f2ecc3380b347d967ab7c134 Mon Sep 17 00:00:00 2001 From: Vyzaldy Sanchez Date: Tue, 9 Apr 2024 11:41:21 -0400 Subject: [PATCH 1/5] Add support for dynamic pipeline runs with persistence (#12367) * Adds new `job_pipeline_spec` relationship * Adds changeset * Fixes migration file name * Fixes migration * Removes primary key constraint * Restores primary key + adds unique index * Fixes migration * Adds a `job_pipeline_specs` record per job being created * Adds `pruning_key` to `pipeline_runs` * Stores `pipeline_runs` records with a JobID * Fixes migration number * Prunes `pipeline_runs` by `pruning_key` instead of `pipeline_spec_id` * Uses `ExecuteAndInsertFinishedRun` on `pipeline_runner_adapter` * Deletes `job_pipeline_specs` related to `jobs` when deleting a job * Traverses `job_pipeline_specs` relation when fetching a job * Fixes linter * Fixes migration number * Fixes migration * Keeps track of JobID after job insertion - fix * Fixes `loadJobPipelineSpec` * Fixes txmgr tests * Fixes pointer error when loading the job spec * Fixes query issue when inserting runs * Fixes `loadAssociations` query method * Fixes tests WIP * Fixes tests WIP * Fixes tests WIP * Fixes lint * Fixes tests WIP * Increases test coverage * Fixes `InsertJob` relationship query * Removes unnecessary method on pipeline runner * Removes some comments from migration file * Adds missing condition on `findJob` query * Improves pipeline orm test * Reverts unnecessary change * Adds pipeline test ORM * Fixes lint * Fixes migrations out of merge conflict * Fixes test out of merge conflict * Fixes lint --- .changeset/famous-pets-confess.md | 5 + core/chains/evm/txmgr/broadcaster_test.go | 1 + core/chains/evm/txmgr/confirmer_test.go | 2 + core/chains/evm/txmgr/evm_tx_store_test.go | 4 +- core/internal/cltest/factories.go | 13 +- core/services/fluxmonitorv2/orm_test.go | 1 + core/services/job/job_orm_test.go | 22 +- .../job/job_pipeline_orm_integration_test.go | 3 +- core/services/job/models.go | 11 +- core/services/job/orm.go | 95 ++++++-- core/services/job/runner_integration_test.go | 26 ++- core/services/keeper/upkeep_executer_test.go | 7 +- .../generic/pipeline_runner_adapter.go | 5 +- .../generic/pipeline_runner_adapter_test.go | 30 ++- core/services/pipeline/mocks/orm.go | 25 ++ core/services/pipeline/mocks/runner.go | 12 +- core/services/pipeline/models.go | 3 + core/services/pipeline/orm.go | 137 +++++++---- core/services/pipeline/orm_test.go | 215 +++++++++++++++--- core/services/pipeline/runner.go | 26 ++- core/services/pipeline/runner_test.go | 47 +++- .../migrations/0231_dynamic_pipeline_runs.sql | 48 ++++ 22 files changed, 587 insertions(+), 151 deletions(-) create mode 100644 .changeset/famous-pets-confess.md create mode 100644 core/store/migrate/migrations/0231_dynamic_pipeline_runs.sql diff --git a/.changeset/famous-pets-confess.md b/.changeset/famous-pets-confess.md new file mode 100644 index 00000000000..583b17a1603 --- /dev/null +++ b/.changeset/famous-pets-confess.md @@ -0,0 +1,5 @@ +--- +"chainlink": minor +--- + +Provides support for dynamic pipeline runs with persistence. diff --git a/core/chains/evm/txmgr/broadcaster_test.go b/core/chains/evm/txmgr/broadcaster_test.go index d9e9364fdf0..1e8f1c73b34 100644 --- a/core/chains/evm/txmgr/broadcaster_test.go +++ b/core/chains/evm/txmgr/broadcaster_test.go @@ -1025,6 +1025,7 @@ func TestEthBroadcaster_ProcessUnstartedEthTxs_Errors(t *testing.T) { eb := NewTestEthBroadcaster(t, txStore, ethClient, ethKeyStore, evmcfg, &testCheckerFactory{}, false, nonceTracker) ctx := testutils.Context(t) + require.NoError(t, commonutils.JustError(db.Exec(`SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`))) require.NoError(t, commonutils.JustError(db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`))) t.Run("if external wallet sent a transaction from the account and now the nonce is one higher than it should be and we got replacement underpriced then we assume a previous transaction of ours was the one that succeeded, and hand off to EthConfirmer", func(t *testing.T) { diff --git a/core/chains/evm/txmgr/confirmer_test.go b/core/chains/evm/txmgr/confirmer_test.go index 3e200d66818..80868d448e0 100644 --- a/core/chains/evm/txmgr/confirmer_test.go +++ b/core/chains/evm/txmgr/confirmer_test.go @@ -22,6 +22,7 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" + commonclient "github.com/smartcontractkit/chainlink/v2/common/client" commonfee "github.com/smartcontractkit/chainlink/v2/common/fee" txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" @@ -2961,6 +2962,7 @@ func TestEthConfirmer_ResumePendingRuns(t *testing.T) { minConfirmations := int64(2) + pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) t.Run("doesn't process task runs that are not suspended (possibly already previously resumed)", func(t *testing.T) { diff --git a/core/chains/evm/txmgr/evm_tx_store_test.go b/core/chains/evm/txmgr/evm_tx_store_test.go index 4679ffd3339..5bb131862ed 100644 --- a/core/chains/evm/txmgr/evm_tx_store_test.go +++ b/core/chains/evm/txmgr/evm_tx_store_test.go @@ -10,6 +10,7 @@ import ( commonconfig "github.com/smartcontractkit/chainlink-common/pkg/config" "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/sqlutil" + txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" txmgrtypes "github.com/smartcontractkit/chainlink/v2/common/txmgr/types" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/assets" @@ -636,6 +637,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { ethClient := evmtest.NewEthClientMockWithDefaultChain(t) _, fromAddress := cltest.MustInsertRandomKeyReturningState(t, ethKeyStore) + pgtest.MustExec(t, db, `SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) pgtest.MustExec(t, db, `SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) head := evmtypes.Head{ @@ -665,7 +667,7 @@ func TestORM_FindTxesPendingCallback(t *testing.T) { pgtest.MustExec(t, db, `UPDATE evm.txes SET pipeline_task_run_id = $1, min_confirmations = $2, signal_callback = TRUE WHERE id = $3`, &tr1.ID, minConfirmations, etx1.ID) // Callback to pipeline service completed. Should be ignored - run2 := cltest.MustInsertPipelineRunWithStatus(t, db, 0, pipeline.RunStatusCompleted) + run2 := cltest.MustInsertPipelineRunWithStatus(t, db, 0, pipeline.RunStatusCompleted, 0) tr2 := cltest.MustInsertUnfinishedPipelineTaskRun(t, db, run2.ID) etx2 := cltest.MustInsertConfirmedEthTxWithLegacyAttempt(t, txStore, 4, 1, fromAddress) pgtest.MustExec(t, db, `UPDATE evm.txes SET meta='{"FailOnRevert": false}'`) diff --git a/core/internal/cltest/factories.go b/core/internal/cltest/factories.go index 66c96c231e7..2ca7b4947c5 100644 --- a/core/internal/cltest/factories.go +++ b/core/internal/cltest/factories.go @@ -407,6 +407,7 @@ func MustInsertKeeperJob(t *testing.T, db *sqlx.DB, korm keeper.ORM, from evmtyp jrm := job.NewORM(db, prm, btORM, nil, tlg, cfg.Database()) err = jrm.InsertJob(&jb) require.NoError(t, err) + jb.PipelineSpec.JobID = jb.ID return jb } @@ -415,13 +416,13 @@ func MustInsertKeeperRegistry(t *testing.T, db *sqlx.DB, korm keeper.ORM, ethKey from := key.EIP55Address t.Helper() contractAddress := NewEIP55Address() - job := MustInsertKeeperJob(t, db, korm, from, contractAddress) + jb := MustInsertKeeperJob(t, db, korm, from, contractAddress) registry := keeper.Registry{ ContractAddress: contractAddress, BlockCountPerTurn: blockCountPerTurn, CheckGas: 150_000, FromAddress: from, - JobID: job.ID, + JobID: jb.ID, KeeperIndex: keeperIndex, NumKeepers: numKeepers, KeeperIndexMap: map[evmtypes.EIP55Address]int32{ @@ -430,7 +431,7 @@ func MustInsertKeeperRegistry(t *testing.T, db *sqlx.DB, korm keeper.ORM, ethKey } err := korm.UpsertRegistry(®istry) require.NoError(t, err) - return registry, job + return registry, jb } func MustInsertUpkeepForRegistry(t *testing.T, db *sqlx.DB, cfg pg.QConfig, registry keeper.Registry) keeper.UpkeepRegistration { @@ -452,11 +453,11 @@ func MustInsertUpkeepForRegistry(t *testing.T, db *sqlx.DB, cfg pg.QConfig, regi } func MustInsertPipelineRun(t *testing.T, db *sqlx.DB) (run pipeline.Run) { - require.NoError(t, db.Get(&run, `INSERT INTO pipeline_runs (state,pipeline_spec_id,created_at) VALUES ($1, 0, NOW()) RETURNING *`, pipeline.RunStatusRunning)) + require.NoError(t, db.Get(&run, `INSERT INTO pipeline_runs (state,pipeline_spec_id,pruning_key,created_at) VALUES ($1, 0, 0, NOW()) RETURNING *`, pipeline.RunStatusRunning)) return run } -func MustInsertPipelineRunWithStatus(t *testing.T, db *sqlx.DB, pipelineSpecID int32, status pipeline.RunStatus) (run pipeline.Run) { +func MustInsertPipelineRunWithStatus(t *testing.T, db *sqlx.DB, pipelineSpecID int32, status pipeline.RunStatus, jobID int32) (run pipeline.Run) { var finishedAt *time.Time var outputs jsonserializable.JSONSerializable var allErrors pipeline.RunErrors @@ -478,7 +479,7 @@ func MustInsertPipelineRunWithStatus(t *testing.T, db *sqlx.DB, pipelineSpecID i default: t.Fatalf("unknown status: %s", status) } - require.NoError(t, db.Get(&run, `INSERT INTO pipeline_runs (state,pipeline_spec_id,finished_at,outputs,all_errors,fatal_errors,created_at) VALUES ($1, $2, $3, $4, $5, $6, NOW()) RETURNING *`, status, pipelineSpecID, finishedAt, outputs, allErrors, fatalErrors)) + require.NoError(t, db.Get(&run, `INSERT INTO pipeline_runs (state,pipeline_spec_id,pruning_key,finished_at,outputs,all_errors,fatal_errors,created_at) VALUES ($1, $2, $3, $4, $5, $6, $7, NOW()) RETURNING *`, status, pipelineSpecID, jobID, finishedAt, outputs, allErrors, fatalErrors)) return run } diff --git a/core/services/fluxmonitorv2/orm_test.go b/core/services/fluxmonitorv2/orm_test.go index 21a80735863..9f85a99b6ea 100644 --- a/core/services/fluxmonitorv2/orm_test.go +++ b/core/services/fluxmonitorv2/orm_test.go @@ -112,6 +112,7 @@ func TestORM_UpdateFluxMonitorRoundStats(t *testing.T) { &pipeline.Run{ State: pipeline.RunStatusCompleted, PipelineSpecID: jb.PipelineSpec.ID, + PruningKey: jb.ID, PipelineSpec: *jb.PipelineSpec, CreatedAt: time.Now(), FinishedAt: null.TimeFrom(f), diff --git a/core/services/job/job_orm_test.go b/core/services/job/job_orm_test.go index d763386a00d..1e714da5908 100644 --- a/core/services/job/job_orm_test.go +++ b/core/services/job/job_orm_test.go @@ -108,6 +108,23 @@ func TestORM(t *testing.T) { compareOCRJobSpecs(t, *jb, returnedSpec) }) + t.Run("it correctly mark job_pipeline_specs as primary when creating a job", func(t *testing.T) { + jb2 := makeOCRJobSpec(t, address, bridge.Name.String(), bridge2.Name.String()) + err := orm.CreateJob(jb2) + require.NoError(t, err) + + var pipelineSpec pipeline.Spec + err = db.Get(&pipelineSpec, "SELECT pipeline_specs.* FROM pipeline_specs JOIN job_pipeline_specs ON (pipeline_specs.id = job_pipeline_specs.pipeline_spec_id) WHERE job_pipeline_specs.job_id = $1", jb2.ID) + require.NoError(t, err) + var jobPipelineSpec job.PipelineSpec + err = db.Get(&jobPipelineSpec, "SELECT * FROM job_pipeline_specs WHERE job_id = $1 AND pipeline_spec_id = $2", jb2.ID, pipelineSpec.ID) + require.NoError(t, err) + + // `jb2.PipelineSpecID` gets loaded when calling `orm.CreateJob()` so we can compare it directly + assert.Equal(t, jb2.PipelineSpecID, pipelineSpec.ID) + assert.True(t, jobPipelineSpec.IsPrimary) + }) + t.Run("autogenerates external job ID if missing", func(t *testing.T) { jb2 := makeOCRJobSpec(t, address, bridge.Name.String(), bridge2.Name.String()) jb2.ExternalJobID = uuid.UUID{} @@ -126,7 +143,7 @@ func TestORM(t *testing.T) { err := db.Select(&dbSpecs, "SELECT * FROM jobs") require.NoError(t, err) - require.Len(t, dbSpecs, 2) + require.Len(t, dbSpecs, 3) err = orm.DeleteJob(jb.ID) require.NoError(t, err) @@ -134,7 +151,7 @@ func TestORM(t *testing.T) { dbSpecs = []job.Job{} err = db.Select(&dbSpecs, "SELECT * FROM jobs") require.NoError(t, err) - require.Len(t, dbSpecs, 1) + require.Len(t, dbSpecs, 2) }) t.Run("increase job spec error occurrence", func(t *testing.T) { @@ -1729,6 +1746,7 @@ func mustInsertPipelineRun(t *testing.T, orm pipeline.ORM, j job.Job) pipeline.R run := pipeline.Run{ PipelineSpecID: j.PipelineSpecID, + PruningKey: j.ID, State: pipeline.RunStatusRunning, Outputs: jsonserializable.JSONSerializable{Valid: false}, AllErrors: pipeline.RunErrors{}, diff --git a/core/services/job/job_pipeline_orm_integration_test.go b/core/services/job/job_pipeline_orm_integration_test.go index dd3062fa14b..c7842e1b160 100644 --- a/core/services/job/job_pipeline_orm_integration_test.go +++ b/core/services/job/job_pipeline_orm_integration_test.go @@ -10,6 +10,7 @@ import ( "github.com/jmoiron/sqlx" commonconfig "github.com/smartcontractkit/chainlink-common/pkg/config" + "github.com/smartcontractkit/chainlink/v2/core/bridges" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils" @@ -165,7 +166,7 @@ func TestPipelineORM_Integration(t *testing.T) { require.NoError(t, jobORM.CreateJob(dbSpec)) var pipelineSpecs []pipeline.Spec - sql := `SELECT * FROM pipeline_specs;` + sql := `SELECT pipeline_specs.*, job_pipeline_specs.job_id FROM pipeline_specs JOIN job_pipeline_specs ON (pipeline_specs.id = job_pipeline_specs.pipeline_spec_id);` require.NoError(t, db.Select(&pipelineSpecs, sql)) require.Len(t, pipelineSpecs, 1) require.Equal(t, dbSpec.PipelineSpecID, pipelineSpecs[0].ID) diff --git a/core/services/job/models.go b/core/services/job/models.go index 218be21bc54..4d75c7d90d2 100644 --- a/core/services/job/models.go +++ b/core/services/job/models.go @@ -163,7 +163,7 @@ type Job struct { EALSpecID *int32 LiquidityBalancerSpec *LiquidityBalancerSpec LiquidityBalancerSpecID *int32 - PipelineSpecID int32 + PipelineSpecID int32 // This is deprecated in favor of the `job_pipeline_specs` table relationship PipelineSpec *pipeline.Spec JobSpecErrors []SpecError Type Type `toml:"type"` @@ -208,6 +208,12 @@ func (j *Job) SetID(value string) error { return nil } +type PipelineSpec struct { + JobID int32 `json:"-"` + PipelineSpecID int32 `json:"-"` + IsPrimary bool `json:"is_primary"` +} + type SpecError struct { ID int64 JobID int32 @@ -229,7 +235,8 @@ func (j *SpecError) SetID(value string) error { } type PipelineRun struct { - ID int64 `json:"-"` + ID int64 `json:"-"` + PruningKey int64 `json:"-"` } func (pr PipelineRun) GetID() string { diff --git a/core/services/job/orm.go b/core/services/job/orm.go index 6c8533d1dee..c05e944ea1e 100644 --- a/core/services/job/orm.go +++ b/core/services/job/orm.go @@ -534,41 +534,51 @@ func (o *orm) InsertWebhookSpec(webhookSpec *WebhookSpec, qopts ...pg.QOpt) erro func (o *orm) InsertJob(job *Job, qopts ...pg.QOpt) error { q := o.q.WithOpts(qopts...) - var query string + return q.Transaction(func(querier pg.Queryer) error { + var query string - // if job has id, emplace otherwise insert with a new id. - if job.ID == 0 { - query = `INSERT INTO jobs (pipeline_spec_id, name, stream_id, schema_version, type, max_task_duration, ocr_oracle_spec_id, ocr2_oracle_spec_id, direct_request_spec_id, flux_monitor_spec_id, + // if job has id, emplace otherwise insert with a new id. + if job.ID == 0 { + query = `INSERT INTO jobs (name, stream_id, schema_version, type, max_task_duration, ocr_oracle_spec_id, ocr2_oracle_spec_id, direct_request_spec_id, flux_monitor_spec_id, keeper_spec_id, cron_spec_id, vrf_spec_id, webhook_spec_id, blockhash_store_spec_id, bootstrap_spec_id, block_header_feeder_spec_id, gateway_spec_id, legacy_gas_station_server_spec_id, legacy_gas_station_sidecar_spec_id, external_job_id, gas_limit, forwarding_allowed, created_at) - VALUES (:pipeline_spec_id, :name, :stream_id, :schema_version, :type, :max_task_duration, :ocr_oracle_spec_id, :ocr2_oracle_spec_id, :direct_request_spec_id, :flux_monitor_spec_id, + VALUES (:name, :stream_id, :schema_version, :type, :max_task_duration, :ocr_oracle_spec_id, :ocr2_oracle_spec_id, :direct_request_spec_id, :flux_monitor_spec_id, :keeper_spec_id, :cron_spec_id, :vrf_spec_id, :webhook_spec_id, :blockhash_store_spec_id, :bootstrap_spec_id, :block_header_feeder_spec_id, :gateway_spec_id, :legacy_gas_station_server_spec_id, :legacy_gas_station_sidecar_spec_id, :external_job_id, :gas_limit, :forwarding_allowed, NOW()) RETURNING *;` - } else { - query = `INSERT INTO jobs (id, pipeline_spec_id, name, stream_id, schema_version, type, max_task_duration, ocr_oracle_spec_id, ocr2_oracle_spec_id, direct_request_spec_id, flux_monitor_spec_id, + } else { + query = `INSERT INTO jobs (id, name, stream_id, schema_version, type, max_task_duration, ocr_oracle_spec_id, ocr2_oracle_spec_id, direct_request_spec_id, flux_monitor_spec_id, keeper_spec_id, cron_spec_id, vrf_spec_id, webhook_spec_id, blockhash_store_spec_id, bootstrap_spec_id, block_header_feeder_spec_id, gateway_spec_id, legacy_gas_station_server_spec_id, legacy_gas_station_sidecar_spec_id, external_job_id, gas_limit, forwarding_allowed, created_at) - VALUES (:id, :pipeline_spec_id, :name, :stream_id, :schema_version, :type, :max_task_duration, :ocr_oracle_spec_id, :ocr2_oracle_spec_id, :direct_request_spec_id, :flux_monitor_spec_id, + VALUES (:id, :name, :stream_id, :schema_version, :type, :max_task_duration, :ocr_oracle_spec_id, :ocr2_oracle_spec_id, :direct_request_spec_id, :flux_monitor_spec_id, :keeper_spec_id, :cron_spec_id, :vrf_spec_id, :webhook_spec_id, :blockhash_store_spec_id, :bootstrap_spec_id, :block_header_feeder_spec_id, :gateway_spec_id, :legacy_gas_station_server_spec_id, :legacy_gas_station_sidecar_spec_id, :external_job_id, :gas_limit, :forwarding_allowed, NOW()) RETURNING *;` - } - return q.GetNamed(query, job, job) + } + err := q.GetNamed(query, job, job) + if err != nil { + return err + } + + // Always inserts the `job_pipeline_specs` record as primary, since this is the first one for the job. + sqlStmt := `INSERT INTO job_pipeline_specs (job_id, pipeline_spec_id, is_primary) VALUES ($1, $2, true)` + _, err = q.Exec(sqlStmt, job.ID, job.PipelineSpecID) + return errors.Wrap(err, "failed to insert job_pipeline_specs relationship") + }) } // DeleteJob removes a job func (o *orm) DeleteJob(id int32, qopts ...pg.QOpt) error { o.lggr.Debugw("Deleting job", "jobID", id) - // Added a 1 minute timeout to this query since this can take a long time as data increases. - // This was added specifically due to an issue with a database that had a millions of pipeline_runs and pipeline_task_runs + // Added a 1-minute timeout to this query since this can take a long time as data increases. + // This was added specifically due to an issue with a database that had a million of pipeline_runs and pipeline_task_runs // and this query was taking ~40secs. qopts = append(qopts, pg.WithLongQueryTimeout()) q := o.q.WithOpts(qopts...) query := ` WITH deleted_jobs AS ( DELETE FROM jobs WHERE id = $1 RETURNING - pipeline_spec_id, + id, ocr_oracle_spec_id, ocr2_oracle_spec_id, keeper_spec_id, @@ -617,8 +627,11 @@ func (o *orm) DeleteJob(id int32, qopts ...pg.QOpt) error { ), deleted_gateway_specs AS ( DELETE FROM gateway_specs WHERE id IN (SELECT gateway_spec_id FROM deleted_jobs) + ), + deleted_job_pipeline_specs AS ( + DELETE FROM job_pipeline_specs WHERE job_id IN (SELECT id FROM deleted_jobs) RETURNING pipeline_spec_id ) - DELETE FROM pipeline_specs WHERE id IN (SELECT pipeline_spec_id FROM deleted_jobs)` + DELETE FROM pipeline_specs WHERE id IN (SELECT pipeline_spec_id FROM deleted_job_pipeline_specs)` res, cancel, err := q.ExecQIter(query, id) defer cancel() if err != nil { @@ -692,7 +705,10 @@ func (o *orm) FindJobs(offset, limit int) (jobs []Job, count int, err error) { return err } - sql = `SELECT * FROM jobs ORDER BY created_at DESC, id DESC OFFSET $1 LIMIT $2;` + sql = `SELECT jobs.*, job_pipeline_specs.pipeline_spec_id as pipeline_spec_id + FROM jobs + JOIN job_pipeline_specs ON (jobs.id = job_pipeline_specs.job_id) + ORDER BY jobs.created_at DESC, jobs.id DESC OFFSET $1 LIMIT $2;` err = tx.Select(&jobs, sql, offset, limit) if err != nil { return err @@ -807,7 +823,7 @@ func (o *orm) FindJob(ctx context.Context, id int32) (jb Job, err error) { // FindJobWithoutSpecErrors returns a job by ID, without loading Spec Errors preloaded func (o *orm) FindJobWithoutSpecErrors(id int32) (jb Job, err error) { err = o.q.Transaction(func(tx pg.Queryer) error { - stmt := "SELECT * FROM jobs WHERE id = $1 LIMIT 1" + stmt := "SELECT jobs.*, job_pipeline_specs.pipeline_spec_id as pipeline_spec_id FROM jobs JOIN job_pipeline_specs ON (jobs.id = job_pipeline_specs.job_id) WHERE jobs.id = $1 LIMIT 1" err = tx.Get(&jb, stmt, id) if err != nil { return errors.Wrap(err, "failed to load job") @@ -897,7 +913,7 @@ WHERE ocr2spec.id IS NOT NULL OR bs.id IS NOT NULL func (o *orm) findJob(jb *Job, col string, arg interface{}, qopts ...pg.QOpt) error { q := o.q.WithOpts(qopts...) err := q.Transaction(func(tx pg.Queryer) error { - sql := fmt.Sprintf(`SELECT * FROM jobs WHERE %s = $1 LIMIT 1`, col) + sql := fmt.Sprintf(`SELECT jobs.*, job_pipeline_specs.pipeline_spec_id FROM jobs JOIN job_pipeline_specs ON (jobs.id = job_pipeline_specs.job_id) WHERE jobs.%s = $1 AND job_pipeline_specs.is_primary = true LIMIT 1`, col) err := tx.Get(jb, sql, arg) if err != nil { return errors.Wrap(err, "failed to load job") @@ -917,7 +933,13 @@ func (o *orm) findJob(jb *Job, col string, arg interface{}, qopts ...pg.QOpt) er func (o *orm) FindJobIDsWithBridge(name string) (jids []int32, err error) { err = o.q.Transaction(func(tx pg.Queryer) error { - query := `SELECT jobs.id, dot_dag_source FROM jobs JOIN pipeline_specs ON pipeline_specs.id = jobs.pipeline_spec_id WHERE dot_dag_source ILIKE '%' || $1 || '%' ORDER BY id` + query := `SELECT + jobs.id, pipeline_specs.dot_dag_source + FROM jobs + JOIN job_pipeline_specs ON job_pipeline_specs.job_id = jobs.id + JOIN pipeline_specs ON pipeline_specs.id = job_pipeline_specs.pipeline_spec_id + WHERE pipeline_specs.dot_dag_source ILIKE '%' || $1 || '%' ORDER BY id` + var rows *sqlx.Rows rows, err = tx.Queryx(query, name) if err != nil { @@ -958,7 +980,7 @@ func (o *orm) FindJobIDsWithBridge(name string) (jids []int32, err error) { // PipelineRunsByJobsIDs returns pipeline runs for multiple jobs, not preloading data func (o *orm) PipelineRunsByJobsIDs(ids []int32) (runs []pipeline.Run, err error) { err = o.q.Transaction(func(tx pg.Queryer) error { - stmt := `SELECT pipeline_runs.* FROM pipeline_runs INNER JOIN jobs ON pipeline_runs.pipeline_spec_id = jobs.pipeline_spec_id WHERE jobs.id = ANY($1) + stmt := `SELECT pipeline_runs.* FROM pipeline_runs INNER JOIN job_pipeline_specs ON pipeline_runs.pipeline_spec_id = job_pipeline_specs.pipeline_spec_id WHERE jobs.id = ANY($1) ORDER BY pipeline_runs.created_at DESC, pipeline_runs.id DESC;` if err = tx.Select(&runs, stmt, ids); err != nil { return errors.Wrap(err, "error loading runs") @@ -987,7 +1009,7 @@ func (o *orm) loadPipelineRunIDs(jobID *int32, offset, limit int, tx pg.Queryer) var filter string if jobID != nil { - filter = fmt.Sprintf("JOIN jobs USING(pipeline_spec_id) WHERE jobs.id = %d AND ", *jobID) + filter = fmt.Sprintf("JOIN job_pipeline_specs USING(pipeline_spec_id) WHERE job_pipeline_specs.job_id = %d AND ", *jobID) } else { filter = "WHERE " } @@ -1132,7 +1154,7 @@ WHERE id = $1 // CountPipelineRunsByJobID returns the total number of pipeline runs for a job. func (o *orm) CountPipelineRunsByJobID(jobID int32) (count int32, err error) { err = o.q.Transaction(func(tx pg.Queryer) error { - stmt := "SELECT COUNT(*) FROM pipeline_runs JOIN jobs USING (pipeline_spec_id) WHERE jobs.id = $1" + stmt := "SELECT COUNT(*) FROM pipeline_runs JOIN job_pipeline_specs USING (pipeline_spec_id) WHERE job_pipeline_specs.job_id = $1" if err = tx.Get(&count, stmt, jobID); err != nil { return errors.Wrap(err, "error counting runs") } @@ -1147,7 +1169,7 @@ func (o *orm) FindJobsByPipelineSpecIDs(ids []int32) ([]Job, error) { var jbs []Job err := o.q.Transaction(func(tx pg.Queryer) error { - stmt := `SELECT * FROM jobs WHERE jobs.pipeline_spec_id = ANY($1) ORDER BY id ASC + stmt := `SELECT jobs.*, job_pipeline_specs.pipeline_spec_id FROM jobs JOIN job_pipeline_specs ON (jobs.id = job_pipeline_specs.job_id) WHERE job_pipeline_specs.pipeline_spec_id = ANY($1) ORDER BY jobs.id ASC ` if err := tx.Select(&jbs, stmt, ids); err != nil { return errors.Wrap(err, "error fetching jobs by pipeline spec IDs") @@ -1169,7 +1191,7 @@ func (o *orm) FindJobsByPipelineSpecIDs(ids []int32) ([]Job, error) { func (o *orm) PipelineRuns(jobID *int32, offset, size int) (runs []pipeline.Run, count int, err error) { var filter string if jobID != nil { - filter = fmt.Sprintf("JOIN jobs USING(pipeline_spec_id) WHERE jobs.id = %d", *jobID) + filter = fmt.Sprintf("JOIN job_pipeline_specs USING(pipeline_spec_id) WHERE job_pipeline_specs.job_id = %d", *jobID) } err = o.q.Transaction(func(tx pg.Queryer) error { sql := fmt.Sprintf(`SELECT count(*) FROM pipeline_runs %s`, filter) @@ -1200,7 +1222,7 @@ func (o *orm) loadPipelineRunsRelations(runs []pipeline.Run, tx pg.Queryer) ([]p for specID := range specM { specIDs = append(specIDs, specID) } - stmt := `SELECT pipeline_specs.*, jobs.id AS job_id FROM pipeline_specs JOIN jobs ON pipeline_specs.id = jobs.pipeline_spec_id WHERE pipeline_specs.id = ANY($1);` + stmt := `SELECT pipeline_specs.*, job_pipeline_specs.job_id AS job_id FROM pipeline_specs JOIN job_pipeline_specs ON pipeline_specs.id = job_pipeline_specs.pipeline_spec_id WHERE pipeline_specs.id = ANY($1);` var specs []pipeline.Spec if err := o.q.Select(&specs, stmt, specIDs); err != nil { return nil, errors.Wrap(err, "error loading specs") @@ -1247,7 +1269,7 @@ func LoadAllJobsTypes(tx pg.Queryer, jobs []Job) error { func LoadAllJobTypes(tx pg.Queryer, job *Job) error { return multierr.Combine( - loadJobType(tx, job, "PipelineSpec", "pipeline_specs", &job.PipelineSpecID), + loadJobPipelineSpec(tx, job, &job.PipelineSpecID), loadJobType(tx, job, "FluxMonitorSpec", "flux_monitor_specs", job.FluxMonitorSpecID), loadJobType(tx, job, "DirectRequestSpec", "direct_request_specs", job.DirectRequestSpecID), loadJobType(tx, job, "OCROracleSpec", "ocr_oracle_specs", job.OCROracleSpecID), @@ -1287,6 +1309,29 @@ func loadJobType(tx pg.Queryer, job *Job, field, table string, id *int32) error return nil } +func loadJobPipelineSpec(tx pg.Queryer, job *Job, id *int32) error { + if id == nil { + return nil + } + pipelineSpecRow := new(pipeline.Spec) + if job.PipelineSpec != nil { + pipelineSpecRow = job.PipelineSpec + } + err := tx.Get( + pipelineSpecRow, + `SELECT pipeline_specs.*, job_pipeline_specs.job_id as job_id + FROM pipeline_specs + JOIN job_pipeline_specs ON(pipeline_specs.id = job_pipeline_specs.pipeline_spec_id) + WHERE job_pipeline_specs.job_id = $1 AND job_pipeline_specs.pipeline_spec_id = $2`, + job.ID, *id, + ) + if err != nil { + return errors.Wrapf(err, "failed to load job type PipelineSpec with id %d", *id) + } + job.PipelineSpec = pipelineSpecRow + return nil +} + func loadVRFJob(tx pg.Queryer, job *Job, id *int32) error { if id == nil { return nil diff --git a/core/services/job/runner_integration_test.go b/core/services/job/runner_integration_test.go index 3a1f69afa1b..110d4a41a91 100644 --- a/core/services/job/runner_integration_test.go +++ b/core/services/job/runner_integration_test.go @@ -126,9 +126,10 @@ func TestRunner(t *testing.T) { m, err := bridges.MarshalBridgeMetaData(big.NewInt(10), big.NewInt(100)) require.NoError(t, err) - runID, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(map[string]interface{}{"jobRun": map[string]interface{}{"meta": m}}), logger.TestLogger(t), true) + runID, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(map[string]interface{}{"jobRun": map[string]interface{}{"meta": m}}), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) require.Len(t, results.Values, 2) require.GreaterOrEqual(t, len(results.FatalErrors), 2) assert.Nil(t, results.FatalErrors[0]) @@ -313,9 +314,10 @@ answer1 [type=median index=0]; err := jobORM.CreateJob(jb) require.NoError(t, err) - runID, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + runID, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) assert.Len(t, results.FatalErrors, 1) assert.Len(t, results.Values, 1) assert.Contains(t, results.FatalErrors[0].Error(), "type cannot be converted to decimal.Decimal") @@ -358,9 +360,10 @@ answer1 [type=median index=0]; err := jobORM.CreateJob(jb) require.NoError(t, err) - runID, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + runID, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) assert.Len(t, results.Values, 1) assert.Len(t, results.FatalErrors, 1) assert.Contains(t, results.FatalErrors[0].Error(), pipeline.ErrTooManyErrors.Error()) @@ -402,9 +405,10 @@ answer1 [type=median index=0]; err := jobORM.CreateJob(jb) require.NoError(t, err) - runID, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + runID, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) assert.Len(t, results.Values, 1) assert.Contains(t, results.FatalErrors[0].Error(), "type cannot be converted to decimal.Decimal") assert.Nil(t, results.Values[0]) @@ -685,8 +689,9 @@ answer1 [type=median index=0]; err := jobORM.CreateJob(jb) require.NoError(t, err) - _, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + _, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) assert.Nil(t, results.Values[0]) // No task timeout should succeed. @@ -694,8 +699,9 @@ answer1 [type=median index=0]; jb.Name = null.NewString("a job 2", true) err = jobORM.CreateJob(jb) require.NoError(t, err) - _, results, err = runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + _, taskResults, err = runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results = taskResults.FinalResult(logger.TestLogger(t)) assert.Equal(t, 10.1, results.Values[0]) assert.Nil(t, results.FatalErrors[0]) @@ -706,9 +712,10 @@ answer1 [type=median index=0]; err = jobORM.CreateJob(jb) require.NoError(t, err) - _, results, err = runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + _, taskResults, err = runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) - assert.NotNil(t, results.FatalErrors[0]) + resultsNoFatalErrs := taskResults.FinalResult(logger.TestLogger(t)) + assert.NotNil(t, resultsNoFatalErrs.FatalErrors[0]) }) t.Run("deleting jobs", func(t *testing.T) { @@ -724,8 +731,9 @@ answer1 [type=median index=0]; err := jobORM.CreateJob(jb) require.NoError(t, err) - _, results, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) + _, taskResults, err := runner.ExecuteAndInsertFinishedRun(testutils.Context(t), *jb.PipelineSpec, pipeline.NewVarsFrom(nil), logger.TestLogger(t), true) require.NoError(t, err) + results := taskResults.FinalResult(logger.TestLogger(t)) assert.Len(t, results.Values, 1) assert.Nil(t, results.FatalErrors[0]) assert.Equal(t, "4242", results.Values[0].(decimal.Decimal).String()) diff --git a/core/services/keeper/upkeep_executer_test.go b/core/services/keeper/upkeep_executer_test.go index 8299f47c853..fbe61f35743 100644 --- a/core/services/keeper/upkeep_executer_test.go +++ b/core/services/keeper/upkeep_executer_test.go @@ -16,6 +16,7 @@ import ( "github.com/jmoiron/sqlx" "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" + "github.com/smartcontractkit/chainlink/v2/core/chains/evm/assets" evmclimocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client/mocks" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas" @@ -85,13 +86,13 @@ func setup(t *testing.T, estimator gas.EvmFeeEstimator, overrideFn func(c *chain jpv2 := cltest.NewJobPipelineV2(t, cfg.WebServer(), cfg.JobPipeline(), cfg.Database(), legacyChains, db, keyStore, nil, nil) ch := evmtest.MustGetDefaultChain(t, legacyChains) orm := keeper.NewORM(db, logger.TestLogger(t), ch.Config().Database()) - registry, job := cltest.MustInsertKeeperRegistry(t, db, orm, keyStore.Eth(), 0, 1, 20) + registry, jb := cltest.MustInsertKeeperRegistry(t, db, orm, keyStore.Eth(), 0, 1, 20) lggr := logger.TestLogger(t) - executer := keeper.NewUpkeepExecuter(job, orm, jpv2.Pr, ethClient, ch.HeadBroadcaster(), ch.GasEstimator(), lggr, ch.Config().Keeper(), job.KeeperSpec.FromAddress.Address()) + executer := keeper.NewUpkeepExecuter(jb, orm, jpv2.Pr, ethClient, ch.HeadBroadcaster(), ch.GasEstimator(), lggr, ch.Config().Keeper(), jb.KeeperSpec.FromAddress.Address()) upkeep := cltest.MustInsertUpkeepForRegistry(t, db, ch.Config().Database(), registry) servicetest.Run(t, executer) - return db, cfg, ethClient, executer, registry, upkeep, job, jpv2, txm, keyStore, ch, orm + return db, cfg, ethClient, executer, registry, upkeep, jb, jpv2, txm, keyStore, ch, orm } var checkUpkeepResponse = struct { diff --git a/core/services/ocr2/plugins/generic/pipeline_runner_adapter.go b/core/services/ocr2/plugins/generic/pipeline_runner_adapter.go index b13d7b35e0b..e6a429a5f73 100644 --- a/core/services/ocr2/plugins/generic/pipeline_runner_adapter.go +++ b/core/services/ocr2/plugins/generic/pipeline_runner_adapter.go @@ -5,6 +5,7 @@ import ( "time" "github.com/smartcontractkit/chainlink-common/pkg/types" + "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/services/job" "github.com/smartcontractkit/chainlink/v2/core/services/pipeline" @@ -14,7 +15,7 @@ import ( var _ types.PipelineRunnerService = (*PipelineRunnerAdapter)(nil) type pipelineRunner interface { - ExecuteRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger) (run *pipeline.Run, trrs pipeline.TaskRunResults, err error) + ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, results pipeline.TaskRunResults, err error) } type PipelineRunnerAdapter struct { @@ -44,7 +45,7 @@ func (p *PipelineRunnerAdapter) ExecuteRun(ctx context.Context, spec string, var merge(defaultVars, vars.Vars) finalVars := pipeline.NewVarsFrom(defaultVars) - _, trrs, err := p.runner.ExecuteRun(ctx, s, finalVars, p.logger) + _, trrs, err := p.runner.ExecuteAndInsertFinishedRun(ctx, s, finalVars, p.logger, true) if err != nil { return nil, err } diff --git a/core/services/ocr2/plugins/generic/pipeline_runner_adapter_test.go b/core/services/ocr2/plugins/generic/pipeline_runner_adapter_test.go index a4bc8eb0b16..569d5b49364 100644 --- a/core/services/ocr2/plugins/generic/pipeline_runner_adapter_test.go +++ b/core/services/ocr2/plugins/generic/pipeline_runner_adapter_test.go @@ -13,7 +13,9 @@ import ( "gopkg.in/guregu/null.v4" "github.com/smartcontractkit/chainlink-common/pkg/types" + "github.com/smartcontractkit/chainlink/v2/core/bridges" + "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/configtest" _ "github.com/smartcontractkit/chainlink/v2/core/internal/testutils/pgtest" @@ -21,8 +23,10 @@ import ( "github.com/smartcontractkit/chainlink/v2/core/services/job" "github.com/smartcontractkit/chainlink/v2/core/services/keystore" "github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/generic" + ocr2validate "github.com/smartcontractkit/chainlink/v2/core/services/ocr2/validate" "github.com/smartcontractkit/chainlink/v2/core/services/pg" "github.com/smartcontractkit/chainlink/v2/core/services/pipeline" + "github.com/smartcontractkit/chainlink/v2/core/testdata/testspecs" "github.com/smartcontractkit/chainlink/v2/core/utils" ) @@ -41,6 +45,7 @@ func TestAdapter_Integration(t *testing.T) { keystore := keystore.NewInMemory(db, utils.FastScryptParams, logger, cfg.Database()) pipelineORM := pipeline.NewORM(db, logger, cfg.Database(), cfg.JobPipeline().MaxSuccessfulRuns()) bridgesORM := bridges.NewORM(db, logger, cfg.Database()) + jobORM := job.NewORM(db, pipelineORM, bridgesORM, keystore, logger, cfg.Database()) pr := pipeline.NewRunner( pipelineORM, bridgesORM, @@ -53,7 +58,24 @@ func TestAdapter_Integration(t *testing.T) { http.DefaultClient, http.DefaultClient, ) - pra := generic.NewPipelineRunnerAdapter(logger, job.Job{}, pr) + err = keystore.Unlock(cfg.Password().Keystore()) + require.NoError(t, err) + jb, err := ocr2validate.ValidatedOracleSpecToml(testutils.Context(t), cfg.OCR2(), cfg.Insecure(), testspecs.GetOCR2EVMSpecMinimal(), nil) + require.NoError(t, err) + + const juelsPerFeeCoinSource = ` + ds [type=http method=GET url="https://chain.link/ETH-USD"]; + ds_parse [type=jsonparse path="data.price" separator="."]; + ds_multiply [type=multiply times=100]; + ds -> ds_parse -> ds_multiply;` + + _, address := cltest.MustInsertRandomKey(t, keystore.Eth()) + jb.Name = null.StringFrom("Job 1") + jb.OCR2OracleSpec.TransmitterID = null.StringFrom(address.String()) + jb.OCR2OracleSpec.PluginConfig["juelsPerFeeCoinSource"] = juelsPerFeeCoinSource + err = jobORM.CreateJob(&jb) + require.NoError(t, err) + pra := generic.NewPipelineRunnerAdapter(logger, jb, pr) results, err := pra.ExecuteRun(testutils.Context(t), spec, types.Vars{Vars: map[string]interface{}{"val": 1}}, types.Options{}) require.NoError(t, err) @@ -69,15 +91,15 @@ func newMockPipelineRunner() *mockPipelineRunner { type mockPipelineRunner struct { results pipeline.TaskRunResults err error - run *pipeline.Run spec pipeline.Spec vars pipeline.Vars } -func (m *mockPipelineRunner) ExecuteRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger) (*pipeline.Run, pipeline.TaskRunResults, error) { +func (m *mockPipelineRunner) ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, results pipeline.TaskRunResults, err error) { m.spec = spec m.vars = vars - return m.run, m.results, m.err + // We never attach a run to the mock, so we can't return a runID + return 0, m.results, m.err } func TestAdapter_AddsDefaultVars(t *testing.T) { diff --git a/core/services/pipeline/mocks/orm.go b/core/services/pipeline/mocks/orm.go index 759686204d4..b06041767a1 100644 --- a/core/services/pipeline/mocks/orm.go +++ b/core/services/pipeline/mocks/orm.go @@ -275,6 +275,31 @@ func (_m *ORM) InsertFinishedRun(run *pipeline.Run, saveSuccessfulTaskRuns bool, return r0 } +// InsertFinishedRunWithSpec provides a mock function with given fields: run, saveSuccessfulTaskRuns, qopts +func (_m *ORM) InsertFinishedRunWithSpec(run *pipeline.Run, saveSuccessfulTaskRuns bool, qopts ...pg.QOpt) error { + _va := make([]interface{}, len(qopts)) + for _i := range qopts { + _va[_i] = qopts[_i] + } + var _ca []interface{} + _ca = append(_ca, run, saveSuccessfulTaskRuns) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for InsertFinishedRunWithSpec") + } + + var r0 error + if rf, ok := ret.Get(0).(func(*pipeline.Run, bool, ...pg.QOpt) error); ok { + r0 = rf(run, saveSuccessfulTaskRuns, qopts...) + } else { + r0 = ret.Error(0) + } + + return r0 +} + // InsertFinishedRuns provides a mock function with given fields: run, saveSuccessfulTaskRuns, qopts func (_m *ORM) InsertFinishedRuns(run []*pipeline.Run, saveSuccessfulTaskRuns bool, qopts ...pg.QOpt) error { _va := make([]interface{}, len(qopts)) diff --git a/core/services/pipeline/mocks/runner.go b/core/services/pipeline/mocks/runner.go index 1de72bbf4c0..3de2703f0c7 100644 --- a/core/services/pipeline/mocks/runner.go +++ b/core/services/pipeline/mocks/runner.go @@ -39,7 +39,7 @@ func (_m *Runner) Close() error { } // ExecuteAndInsertFinishedRun provides a mock function with given fields: ctx, spec, vars, l, saveSuccessfulTaskRuns -func (_m *Runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (int64, pipeline.FinalResult, error) { +func (_m *Runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline.Spec, vars pipeline.Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (int64, pipeline.TaskRunResults, error) { ret := _m.Called(ctx, spec, vars, l, saveSuccessfulTaskRuns) if len(ret) == 0 { @@ -47,9 +47,9 @@ func (_m *Runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline } var r0 int64 - var r1 pipeline.FinalResult + var r1 pipeline.TaskRunResults var r2 error - if rf, ok := ret.Get(0).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) (int64, pipeline.FinalResult, error)); ok { + if rf, ok := ret.Get(0).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) (int64, pipeline.TaskRunResults, error)); ok { return rf(ctx, spec, vars, l, saveSuccessfulTaskRuns) } if rf, ok := ret.Get(0).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) int64); ok { @@ -58,10 +58,12 @@ func (_m *Runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec pipeline r0 = ret.Get(0).(int64) } - if rf, ok := ret.Get(1).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) pipeline.FinalResult); ok { + if rf, ok := ret.Get(1).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) pipeline.TaskRunResults); ok { r1 = rf(ctx, spec, vars, l, saveSuccessfulTaskRuns) } else { - r1 = ret.Get(1).(pipeline.FinalResult) + if ret.Get(1) != nil { + r1 = ret.Get(1).(pipeline.TaskRunResults) + } } if rf, ok := ret.Get(2).(func(context.Context, pipeline.Spec, pipeline.Vars, logger.Logger, bool) error); ok { diff --git a/core/services/pipeline/models.go b/core/services/pipeline/models.go index e0596700e08..fa1a2605f03 100644 --- a/core/services/pipeline/models.go +++ b/core/services/pipeline/models.go @@ -15,6 +15,7 @@ import ( "gopkg.in/guregu/null.v4" "github.com/smartcontractkit/chainlink-common/pkg/utils/jsonserializable" + "github.com/smartcontractkit/chainlink/v2/core/store/models" ) @@ -46,7 +47,9 @@ func (s *Spec) ParsePipeline() (*Pipeline, error) { type Run struct { ID int64 `json:"-"` + JobID int32 `json:"-"` PipelineSpecID int32 `json:"-"` + PruningKey int32 `json:"-"` // This currently refers to the upstream job ID PipelineSpec Spec `json:"pipelineSpec"` Meta jsonserializable.JSONSerializable `json:"meta"` // The errors are only ever strings diff --git a/core/services/pipeline/orm.go b/core/services/pipeline/orm.go index 602746ffffb..c32693e4db4 100644 --- a/core/services/pipeline/orm.go +++ b/core/services/pipeline/orm.go @@ -82,6 +82,7 @@ type ORM interface { StoreRun(run *Run, qopts ...pg.QOpt) (restart bool, err error) UpdateTaskRunResult(taskID uuid.UUID, result Result) (run Run, start bool, err error) InsertFinishedRun(run *Run, saveSuccessfulTaskRuns bool, qopts ...pg.QOpt) (err error) + InsertFinishedRunWithSpec(run *Run, saveSuccessfulTaskRuns bool, qopts ...pg.QOpt) (err error) // InsertFinishedRuns inserts all the given runs into the database. // If saveSuccessfulTaskRuns is false, only errored runs are saved. @@ -194,11 +195,11 @@ func (o *orm) CreateRun(run *Run, qopts ...pg.QOpt) (err error) { // InsertRun inserts a run into the database func (o *orm) InsertRun(run *Run, qopts ...pg.QOpt) error { if run.Status() == RunStatusCompleted { - defer o.Prune(o.q, run.PipelineSpecID) + defer o.Prune(o.q, run.PruningKey) } q := o.q.WithOpts(qopts...) - sql := `INSERT INTO pipeline_runs (pipeline_spec_id, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) - VALUES (:pipeline_spec_id, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) + sql := `INSERT INTO pipeline_runs (pipeline_spec_id, pruning_key, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) + VALUES (:pipeline_spec_id, :pruning_key, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) RETURNING *;` return q.GetNamed(sql, run, run) } @@ -249,7 +250,7 @@ func (o *orm) StoreRun(run *Run, qopts ...pg.QOpt) (restart bool, err error) { return errors.Wrap(err, "StoreRun") } } else { - defer o.Prune(tx, run.PipelineSpecID) + defer o.Prune(tx, run.PruningKey) // Simply finish the run, no need to do any sort of locking if run.Outputs.Val == nil || len(run.FatalErrors)+len(run.AllErrors) == 0 { return errors.Errorf("run must have both Outputs and Errors, got Outputs: %#v, FatalErrors: %#v, AllErrors: %#v", run.Outputs.Val, run.FatalErrors, run.AllErrors) @@ -299,14 +300,15 @@ func (o *orm) UpdateTaskRunResult(taskID uuid.UUID, result Result) (run Run, sta } err = o.q.Transaction(func(tx pg.Queryer) error { sql := ` - SELECT pipeline_runs.*, pipeline_specs.dot_dag_source "pipeline_spec.dot_dag_source" + SELECT pipeline_runs.*, pipeline_specs.dot_dag_source "pipeline_spec.dot_dag_source", job_pipeline_specs.job_id "job_id" FROM pipeline_runs JOIN pipeline_task_runs ON (pipeline_task_runs.pipeline_run_id = pipeline_runs.id) JOIN pipeline_specs ON (pipeline_specs.id = pipeline_runs.pipeline_spec_id) + JOIN job_pipeline_specs ON (job_pipeline_specs.pipeline_spec_id = pipeline_specs.id) WHERE pipeline_task_runs.id = $1 AND pipeline_runs.state in ('running', 'suspended') FOR UPDATE` if err = tx.Get(&run, sql, taskID); err != nil { - return fmt.Errorf("failed to find pipeline run for ID %s: %w", taskID.String(), err) + return fmt.Errorf("failed to find pipeline run for task ID %s: %w", taskID.String(), err) } // Update the task with result @@ -337,9 +339,9 @@ func (o *orm) InsertFinishedRuns(runs []*Run, saveSuccessfulTaskRuns bool, qopts err := q.Transaction(func(tx pg.Queryer) error { pipelineRunsQuery := ` INSERT INTO pipeline_runs - (pipeline_spec_id, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) + (pipeline_spec_id, pruning_key, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) VALUES - (:pipeline_spec_id, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) + (:pipeline_spec_id, :pruning_key, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) RETURNING id ` rows, errQ := tx.NamedQuery(pipelineRunsQuery, runs) @@ -357,17 +359,17 @@ RETURNING id runIDs = append(runIDs, runID) } - pipelineSpecIDm := make(map[int32]struct{}) + pruningKeysm := make(map[int32]struct{}) for i, run := range runs { - pipelineSpecIDm[run.PipelineSpecID] = struct{}{} + pruningKeysm[run.PruningKey] = struct{}{} for j := range run.PipelineTaskRuns { run.PipelineTaskRuns[j].PipelineRunID = runIDs[i] } } defer func() { - for pipelineSpecID := range pipelineSpecIDm { - o.Prune(tx, pipelineSpecID) + for pruningKey := range pruningKeysm { + o.Prune(tx, pruningKey) } }() @@ -419,10 +421,46 @@ func (o *orm) InsertFinishedRun(run *Run, saveSuccessfulTaskRuns bool, qopts ... return nil } + q := o.q.WithOpts(qopts...) + err = q.Transaction(o.insertFinishedRunTx(run, saveSuccessfulTaskRuns)) + return errors.Wrap(err, "InsertFinishedRun failed") +} + +// InsertFinishedRunWithSpec works like InsertFinishedRun but also inserts the pipeline spec. +func (o *orm) InsertFinishedRunWithSpec(run *Run, saveSuccessfulTaskRuns bool, qopts ...pg.QOpt) (err error) { + if err = o.checkFinishedRun(run, saveSuccessfulTaskRuns); err != nil { + return err + } + + if o.maxSuccessfulRuns == 0 { + // optimisation: avoid persisting if we oughtn't to save any + return nil + } + q := o.q.WithOpts(qopts...) err = q.Transaction(func(tx pg.Queryer) error { - sql := `INSERT INTO pipeline_runs (pipeline_spec_id, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) - VALUES (:pipeline_spec_id, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) + sqlStmt1 := `INSERT INTO pipeline_specs (dot_dag_source, max_task_duration, created_at) + VALUES ($1, $2, NOW()) + RETURNING id;` + err = tx.Get(&run.PipelineSpecID, sqlStmt1, run.PipelineSpec.DotDagSource, run.PipelineSpec.MaxTaskDuration) + if err != nil { + return errors.Wrap(err, "failed to insert pipeline_specs") + } + // This `job_pipeline_specs` record won't be primary since when this method is called, the job already exists, so it will have primary record. + sqlStmt2 := `INSERT INTO job_pipeline_specs (job_id, pipeline_spec_id, is_primary) VALUES ($1, $2, false)` + _, err = tx.Exec(sqlStmt2, run.JobID, run.PipelineSpecID) + if err != nil { + return errors.Wrap(err, "failed to insert job_pipeline_specs") + } + return o.insertFinishedRunTx(run, saveSuccessfulTaskRuns)(tx) + }) + return errors.Wrap(err, "InsertFinishedRun failed") +} + +func (o *orm) insertFinishedRunTx(run *Run, saveSuccessfulTaskRuns bool) func(tx pg.Queryer) error { + return func(tx pg.Queryer) error { + sql := `INSERT INTO pipeline_runs (pipeline_spec_id, pruning_key, meta, all_errors, fatal_errors, inputs, outputs, created_at, finished_at, state) + VALUES (:pipeline_spec_id, :pruning_key, :meta, :all_errors, :fatal_errors, :inputs, :outputs, :created_at, :finished_at, :state) RETURNING id;` query, args, e := tx.BindNamed(sql, run) @@ -430,7 +468,7 @@ func (o *orm) InsertFinishedRun(run *Run, saveSuccessfulTaskRuns bool, qopts ... return errors.Wrap(e, "failed to bind") } - if err = tx.QueryRowx(query, args...).Scan(&run.ID); err != nil { + if err := tx.QueryRowx(query, args...).Scan(&run.ID); err != nil { return errors.Wrap(err, "error inserting finished pipeline_run") } @@ -443,14 +481,13 @@ func (o *orm) InsertFinishedRun(run *Run, saveSuccessfulTaskRuns bool, qopts ... return nil } - defer o.Prune(tx, run.PipelineSpecID) + defer o.Prune(tx, run.PruningKey) sql = ` INSERT INTO pipeline_task_runs (pipeline_run_id, id, type, index, output, error, dot_id, created_at, finished_at) VALUES (:pipeline_run_id, :id, :type, :index, :output, :error, :dot_id, :created_at, :finished_at);` - _, err = tx.NamedExec(sql, run.PipelineTaskRuns) + _, err := tx.NamedExec(sql, run.PipelineTaskRuns) return errors.Wrap(err, "failed to insert pipeline_task_runs") - }) - return errors.Wrap(err, "InsertFinishedRun failed") + } } // DeleteRunsOlderThan deletes all pipeline_runs that have been finished for a certain threshold to free DB space @@ -586,7 +623,19 @@ func loadAssociations(q pg.Queryer, runs []*Run) error { pipelineSpecIDM[run.PipelineSpecID] = Spec{} } } - if err := q.Select(&specs, `SELECT ps.id, ps.dot_dag_source, ps.created_at, ps.max_task_duration, coalesce(jobs.id, 0) "job_id", coalesce(jobs.name, '') "job_name", coalesce(jobs.type, '') "job_type" FROM pipeline_specs ps LEFT OUTER JOIN jobs ON jobs.pipeline_spec_id=ps.id WHERE ps.id = ANY($1)`, pipelineSpecIDs); err != nil { + sqlQuery := `SELECT + ps.id, + ps.dot_dag_source, + ps.created_at, + ps.max_task_duration, + coalesce(jobs.id, 0) "job_id", + coalesce(jobs.name, '') "job_name", + coalesce(jobs.type, '') "job_type" + FROM pipeline_specs ps + LEFT JOIN job_pipeline_specs jps ON jps.pipeline_spec_id=ps.id + LEFT JOIN jobs ON jobs.id=jps.job_id + WHERE ps.id = ANY($1)` + if err := q.Select(&specs, sqlQuery, pipelineSpecIDs); err != nil { return errors.Wrap(err, "failed to postload pipeline_specs for runs") } for _, spec := range specs { @@ -617,14 +666,14 @@ func (o *orm) GetQ() pg.Q { return o.q } -func (o *orm) loadCount(pipelineSpecID int32) *atomic.Uint64 { +func (o *orm) loadCount(jobID int32) *atomic.Uint64 { // fast path; avoids allocation - actual, exists := o.pm.Load(pipelineSpecID) + actual, exists := o.pm.Load(jobID) if exists { return actual.(*atomic.Uint64) } // "slow" path - actual, _ = o.pm.LoadOrStore(pipelineSpecID, new(atomic.Uint64)) + actual, _ = o.pm.LoadOrStore(jobID, new(atomic.Uint64)) return actual.(*atomic.Uint64) } @@ -633,74 +682,74 @@ func (o *orm) loadCount(pipelineSpecID int32) *atomic.Uint64 { const syncLimit = 1000 // Prune attempts to keep the pipeline_runs table capped close to the -// maxSuccessfulRuns length for each pipeline_spec_id. +// maxSuccessfulRuns length for each job_id. // // It does this synchronously for small values and async/sampled for large // values. // // Note this does not guarantee the pipeline_runs table is kept to exactly the // max length, rather that it doesn't excessively larger than it. -func (o *orm) Prune(tx pg.Queryer, pipelineSpecID int32) { - if pipelineSpecID == 0 { - o.lggr.Panic("expected a non-zero pipeline spec ID") +func (o *orm) Prune(tx pg.Queryer, jobID int32) { + if jobID == 0 { + o.lggr.Panic("expected a non-zero job ID") } // For small maxSuccessfulRuns its fast enough to prune every time if o.maxSuccessfulRuns < syncLimit { - o.execPrune(tx, pipelineSpecID) + o.execPrune(tx, jobID) return } // for large maxSuccessfulRuns we do it async on a sampled basis every := o.maxSuccessfulRuns / 20 // it can get up to 5% larger than maxSuccessfulRuns before a prune - cnt := o.loadCount(pipelineSpecID) + cnt := o.loadCount(jobID) val := cnt.Add(1) if val%every == 0 { ok := o.IfStarted(func() { o.wg.Add(1) go func() { - o.lggr.Debugw("Pruning runs", "pipelineSpecID", pipelineSpecID, "count", val, "every", every, "maxSuccessfulRuns", o.maxSuccessfulRuns) + o.lggr.Debugw("Pruning runs", "jobID", jobID, "count", val, "every", every, "maxSuccessfulRuns", o.maxSuccessfulRuns) defer o.wg.Done() // Must not use tx here since it's async and the transaction // could be stale - o.execPrune(o.q.WithOpts(pg.WithLongQueryTimeout()), pipelineSpecID) + o.execPrune(o.q.WithOpts(pg.WithLongQueryTimeout()), jobID) }() }) if !ok { - o.lggr.Warnw("Cannot prune: ORM is not running", "pipelineSpecID", pipelineSpecID) + o.lggr.Warnw("Cannot prune: ORM is not running", "jobID", jobID) return } } } -func (o *orm) execPrune(q pg.Queryer, pipelineSpecID int32) { - res, err := q.ExecContext(o.ctx, `DELETE FROM pipeline_runs WHERE pipeline_spec_id = $1 AND state = $2 AND id NOT IN ( +func (o *orm) execPrune(q pg.Queryer, jobID int32) { + res, err := q.ExecContext(o.ctx, `DELETE FROM pipeline_runs WHERE pruning_key = $1 AND state = $2 AND id NOT IN ( SELECT id FROM pipeline_runs -WHERE pipeline_spec_id = $1 AND state = $2 +WHERE pruning_key = $1 AND state = $2 ORDER BY id DESC LIMIT $3 -)`, pipelineSpecID, RunStatusCompleted, o.maxSuccessfulRuns) +)`, jobID, RunStatusCompleted, o.maxSuccessfulRuns) if err != nil { - o.lggr.Errorw("Failed to prune runs", "err", err, "pipelineSpecID", pipelineSpecID) + o.lggr.Errorw("Failed to prune runs", "err", err, "jobID", jobID) return } rowsAffected, err := res.RowsAffected() if err != nil { - o.lggr.Errorw("Failed to get RowsAffected while pruning runs", "err", err, "pipelineSpecID", pipelineSpecID) + o.lggr.Errorw("Failed to get RowsAffected while pruning runs", "err", err, "jobID", jobID) return } if rowsAffected == 0 { // check the spec still exists and garbage collect if necessary var exists bool - if err := q.GetContext(o.ctx, &exists, `SELECT EXISTS(SELECT * FROM pipeline_specs WHERE id = $1)`, pipelineSpecID); err != nil { - o.lggr.Errorw("Failed check existence of pipeline_spec while pruning runs", "err", err, "pipelineSpecID", pipelineSpecID) + if err := q.GetContext(o.ctx, &exists, `SELECT EXISTS(SELECT ps.* FROM pipeline_specs ps JOIN job_pipeline_specs jps ON (ps.id=jps.pipeline_spec_id) WHERE jps.job_id = $1)`, jobID); err != nil { + o.lggr.Errorw("Failed check existence of pipeline_spec while pruning runs", "err", err, "jobID", jobID) return } if !exists { - o.lggr.Debugw("Pipeline spec no longer exists, removing prune count", "pipelineSpecID", pipelineSpecID) - o.pm.Delete(pipelineSpecID) + o.lggr.Debugw("Pipeline spec no longer exists, removing prune count", "jobID", jobID) + o.pm.Delete(jobID) } } else if o.maxSuccessfulRuns < syncLimit { - o.lggr.Tracew("Pruned runs", "rowsAffected", rowsAffected, "pipelineSpecID", pipelineSpecID) + o.lggr.Tracew("Pruned runs", "rowsAffected", rowsAffected, "jobID", jobID) } else { - o.lggr.Debugw("Pruned runs", "rowsAffected", rowsAffected, "pipelineSpecID", pipelineSpecID) + o.lggr.Debugw("Pruned runs", "rowsAffected", rowsAffected, "jobID", jobID) } } diff --git a/core/services/pipeline/orm_test.go b/core/services/pipeline/orm_test.go index 6a6efa0dc3a..c59fc0c32c6 100644 --- a/core/services/pipeline/orm_test.go +++ b/core/services/pipeline/orm_test.go @@ -14,6 +14,7 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/utils/hex" "github.com/smartcontractkit/chainlink-common/pkg/utils/jsonserializable" + "github.com/smartcontractkit/chainlink/v2/core/bridges" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils/big" "github.com/smartcontractkit/chainlink/v2/core/internal/cltest" @@ -35,7 +36,33 @@ type ormconfig struct { func (ormconfig) JobPipelineMaxSuccessfulRuns() uint64 { return 123456 } -func setupORM(t *testing.T, heavy bool) (db *sqlx.DB, orm pipeline.ORM) { +type testOnlyORM interface { + pipeline.ORM + AddJobPipelineSpecWithoutConstraints(jobID, pipelineSpecID int32) error +} + +type testORM struct { + pipeline.ORM + db *sqlx.DB +} + +func (torm *testORM) AddJobPipelineSpecWithoutConstraints(jobID, pipelineSpecID int32) error { + _, err := torm.db.Exec(`SET CONSTRAINTS fk_job_pipeline_spec_job DEFERRED`) + if err != nil { + return err + } + _, err = torm.db.Exec(`INSERT INTO job_pipeline_specs (job_id,pipeline_spec_id, is_primary) VALUES ($1, $2, false)`, jobID, pipelineSpecID) + if err != nil { + return err + } + return nil +} + +func newTestORM(orm pipeline.ORM, db *sqlx.DB) testOnlyORM { + return &testORM{ORM: orm, db: db} +} + +func setupORM(t *testing.T, heavy bool) (db *sqlx.DB, orm pipeline.ORM, jorm job.ORM) { t.Helper() if heavy { @@ -45,20 +72,26 @@ func setupORM(t *testing.T, heavy bool) (db *sqlx.DB, orm pipeline.ORM) { } cfg := ormconfig{pgtest.NewQConfig(true)} orm = pipeline.NewORM(db, logger.TestLogger(t), cfg, cfg.JobPipelineMaxSuccessfulRuns()) + config := configtest.NewTestGeneralConfig(t) + lggr := logger.TestLogger(t) + keyStore := cltest.NewKeyStore(t, db, config.Database()) + bridgeORM := bridges.NewORM(db, lggr, config.Database()) + + jorm = job.NewORM(db, orm, bridgeORM, keyStore, lggr, config.Database()) return } -func setupHeavyORM(t *testing.T) (db *sqlx.DB, orm pipeline.ORM) { +func setupHeavyORM(t *testing.T) (db *sqlx.DB, orm pipeline.ORM, jorm job.ORM) { return setupORM(t, true) } -func setupLiteORM(t *testing.T) (db *sqlx.DB, orm pipeline.ORM) { +func setupLiteORM(t *testing.T) (db *sqlx.DB, orm pipeline.ORM, jorm job.ORM) { return setupORM(t, false) } func Test_PipelineORM_CreateSpec(t *testing.T) { - db, orm := setupLiteORM(t) + db, orm, _ := setupLiteORM(t) var ( source = "" @@ -80,9 +113,11 @@ func Test_PipelineORM_CreateSpec(t *testing.T) { } func Test_PipelineORM_FindRun(t *testing.T) { - db, orm := setupLiteORM(t) + db, orm, _ := setupLiteORM(t) - _, err := db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) + _, err := db.Exec(`SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) + require.NoError(t, err) + _, err = db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) require.NoError(t, err) expected := mustInsertPipelineRun(t, orm) @@ -107,7 +142,7 @@ func mustInsertPipelineRun(t *testing.T, orm pipeline.ORM) pipeline.Run { return run } -func mustInsertAsyncRun(t *testing.T, orm pipeline.ORM) *pipeline.Run { +func mustInsertAsyncRun(t *testing.T, orm pipeline.ORM, jobORM job.ORM) *pipeline.Run { t.Helper() s := ` @@ -120,17 +155,24 @@ ds1->ds1_parse->ds1_multiply->answer1; answer1 [type=median index=0]; answer2 [type=bridge name=election_winner index=1]; ` - - p, err := pipeline.Parse(s) - require.NoError(t, err) - require.NotNil(t, p) - - maxTaskDuration := models.Interval(1 * time.Minute) - specID, err := orm.CreateSpec(*p, maxTaskDuration) + jb := job.Job{ + Type: job.DirectRequest, + SchemaVersion: 1, + MaxTaskDuration: models.Interval(1 * time.Minute), + DirectRequestSpec: &job.DirectRequestSpec{ + ContractAddress: cltest.NewEIP55Address(), + EVMChainID: (*big.Big)(&cltest.FixtureChainID), + }, + PipelineSpec: &pipeline.Spec{ + DotDagSource: s, + }, + } + err := jobORM.CreateJob(&jb) require.NoError(t, err) run := &pipeline.Run{ - PipelineSpecID: specID, + PipelineSpecID: jb.PipelineSpecID, + PruningKey: jb.ID, State: pipeline.RunStatusRunning, Outputs: jsonserializable.JSONSerializable{}, CreatedAt: time.Now(), @@ -142,9 +184,11 @@ answer2 [type=bridge name=election_winner index=1]; } func TestInsertFinishedRuns(t *testing.T) { - db, orm := setupLiteORM(t) + db, orm, _ := setupLiteORM(t) - _, err := db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) + _, err := db.Exec(`SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) + require.NoError(t, err) + _, err = db.Exec(`SET CONSTRAINTS pipeline_runs_pipeline_spec_id_fkey DEFERRED`) require.NoError(t, err) ps := cltest.MustInsertPipelineSpec(t, db) @@ -154,6 +198,7 @@ func TestInsertFinishedRuns(t *testing.T) { now := time.Now() r := pipeline.Run{ PipelineSpecID: ps.ID, + PruningKey: ps.ID, // using the spec ID as the pruning key for test purposes, this is supposed to be the job ID State: pipeline.RunStatusRunning, AllErrors: pipeline.RunErrors{}, FatalErrors: pipeline.RunErrors{}, @@ -198,11 +243,96 @@ func TestInsertFinishedRuns(t *testing.T) { } +func Test_PipelineORM_InsertFinishedRunWithSpec(t *testing.T) { + db, orm, jorm := setupLiteORM(t) + + s := ` +ds1 [type=bridge async=true name="example-bridge" timeout=0 requestData=<{"data": {"coin": "BTC", "market": "USD"}}>] +ds1_parse [type=jsonparse lax=false path="data,result"] +ds1_multiply [type=multiply times=1000000000000000000] + +ds1->ds1_parse->ds1_multiply->answer1; + +answer1 [type=median index=0]; +answer2 [type=bridge name=election_winner index=1]; +` + jb := job.Job{ + Type: job.DirectRequest, + SchemaVersion: 1, + MaxTaskDuration: models.Interval(1 * time.Minute), + DirectRequestSpec: &job.DirectRequestSpec{ + ContractAddress: cltest.NewEIP55Address(), + EVMChainID: (*big.Big)(&cltest.FixtureChainID), + }, + PipelineSpec: &pipeline.Spec{ + DotDagSource: s, + }, + } + err := jorm.CreateJob(&jb) + require.NoError(t, err) + spec := pipeline.Spec{ + DotDagSource: s, + CreatedAt: time.Now(), + MaxTaskDuration: models.Interval(1 * time.Minute), + JobID: jb.ID, + JobName: jb.Name.ValueOrZero(), + JobType: string(jb.Type), + } + defaultVars := map[string]interface{}{ + "jb": map[string]interface{}{ + "databaseID": jb.ID, + "externalJobID": jb.ExternalJobID, + "name": jb.Name.ValueOrZero(), + }, + } + now := time.Now() + run := pipeline.NewRun(spec, pipeline.NewVarsFrom(defaultVars)) + run.PipelineTaskRuns = []pipeline.TaskRun{ + { + ID: uuid.New(), + PipelineRunID: run.ID, + Type: "bridge", + DotID: "ds1", + CreatedAt: now, + FinishedAt: null.TimeFrom(now.Add(100 * time.Millisecond)), + }, + { + ID: uuid.New(), + PipelineRunID: run.ID, + Type: "median", + DotID: "answer2", + Output: jsonserializable.JSONSerializable{Val: 1, Valid: true}, + CreatedAt: now, + FinishedAt: null.TimeFrom(now.Add(200 * time.Millisecond)), + }, + } + run.FinishedAt = null.TimeFrom(now.Add(300 * time.Millisecond)) + run.Outputs = jsonserializable.JSONSerializable{ + Val: "stuff", + Valid: true, + } + run.AllErrors = append(run.AllErrors, null.NewString("", false)) + run.State = pipeline.RunStatusCompleted + + err = orm.InsertFinishedRunWithSpec(run, true) + require.NoError(t, err) + + var pipelineSpec pipeline.Spec + err = db.Get(&pipelineSpec, "SELECT pipeline_specs.* FROM pipeline_specs JOIN job_pipeline_specs ON (pipeline_specs.id = job_pipeline_specs.pipeline_spec_id) WHERE job_pipeline_specs.job_id = $1 AND pipeline_specs.id = $2", jb.ID, run.PipelineSpecID) + require.NoError(t, err) + var jobPipelineSpec job.PipelineSpec + err = db.Get(&jobPipelineSpec, "SELECT * FROM job_pipeline_specs WHERE job_id = $1 AND pipeline_spec_id = $2", jb.ID, pipelineSpec.ID) + require.NoError(t, err) + + assert.Equal(t, run.PipelineSpecID, pipelineSpec.ID) + assert.False(t, jobPipelineSpec.IsPrimary) +} + // Tests that inserting run results, then later updating the run results via upsert will work correctly. func Test_PipelineORM_StoreRun_ShouldUpsert(t *testing.T) { - _, orm := setupLiteORM(t) + _, orm, jorm := setupLiteORM(t) - run := mustInsertAsyncRun(t, orm) + run := mustInsertAsyncRun(t, orm, jorm) now := time.Now() @@ -279,9 +409,9 @@ func Test_PipelineORM_StoreRun_ShouldUpsert(t *testing.T) { // Tests that trying to persist a partial run while new data became available (i.e. via /v2/restart) // will detect a restart and update the result data on the Run. func Test_PipelineORM_StoreRun_DetectsRestarts(t *testing.T) { - db, orm := setupLiteORM(t) + db, orm, jorm := setupLiteORM(t) - run := mustInsertAsyncRun(t, orm) + run := mustInsertAsyncRun(t, orm, jorm) r, err := orm.FindRun(run.ID) require.NoError(t, err) @@ -344,9 +474,9 @@ func Test_PipelineORM_StoreRun_DetectsRestarts(t *testing.T) { } func Test_PipelineORM_StoreRun_UpdateTaskRunResult(t *testing.T) { - _, orm := setupLiteORM(t) + _, orm, jorm := setupLiteORM(t) - run := mustInsertAsyncRun(t, orm) + run := mustInsertAsyncRun(t, orm, jorm) ds1_id := uuid.New() now := time.Now() @@ -425,9 +555,9 @@ func Test_PipelineORM_StoreRun_UpdateTaskRunResult(t *testing.T) { } func Test_PipelineORM_DeleteRun(t *testing.T) { - _, orm := setupLiteORM(t) + _, orm, jorm := setupLiteORM(t) - run := mustInsertAsyncRun(t, orm) + run := mustInsertAsyncRun(t, orm, jorm) now := time.Now() @@ -467,12 +597,12 @@ func Test_PipelineORM_DeleteRun(t *testing.T) { } func Test_PipelineORM_DeleteRunsOlderThan(t *testing.T) { - _, orm := setupHeavyORM(t) + _, orm, jorm := setupHeavyORM(t) var runsIds []int64 for i := 1; i <= 2000; i++ { - run := mustInsertAsyncRun(t, orm) + run := mustInsertAsyncRun(t, orm, jorm) now := time.Now() @@ -556,6 +686,7 @@ func Test_GetUnfinishedRuns_Keepers(t *testing.T) { err = porm.CreateRun(&pipeline.Run{ PipelineSpecID: keeperJob.PipelineSpecID, + PruningKey: keeperJob.ID, State: pipeline.RunStatusRunning, Outputs: jsonserializable.JSONSerializable{}, CreatedAt: time.Now(), @@ -572,6 +703,7 @@ func Test_GetUnfinishedRuns_Keepers(t *testing.T) { err = porm.CreateRun(&pipeline.Run{ PipelineSpecID: keeperJob.PipelineSpecID, + PruningKey: keeperJob.ID, State: pipeline.RunStatusRunning, Outputs: jsonserializable.JSONSerializable{}, CreatedAt: time.Now(), @@ -654,6 +786,7 @@ func Test_GetUnfinishedRuns_DirectRequest(t *testing.T) { err = porm.CreateRun(&pipeline.Run{ PipelineSpecID: drJob.PipelineSpecID, + PruningKey: drJob.ID, State: pipeline.RunStatusRunning, Outputs: jsonserializable.JSONSerializable{}, CreatedAt: time.Now(), @@ -670,6 +803,7 @@ func Test_GetUnfinishedRuns_DirectRequest(t *testing.T) { err = porm.CreateRun(&pipeline.Run{ PipelineSpecID: drJob.PipelineSpecID, + PruningKey: drJob.ID, State: pipeline.RunStatusSuspended, Outputs: jsonserializable.JSONSerializable{}, CreatedAt: time.Now(), @@ -713,43 +847,54 @@ func Test_Prune(t *testing.T) { lggr, observed := logger.TestLoggerObserved(t, zapcore.DebugLevel) db := pgtest.NewSqlxDB(t) porm := pipeline.NewORM(db, lggr, cfg.Database(), cfg.JobPipeline().MaxSuccessfulRuns()) + torm := newTestORM(porm, db) ps1 := cltest.MustInsertPipelineSpec(t, db) + // We need a job_pipeline_specs entry to test the pruning mechanism + err := torm.AddJobPipelineSpecWithoutConstraints(ps1.ID, ps1.ID) + require.NoError(t, err) + + jobID := ps1.ID + t.Run("when there are no runs to prune, does nothing", func(t *testing.T) { - porm.Prune(db, ps1.ID) + porm.Prune(db, jobID) // no error logs; it did nothing assert.Empty(t, observed.All()) }) + _, err = db.Exec(`SET CONSTRAINTS fk_pipeline_runs_pruning_key DEFERRED`) + require.NoError(t, err) + // ps1 has: // - 20 completed runs for i := 0; i < 20; i++ { - cltest.MustInsertPipelineRunWithStatus(t, db, ps1.ID, pipeline.RunStatusCompleted) + cltest.MustInsertPipelineRunWithStatus(t, db, ps1.ID, pipeline.RunStatusCompleted, jobID) } ps2 := cltest.MustInsertPipelineSpec(t, db) + jobID2 := ps2.ID // ps2 has: // - 12 completed runs // - 3 errored runs - // - 3 running run + // - 3 running runs // - 3 suspended run for i := 0; i < 12; i++ { - cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusCompleted) + cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusCompleted, jobID2) } for i := 0; i < 3; i++ { - cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusErrored) + cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusErrored, jobID2) } for i := 0; i < 3; i++ { - cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusRunning) + cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusRunning, jobID2) } for i := 0; i < 3; i++ { - cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusSuspended) + cltest.MustInsertPipelineRunWithStatus(t, db, ps2.ID, pipeline.RunStatusSuspended, jobID2) } - porm.Prune(db, ps2.ID) + porm.Prune(db, jobID2) cnt := pgtest.MustCount(t, db, "SELECT count(*) FROM pipeline_runs WHERE pipeline_spec_id = $1 AND state = $2", ps1.ID, pipeline.RunStatusCompleted) assert.Equal(t, cnt, 20) diff --git a/core/services/pipeline/runner.go b/core/services/pipeline/runner.go index 3b89a1d4945..08d371716fc 100644 --- a/core/services/pipeline/runner.go +++ b/core/services/pipeline/runner.go @@ -17,10 +17,10 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/services" commonutils "github.com/smartcontractkit/chainlink-common/pkg/utils" "github.com/smartcontractkit/chainlink-common/pkg/utils/jsonserializable" - "github.com/smartcontractkit/chainlink/v2/core/config/env" "github.com/smartcontractkit/chainlink/v2/core/bridges" "github.com/smartcontractkit/chainlink/v2/core/chains/legacyevm" + "github.com/smartcontractkit/chainlink/v2/core/config/env" "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/recovery" "github.com/smartcontractkit/chainlink/v2/core/services/pg" @@ -49,7 +49,8 @@ type Runner interface { // ExecuteAndInsertFinishedRun executes a new run in-memory according to a spec, persists and saves the results. // It is a combination of ExecuteRun and InsertFinishedRun. // Note that the spec MUST have a DOT graph for this to work. - ExecuteAndInsertFinishedRun(ctx context.Context, spec Spec, vars Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, finalResult FinalResult, err error) + // This will persist the Spec in the DB if it doesn't have an ID. + ExecuteAndInsertFinishedRun(ctx context.Context, spec Spec, vars Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, results TaskRunResults, err error) OnRunFinished(func(*Run)) InitializePipeline(spec Spec) (*Pipeline, error) @@ -203,6 +204,8 @@ func (err ErrRunPanicked) Error() string { func NewRun(spec Spec, vars Vars) *Run { return &Run{ State: RunStatusRunning, + JobID: spec.JobID, + PruningKey: spec.JobID, PipelineSpec: spec, PipelineSpecID: spec.ID, Inputs: jsonserializable.JSONSerializable{Val: vars.vars, Valid: true}, @@ -551,23 +554,26 @@ func logTaskRunToPrometheus(trr TaskRunResult, spec Spec) { } // ExecuteAndInsertFinishedRun executes a run in memory then inserts the finished run/task run records, returning the final result -func (r *runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec Spec, vars Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, finalResult FinalResult, err error) { +func (r *runner) ExecuteAndInsertFinishedRun(ctx context.Context, spec Spec, vars Vars, l logger.Logger, saveSuccessfulTaskRuns bool) (runID int64, results TaskRunResults, err error) { run, trrs, err := r.ExecuteRun(ctx, spec, vars, l) if err != nil { - return 0, finalResult, pkgerrors.Wrapf(err, "error executing run for spec ID %v", spec.ID) + return 0, trrs, pkgerrors.Wrapf(err, "error executing run for spec ID %v", spec.ID) } - finalResult = trrs.FinalResult(l) - // don't insert if we exited early if run.FailSilently { - return 0, finalResult, nil + return 0, trrs, nil } - if err = r.orm.InsertFinishedRun(run, saveSuccessfulTaskRuns); err != nil { - return 0, finalResult, pkgerrors.Wrapf(err, "error inserting finished results for spec ID %v", spec.ID) + if spec.ID == 0 { + err = r.orm.InsertFinishedRunWithSpec(run, saveSuccessfulTaskRuns) + } else { + err = r.orm.InsertFinishedRun(run, saveSuccessfulTaskRuns) + } + if err != nil { + return 0, trrs, pkgerrors.Wrapf(err, "error inserting finished results for spec ID %v", run.PipelineSpecID) } - return run.ID, finalResult, nil + return run.ID, trrs, nil } diff --git a/core/services/pipeline/runner_test.go b/core/services/pipeline/runner_test.go index 7a417ef9d94..cdf63aa975f 100644 --- a/core/services/pipeline/runner_test.go +++ b/core/services/pipeline/runner_test.go @@ -486,7 +486,49 @@ func Test_PipelineRunner_HandleFaultsPersistRun(t *testing.T) { lggr := logger.TestLogger(t) r := pipeline.NewRunner(orm, btORM, cfg.JobPipeline(), cfg.WebServer(), legacyChains, ethKeyStore, nil, lggr, nil, nil) - spec := pipeline.Spec{DotDagSource: ` + spec := pipeline.Spec{ + ID: 1, + DotDagSource: ` +fail_but_i_dont_care [type=fail] +succeed1 [type=memo value=10] +succeed2 [type=memo value=11] +final [type=mean] + +fail_but_i_dont_care -> final; +succeed1 -> final; +succeed2 -> final; +`} + vars := pipeline.NewVarsFrom(nil) + + _, taskResults, err := r.ExecuteAndInsertFinishedRun(testutils.Context(t), spec, vars, lggr, false) + finalResult := taskResults.FinalResult(lggr) + require.NoError(t, err) + assert.True(t, finalResult.HasErrors()) + assert.False(t, finalResult.HasFatalErrors()) + require.Len(t, finalResult.Values, 1) + assert.Equal(t, "10.5", finalResult.Values[0].(decimal.Decimal).String()) +} + +func Test_PipelineRunner_ExecuteAndInsertFinishedRun_SavingTheSpec(t *testing.T) { + db := pgtest.NewSqlxDB(t) + orm := mocks.NewORM(t) + btORM := bridgesMocks.NewORM(t) + q := pg.NewQ(db, logger.TestLogger(t), configtest.NewTestGeneralConfig(t).Database()) + orm.On("GetQ").Return(q).Maybe() + orm.On("InsertFinishedRunWithSpec", mock.Anything, mock.Anything, mock.Anything). + Run(func(args mock.Arguments) { + args.Get(0).(*pipeline.Run).ID = 1 + }). + Return(nil) + cfg := configtest.NewTestGeneralConfig(t) + ethKeyStore := cltest.NewKeyStore(t, db, cfg.Database()).Eth() + relayExtenders := evmtest.NewChainRelayExtenders(t, evmtest.TestChainOpts{DB: db, GeneralConfig: cfg, KeyStore: ethKeyStore}) + legacyChains := evmrelay.NewLegacyChainsFromRelayerExtenders(relayExtenders) + lggr := logger.TestLogger(t) + r := pipeline.NewRunner(orm, btORM, cfg.JobPipeline(), cfg.WebServer(), legacyChains, ethKeyStore, nil, lggr, nil, nil) + + spec := pipeline.Spec{ + DotDagSource: ` fail_but_i_dont_care [type=fail] succeed1 [type=memo value=10] succeed2 [type=memo value=11] @@ -498,7 +540,8 @@ succeed2 -> final; `} vars := pipeline.NewVarsFrom(nil) - _, finalResult, err := r.ExecuteAndInsertFinishedRun(testutils.Context(t), spec, vars, lggr, false) + _, taskResults, err := r.ExecuteAndInsertFinishedRun(testutils.Context(t), spec, vars, lggr, false) + finalResult := taskResults.FinalResult(lggr) require.NoError(t, err) assert.True(t, finalResult.HasErrors()) assert.False(t, finalResult.HasFatalErrors()) diff --git a/core/store/migrate/migrations/0231_dynamic_pipeline_runs.sql b/core/store/migrate/migrations/0231_dynamic_pipeline_runs.sql new file mode 100644 index 00000000000..2e51af8f922 --- /dev/null +++ b/core/store/migrate/migrations/0231_dynamic_pipeline_runs.sql @@ -0,0 +1,48 @@ +-- +goose Up +-- +goose StatementBegin +CREATE TABLE job_pipeline_specs ( + job_id INT NOT NULL, + pipeline_spec_id INT NOT NULL, + is_primary BOOLEAN NOT NULL DEFAULT FALSE, + CONSTRAINT pk_job_pipeline_spec PRIMARY KEY (job_id, pipeline_spec_id), + CONSTRAINT fk_job_pipeline_spec_job FOREIGN KEY (job_id) REFERENCES jobs(id) ON DELETE CASCADE DEFERRABLE, + CONSTRAINT fk_job_pipeline_spec_pipeline_spec FOREIGN KEY (pipeline_spec_id) REFERENCES pipeline_specs(id) ON DELETE CASCADE DEFERRABLE +); + +CREATE UNIQUE INDEX idx_unique_job_pipeline_spec_primary_per_job ON job_pipeline_specs(job_id) WHERE is_primary; + +-- The moment this runs, we only have one job+pipeline_spec combination per job, complying with the unique index. +INSERT INTO job_pipeline_specs (job_id, pipeline_spec_id, is_primary) +SELECT id, pipeline_spec_id, TRUE +FROM jobs; + +ALTER TABLE jobs DROP COLUMN pipeline_spec_id; + +ALTER TABLE pipeline_runs ADD COLUMN pruning_key INT; + +UPDATE pipeline_runs +SET pruning_key = pjps.job_id +FROM job_pipeline_specs pjps +WHERE pjps.pipeline_spec_id = pipeline_runs.pipeline_spec_id; + +ALTER TABLE pipeline_runs ALTER COLUMN pruning_key SET NOT NULL; + +ALTER TABLE pipeline_runs ADD CONSTRAINT fk_pipeline_runs_pruning_key FOREIGN KEY (pruning_key) REFERENCES jobs(id) ON DELETE CASCADE DEFERRABLE; +-- +goose StatementEnd + +-- +goose Down +-- +goose StatementBegin +ALTER TABLE jobs ADD COLUMN pipeline_spec_id INT; + +UPDATE jobs +SET pipeline_spec_id = jps.pipeline_spec_id +FROM job_pipeline_specs jps +WHERE jps.job_id = jobs.id + AND jps.is_primary = TRUE; + +ALTER TABLE pipeline_runs DROP COLUMN pruning_key; + +DROP INDEX IF EXISTS idx_unique_primary_per_job; + +DROP TABLE IF EXISTS job_pipeline_specs; +-- +goose StatementEnd \ No newline at end of file From 1fd2c91c7e145bce9ab32a1c1003ece3e42eaa6a Mon Sep 17 00:00:00 2001 From: frank zhu Date: Tue, 9 Apr 2024 09:31:36 -0700 Subject: [PATCH 2/5] update changelog path (#12757) --- .gitignore | 3 --- docs/CHANGELOG.md => CHANGELOG.md | 4 ---- README.md | 4 ++++ 3 files changed, 4 insertions(+), 7 deletions(-) rename docs/CHANGELOG.md => CHANGELOG.md (99%) diff --git a/.gitignore b/.gitignore index 1091b453326..ccf8a006e7b 100644 --- a/.gitignore +++ b/.gitignore @@ -97,6 +97,3 @@ override*.toml # Pythin venv .venv/ - -# Temp Changelog migration -CHANGELOG.md diff --git a/docs/CHANGELOG.md b/CHANGELOG.md similarity index 99% rename from docs/CHANGELOG.md rename to CHANGELOG.md index b114bfb89f1..b5566c64e58 100644 --- a/docs/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,9 +1,5 @@ # Changelog Chainlink Core -All notable changes to this project will be documented in this file. - -The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), -and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html). diff --git a/README.md b/README.md index 882f408050a..0d336dd22b3 100644 --- a/README.md +++ b/README.md @@ -304,6 +304,10 @@ To install `changesets`: Either after or before you create a commit, run the `pnpm changeset` command to create an accompanying changeset entry which will reflect on the CHANGELOG for the next release. +The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), + +and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html). + ### Tips For more tips on how to build and test Chainlink, see our [development tips page](https://github.com/smartcontractkit/chainlink/wiki/Development-Tips). From e364955e22c6ad26b07ff5f92f7def7629a86f6b Mon Sep 17 00:00:00 2001 From: Cedric Date: Tue, 9 Apr 2024 17:35:06 +0100 Subject: [PATCH 3/5] [KS-136] Update staging (#12703) * Update to latest of chainlink common; this fixes a panic due to incorrect list element handling in the values library, and moves feedIds to bytes rather than uints. * Modify hardcoded workflow with the correct feedIds. * Move all initialization logic of capabilities to `ServicesForSpec` so that we can merge the mercury loop into develop. * Deterministically generate execution IDs, and move the hardcoded workflow to the delegate. --- core/scripts/go.mod | 2 +- core/scripts/go.sum | 4 +- core/services/relay/evm/cap_encoder.go | 42 ++++++--- core/services/relay/evm/cap_encoder_test.go | 51 +++++++++-- core/services/workflows/delegate.go | 88 ++++++++++++++++--- core/services/workflows/engine.go | 64 ++++++++++---- core/services/workflows/engine_test.go | 6 +- core/services/workflows/models.go | 1 + .../workflows/marshalling/workflow_1.yaml | 2 +- go.mod | 2 +- go.sum | 4 +- integration-tests/go.mod | 2 +- integration-tests/go.sum | 4 +- integration-tests/load/go.mod | 2 +- integration-tests/load/go.sum | 4 +- 15 files changed, 212 insertions(+), 66 deletions(-) diff --git a/core/scripts/go.mod b/core/scripts/go.mod index c4e32d4f276..2176664f561 100644 --- a/core/scripts/go.mod +++ b/core/scripts/go.mod @@ -21,7 +21,7 @@ require ( github.com/prometheus/client_golang v1.17.0 github.com/shopspring/decimal v1.3.1 github.com/smartcontractkit/chainlink-automation v1.0.2 - github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 + github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 github.com/smartcontractkit/chainlink-vrf v0.0.0-20231120191722-fef03814f868 github.com/smartcontractkit/chainlink/v2 v2.0.0-00010101000000-000000000000 github.com/smartcontractkit/libocr v0.0.0-20240326191951-2bbe9382d052 diff --git a/core/scripts/go.sum b/core/scripts/go.sum index 98b5142ba0a..5d74d23da68 100644 --- a/core/scripts/go.sum +++ b/core/scripts/go.sum @@ -1187,8 +1187,8 @@ github.com/smartcontractkit/chain-selectors v1.0.10 h1:t9kJeE6B6G+hKD0GYR4kGJSCq github.com/smartcontractkit/chain-selectors v1.0.10/go.mod h1:d4Hi+E1zqjy9HqMkjBE5q1vcG9VGgxf5VxiRHfzi2kE= github.com/smartcontractkit/chainlink-automation v1.0.2 h1:xsfyuswL15q2YBGQT3qn2SBz6fnSKiSW7XZ8IZQLpnI= github.com/smartcontractkit/chainlink-automation v1.0.2/go.mod h1:RjboV0Qd7YP+To+OrzHGXaxUxoSONveCoAK2TQ1INLU= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 h1:fY2wMtlr/VQxPyVVQdi1jFvQHi0VbDnGGVXzLKOZTOY= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 h1:LCVHf/ooB4HDkgfLUq+jK4CuCr6SsdNCQZt3/etJ8ms= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8 h1:I326nw5GwHQHsLKHwtu5Sb9EBLylC8CfUd7BFAS0jtg= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8/go.mod h1:a65NtrK4xZb01mf0dDNghPkN2wXgcqFQ55ADthVBgMc= github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240220203239-09be0ea34540 h1:xFSv8561jsLtF6gYZr/zW2z5qUUAkcFkApin2mnbYTo= diff --git a/core/services/relay/evm/cap_encoder.go b/core/services/relay/evm/cap_encoder.go index b6865096af9..1c7814ab16a 100644 --- a/core/services/relay/evm/cap_encoder.go +++ b/core/services/relay/evm/cap_encoder.go @@ -2,6 +2,7 @@ package evm import ( "context" + "encoding/hex" "encoding/json" "fmt" @@ -75,23 +76,36 @@ func (c *capEncoder) Encode(ctx context.Context, input values.Map) ([]byte, erro return append(append(workflowIDbytes, executionIDBytes...), userPayload...), nil } +func decodeID(input map[string]any, key string) ([]byte, error) { + id, ok := input[key].(string) + if !ok { + return nil, fmt.Errorf("expected %s to be a string", key) + } + + b, err := hex.DecodeString(id) + if err != nil { + return nil, err + } + + if len(b) < 32 { + return nil, fmt.Errorf("incorrect length for id %s, expected 32 bytes, got %d", id, len(b)) + } + + return b, nil +} + // extract workflowID and executionID from the input map, validate and align to 32 bytes // NOTE: consider requiring them to be exactly 32 bytes to avoid issues with padding func extractIDs(input map[string]any) ([]byte, []byte, error) { - workflowID, ok := input[consensustypes.WorkflowIDFieldName].(string) - if !ok { - return nil, nil, fmt.Errorf("expected %s to be a string", consensustypes.WorkflowIDFieldName) - } - executionID, ok := input[consensustypes.ExecutionIDFieldName].(string) - if !ok { - return nil, nil, fmt.Errorf("expected %s to be a string", consensustypes.ExecutionIDFieldName) + workflowID, err := decodeID(input, consensustypes.WorkflowIDFieldName) + if err != nil { + return nil, nil, err } - if len(workflowID) > 32 || len(executionID) > 32 { - return nil, nil, fmt.Errorf("IDs too long: %d, %d", len(workflowID), len(executionID)) + + executionID, err := decodeID(input, consensustypes.ExecutionIDFieldName) + if err != nil { + return nil, nil, err } - alignedWorkflowID := make([]byte, idLen) - copy(alignedWorkflowID, workflowID) - alignedExecutionID := make([]byte, idLen) - copy(alignedExecutionID, executionID) - return alignedWorkflowID, alignedExecutionID, nil + + return workflowID, executionID, nil } diff --git a/core/services/relay/evm/cap_encoder_test.go b/core/services/relay/evm/cap_encoder_test.go index 1d8b6da4610..186968df9b2 100644 --- a/core/services/relay/evm/cap_encoder_test.go +++ b/core/services/relay/evm/cap_encoder_test.go @@ -4,6 +4,7 @@ import ( "encoding/hex" "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" consensustypes "github.com/smartcontractkit/chainlink-common/pkg/capabilities/consensus/ocr3/types" @@ -13,10 +14,15 @@ import ( ) var ( - reportA = []byte{0x01, 0x02, 0x03} - reportB = []byte{0xaa, 0xbb, 0xcc, 0xdd} - workflowID = "my_id" - executionID = "my_execution_id" + reportA = []byte{0x01, 0x02, 0x03} + reportB = []byte{0xaa, 0xbb, 0xcc, 0xdd} + + // hex encoded 32 byte strings + workflowID = "15c631d295ef5e32deb99a10ee6804bc4af1385568f9b3363f6552ac6dbb2cef" + executionID = "8d4e66421db647dd916d3ec28d56188c8d7dae5f808e03d03339ed2562f13bb0" + + invalidID = "not_valid" + wrongLength = "8d4e66" ) func TestEVMEncoder(t *testing.T) { @@ -41,8 +47,8 @@ func TestEVMEncoder(t *testing.T) { expected := // start of the outer tuple ((user_fields), workflow_id, workflow_execution_id) - "6d795f6964000000000000000000000000000000000000000000000000000000" + // workflow ID - "6d795f657865637574696f6e5f69640000000000000000000000000000000000" + // execution ID + workflowID + + executionID + // start of the inner tuple (user_fields) "0000000000000000000000000000000000000000000000000000000000000020" + // offset of mercury_reports array "0000000000000000000000000000000000000000000000000000000000000002" + // length of mercury_reports array @@ -56,3 +62,36 @@ func TestEVMEncoder(t *testing.T) { require.Equal(t, expected, hex.EncodeToString(encoded)) } + +func TestEVMEncoder_InvalidIDs(t *testing.T) { + config := map[string]any{ + "abi": "mercury_reports bytes[]", + } + wrapped, err := values.NewMap(config) + require.NoError(t, err) + enc, err := evm.NewEVMEncoder(wrapped) + require.NoError(t, err) + + // output of a DF2.0 aggregator + metadata fields appended by OCR + // using an invalid ID + input := map[string]any{ + "mercury_reports": []any{reportA, reportB}, + consensustypes.WorkflowIDFieldName: invalidID, + consensustypes.ExecutionIDFieldName: executionID, + } + wrapped, err = values.NewMap(input) + require.NoError(t, err) + _, err = enc.Encode(testutils.Context(t), *wrapped) + assert.ErrorContains(t, err, "invalid byte") + + // using valid hex string of wrong length + input = map[string]any{ + "mercury_reports": []any{reportA, reportB}, + consensustypes.WorkflowIDFieldName: wrongLength, + consensustypes.ExecutionIDFieldName: executionID, + } + wrapped, err = values.NewMap(input) + require.NoError(t, err) + _, err = enc.Encode(testutils.Context(t), *wrapped) + assert.ErrorContains(t, err, "incorrect length for id") +} diff --git a/core/services/workflows/delegate.go b/core/services/workflows/delegate.go index fb9540844fa..bde7aa275c4 100644 --- a/core/services/workflows/delegate.go +++ b/core/services/workflows/delegate.go @@ -3,10 +3,13 @@ package workflows import ( "context" "fmt" + "time" "github.com/google/uuid" "github.com/pelletier/go-toml" + "github.com/smartcontractkit/chainlink-common/pkg/capabilities/mercury" + "github.com/smartcontractkit/chainlink-common/pkg/capabilities/triggers" "github.com/smartcontractkit/chainlink-common/pkg/types" "github.com/smartcontractkit/chainlink/v2/core/capabilities/targets" "github.com/smartcontractkit/chainlink/v2/core/chains/legacyevm" @@ -17,12 +20,12 @@ import ( const hardcodedWorkflow = ` triggers: - - type: "on_mercury_report" + - type: "mercury-trigger" config: - feedlist: - - "0x1111111111111111111100000000000000000000000000000000000000000000" # ETHUSD - - "0x2222222222222222222200000000000000000000000000000000000000000000" # LINKUSD - - "0x3333333333333333333300000000000000000000000000000000000000000000" # BTCUSD + feedIds: + - "0x1111111111111111111100000000000000000000000000000000000000000000" + - "0x2222222222222222222200000000000000000000000000000000000000000000" + - "0x3333333333333333333300000000000000000000000000000000000000000000" consensus: - type: "offchain_reporting" @@ -64,8 +67,9 @@ targets: ` type Delegate struct { - registry types.CapabilitiesRegistry - logger logger.Logger + registry types.CapabilitiesRegistry + logger logger.Logger + legacyEVMChains legacyevm.LegacyChainContainer } var _ job.Delegate = (*Delegate)(nil) @@ -84,10 +88,25 @@ func (d *Delegate) OnDeleteJob(ctx context.Context, jb job.Job, q pg.Queryer) er // ServicesForSpec satisfies the job.Delegate interface. func (d *Delegate) ServicesForSpec(ctx context.Context, spec job.Job) ([]job.ServiceCtx, error) { + // NOTE: we temporarily do registration inside ServicesForSpec, this will be moved out of job specs in the future + err := targets.InitializeWrite(d.registry, d.legacyEVMChains, d.logger) + if err != nil { + d.logger.Errorw("could not initialize writes", err) + } + + trigger := triggers.NewMercuryTriggerService() + err = d.registry.Add(context.Background(), trigger) + if err != nil { + d.logger.Errorw("could not add mercury trigger to registry", err) + } else { + go mercuryEventLoop(trigger, d.logger) + } + cfg := Config{ - Lggr: d.logger, - Spec: hardcodedWorkflow, - Registry: d.registry, + Lggr: d.logger, + Spec: hardcodedWorkflow, + Registry: d.registry, + WorkflowID: mockedWorkflowID, } engine, err := NewEngine(cfg) if err != nil { @@ -97,10 +116,53 @@ func (d *Delegate) ServicesForSpec(ctx context.Context, spec job.Job) ([]job.Ser } func NewDelegate(logger logger.Logger, registry types.CapabilitiesRegistry, legacyEVMChains legacyevm.LegacyChainContainer) *Delegate { - // NOTE: we temporarily do registration inside NewDelegate, this will be moved out of job specs in the future - _ = targets.InitializeWrite(registry, legacyEVMChains, logger) + return &Delegate{logger: logger, registry: registry, legacyEVMChains: legacyEVMChains} +} + +func mercuryEventLoop(trigger *triggers.MercuryTriggerService, logger logger.Logger) { + sleepSec := 60 * time.Second + ticker := time.NewTicker(sleepSec) + defer ticker.Stop() + + prices := []int64{300000, 2000, 5000000} + + for range ticker.C { + for i := range prices { + prices[i] = prices[i] + 1 + } + + t := time.Now().Round(sleepSec).Unix() + reports, err := emitReports(logger, trigger, t, prices) + if err != nil { + logger.Errorw("failed to process Mercury reports", "err", err, "timestamp", time.Now().Unix(), "payload", reports) + } + } +} + +func emitReports(logger logger.Logger, trigger *triggers.MercuryTriggerService, t int64, prices []int64) ([]triggers.FeedReport, error) { + reports := []triggers.FeedReport{ + { + FeedID: mercury.FeedID("0x1111111111111111111100000000000000000000000000000000000000000000").Bytes(), + FullReport: []byte{}, + BenchmarkPrice: prices[0], + ObservationTimestamp: t, + }, + { + FeedID: mercury.FeedID("0x2222222222222222222200000000000000000000000000000000000000000000").Bytes(), + FullReport: []byte{}, + BenchmarkPrice: prices[1], + ObservationTimestamp: t, + }, + { + FeedID: mercury.FeedID("0x3333333333333333333300000000000000000000000000000000000000000000").Bytes(), + FullReport: []byte{}, + BenchmarkPrice: prices[2], + ObservationTimestamp: t, + }, + } - return &Delegate{logger: logger, registry: registry} + logger.Infow("New set of Mercury reports", "timestamp", time.Now().Unix(), "payload", reports) + return reports, trigger.ProcessReport(reports) } func ValidatedWorkflowSpec(tomlString string) (job.Job, error) { diff --git a/core/services/workflows/engine.go b/core/services/workflows/engine.go index 8198152fb14..f3bb9554095 100644 --- a/core/services/workflows/engine.go +++ b/core/services/workflows/engine.go @@ -2,12 +2,12 @@ package workflows import ( "context" + "crypto/sha256" + "encoding/hex" "fmt" "sync" "time" - "github.com/google/uuid" - "github.com/smartcontractkit/chainlink-common/pkg/capabilities" "github.com/smartcontractkit/chainlink-common/pkg/services" "github.com/smartcontractkit/chainlink-common/pkg/types" @@ -17,9 +17,8 @@ import ( const ( // NOTE: max 32 bytes per ID - consider enforcing exactly 32 bytes? - mockedWorkflowID = "aaaaaaaaaa0000000000000000000000" - mockedExecutionID = "bbbbbbbbbb0000000000000000000000" - mockedTriggerID = "cccccccccc0000000000000000000000" + mockedTriggerID = "cccccccccc0000000000000000000000" + mockedWorkflowID = "15c631d295ef5e32deb99a10ee6804bc4af1385568f9b3363f6552ac6dbb2cef" ) // Engine handles the lifecycle of a single workflow and its executions. @@ -95,7 +94,7 @@ LOOP: return nil } - // If the capability is already cached, that means we've already registered it + // If the capability already exists, that means we've already registered it if s.capability != nil { return nil } @@ -122,14 +121,14 @@ LOOP: reg := capabilities.RegisterToWorkflowRequest{ Metadata: capabilities.RegistrationMetadata{ - WorkflowID: mockedWorkflowID, + WorkflowID: e.workflow.id, }, Config: s.config, } innerErr = cc.RegisterToWorkflow(ctx, reg) if innerErr != nil { - return fmt.Errorf("failed to register to workflow: %+v", reg) + return fmt.Errorf("failed to register to workflow (%+v): %w", reg, innerErr) } s.capability = cc @@ -177,7 +176,7 @@ func (e *Engine) registerTrigger(ctx context.Context, t *triggerCapability) erro triggerRegRequest := capabilities.CapabilityRequest{ Metadata: capabilities.RequestMetadata{ - WorkflowID: mockedWorkflowID, + WorkflowID: e.workflow.id, }, Config: tc, Inputs: triggerInputs, @@ -217,7 +216,20 @@ func (e *Engine) loop(ctx context.Context) { continue } - err := e.startExecution(ctx, resp.Value) + te := &capabilities.TriggerEvent{} + err := resp.Value.UnwrapTo(te) + if err != nil { + e.logger.Errorf("could not unwrap trigger event", resp.Err) + continue + } + + executionID, err := generateExecutionID(e.workflow.id, te.ID) + if err != nil { + e.logger.Errorf("could not generate execution ID", resp.Err) + continue + } + + err = e.startExecution(ctx, executionID, resp.Value) if err != nil { e.logger.Errorf("failed to start execution: %w", err) } @@ -245,9 +257,23 @@ func (e *Engine) loop(ctx context.Context) { } } +func generateExecutionID(workflowID, eventID string) (string, error) { + s := sha256.New() + _, err := s.Write([]byte(workflowID)) + if err != nil { + return "", err + } + + _, err = s.Write([]byte(eventID)) + if err != nil { + return "", err + } + + return hex.EncodeToString(s.Sum(nil)), nil +} + // startExecution kicks off a new workflow execution when a trigger event is received. -func (e *Engine) startExecution(ctx context.Context, event values.Value) error { - executionID := uuid.New().String() +func (e *Engine) startExecution(ctx context.Context, executionID string, event values.Value) error { e.logger.Debugw("executing on a trigger event", "event", event, "executionID", executionID) ec := &executionState{ steps: map[string]*stepState{ @@ -258,7 +284,7 @@ func (e *Engine) startExecution(ctx context.Context, event values.Value) error { status: statusCompleted, }, }, - workflowID: mockedWorkflowID, + workflowID: e.workflow.id, executionID: executionID, status: statusStarted, } @@ -375,6 +401,7 @@ func (e *Engine) queueIfReady(state executionState, step *step) { } func (e *Engine) finishExecution(ctx context.Context, executionID string, status string) error { + e.logger.Infow("finishing execution", "executionID", executionID, "status", status) err := e.executionStates.updateStatus(ctx, executionID, status) if err != nil { return err @@ -404,11 +431,11 @@ func (e *Engine) workerForStepRequest(ctx context.Context, msg stepRequest) { inputs, outputs, err := e.executeStep(ctx, msg) if err != nil { - e.logger.Errorf("error executing step request: %w", err, "executionID", msg.state.executionID, "stepRef", msg.stepRef) + e.logger.Errorf("error executing step request: %s", err, "executionID", msg.state.executionID, "stepRef", msg.stepRef) stepState.outputs.err = err stepState.status = statusErrored } else { - e.logger.Debugw("step executed successfully", "executionID", msg.state.executionID, "stepRef", msg.stepRef, "outputs", outputs) + e.logger.Infow("step executed successfully", "executionID", msg.state.executionID, "stepRef", msg.stepRef, "outputs", outputs) stepState.outputs.value = outputs stepState.status = statusCompleted } @@ -479,7 +506,7 @@ func (e *Engine) deregisterTrigger(ctx context.Context, t *triggerCapability) er } deregRequest := capabilities.CapabilityRequest{ Metadata: capabilities.RequestMetadata{ - WorkflowID: mockedWorkflowID, + WorkflowID: e.workflow.id, }, Inputs: triggerInputs, Config: t.config, @@ -511,7 +538,7 @@ func (e *Engine) Close() error { reg := capabilities.UnregisterFromWorkflowRequest{ Metadata: capabilities.RegistrationMetadata{ - WorkflowID: mockedWorkflowID, + WorkflowID: e.workflow.id, }, Config: s.config, } @@ -533,6 +560,7 @@ func (e *Engine) Close() error { type Config struct { Spec string + WorkflowID string Lggr logger.Logger Registry types.CapabilitiesRegistry MaxWorkerLimit int @@ -572,6 +600,8 @@ func NewEngine(cfg Config) (engine *Engine, err error) { return nil, err } + workflow.id = cfg.WorkflowID + // Instantiate semaphore to put a limit on the number of workers newWorkerCh := make(chan struct{}, cfg.MaxWorkerLimit) for i := 0; i < cfg.MaxWorkerLimit; i++ { diff --git a/core/services/workflows/engine_test.go b/core/services/workflows/engine_test.go index e456eefb729..57de4cb6faa 100644 --- a/core/services/workflows/engine_test.go +++ b/core/services/workflows/engine_test.go @@ -120,7 +120,7 @@ func TestEngineWithHardcodedWorkflow(t *testing.T) { const ( simpleWorkflow = ` triggers: - - type: "on_mercury_report" + - type: "mercury-trigger" config: feedlist: - "0x1111111111111111111100000000000000000000000000000000000000000000" # ETHUSD @@ -163,7 +163,7 @@ targets: func mockTrigger(t *testing.T) (capabilities.TriggerCapability, capabilities.CapabilityResponse) { mt := &mockTriggerCapability{ CapabilityInfo: capabilities.MustNewCapabilityInfo( - "on_mercury_report", + "mercury-trigger", capabilities.CapabilityTypeTrigger, "issues a trigger when a mercury report is received.", "v1.0.0", @@ -273,7 +273,7 @@ func TestEngine_ErrorsTheWorkflowIfAStepErrors(t *testing.T) { const ( multiStepWorkflow = ` triggers: - - type: "on_mercury_report" + - type: "mercury-trigger" config: feedlist: - "0x1111111111111111111100000000000000000000000000000000000000000000" # ETHUSD diff --git a/core/services/workflows/models.go b/core/services/workflows/models.go index 9c1c56d6054..e6c92a641e4 100644 --- a/core/services/workflows/models.go +++ b/core/services/workflows/models.go @@ -47,6 +47,7 @@ func (w *workflowSpec) steps() []stepDefinition { // treated differently due to their nature of being the starting // point of a workflow. type workflow struct { + id string graph.Graph[string, *step] triggers []*triggerCapability diff --git a/core/services/workflows/testdata/fixtures/workflows/marshalling/workflow_1.yaml b/core/services/workflows/testdata/fixtures/workflows/marshalling/workflow_1.yaml index 0fab758ac44..cbd33f4a90e 100644 --- a/core/services/workflows/testdata/fixtures/workflows/marshalling/workflow_1.yaml +++ b/core/services/workflows/testdata/fixtures/workflows/marshalling/workflow_1.yaml @@ -1,5 +1,5 @@ triggers: - - type: on_mercury_report@1 + - type: mercury-trigger@1 ref: report_data config: boolean_coercion: diff --git a/go.mod b/go.mod index 38f18481e66..32ba665e937 100644 --- a/go.mod +++ b/go.mod @@ -72,7 +72,7 @@ require ( github.com/shopspring/decimal v1.3.1 github.com/smartcontractkit/chain-selectors v1.0.10 github.com/smartcontractkit/chainlink-automation v1.0.2 - github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 + github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240220203239-09be0ea34540 github.com/smartcontractkit/chainlink-feeds v0.0.0-20240119021347-3c541a78cdb8 diff --git a/go.sum b/go.sum index 3bba85e2c95..628976fa1eb 100644 --- a/go.sum +++ b/go.sum @@ -1182,8 +1182,8 @@ github.com/smartcontractkit/chain-selectors v1.0.10 h1:t9kJeE6B6G+hKD0GYR4kGJSCq github.com/smartcontractkit/chain-selectors v1.0.10/go.mod h1:d4Hi+E1zqjy9HqMkjBE5q1vcG9VGgxf5VxiRHfzi2kE= github.com/smartcontractkit/chainlink-automation v1.0.2 h1:xsfyuswL15q2YBGQT3qn2SBz6fnSKiSW7XZ8IZQLpnI= github.com/smartcontractkit/chainlink-automation v1.0.2/go.mod h1:RjboV0Qd7YP+To+OrzHGXaxUxoSONveCoAK2TQ1INLU= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 h1:fY2wMtlr/VQxPyVVQdi1jFvQHi0VbDnGGVXzLKOZTOY= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 h1:LCVHf/ooB4HDkgfLUq+jK4CuCr6SsdNCQZt3/etJ8ms= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8 h1:I326nw5GwHQHsLKHwtu5Sb9EBLylC8CfUd7BFAS0jtg= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8/go.mod h1:a65NtrK4xZb01mf0dDNghPkN2wXgcqFQ55ADthVBgMc= github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240220203239-09be0ea34540 h1:xFSv8561jsLtF6gYZr/zW2z5qUUAkcFkApin2mnbYTo= diff --git a/integration-tests/go.mod b/integration-tests/go.mod index 81d6a805abd..52b5a006662 100644 --- a/integration-tests/go.mod +++ b/integration-tests/go.mod @@ -24,7 +24,7 @@ require ( github.com/segmentio/ksuid v1.0.4 github.com/slack-go/slack v0.12.2 github.com/smartcontractkit/chainlink-automation v1.0.2 - github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 + github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 github.com/smartcontractkit/chainlink-testing-framework v1.28.1 github.com/smartcontractkit/chainlink-vrf v0.0.0-20231120191722-fef03814f868 github.com/smartcontractkit/chainlink/v2 v2.0.0-00010101000000-000000000000 diff --git a/integration-tests/go.sum b/integration-tests/go.sum index 16af04e283c..4ce49bd0af4 100644 --- a/integration-tests/go.sum +++ b/integration-tests/go.sum @@ -1521,8 +1521,8 @@ github.com/smartcontractkit/chain-selectors v1.0.10 h1:t9kJeE6B6G+hKD0GYR4kGJSCq github.com/smartcontractkit/chain-selectors v1.0.10/go.mod h1:d4Hi+E1zqjy9HqMkjBE5q1vcG9VGgxf5VxiRHfzi2kE= github.com/smartcontractkit/chainlink-automation v1.0.2 h1:xsfyuswL15q2YBGQT3qn2SBz6fnSKiSW7XZ8IZQLpnI= github.com/smartcontractkit/chainlink-automation v1.0.2/go.mod h1:RjboV0Qd7YP+To+OrzHGXaxUxoSONveCoAK2TQ1INLU= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 h1:fY2wMtlr/VQxPyVVQdi1jFvQHi0VbDnGGVXzLKOZTOY= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 h1:LCVHf/ooB4HDkgfLUq+jK4CuCr6SsdNCQZt3/etJ8ms= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8 h1:I326nw5GwHQHsLKHwtu5Sb9EBLylC8CfUd7BFAS0jtg= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8/go.mod h1:a65NtrK4xZb01mf0dDNghPkN2wXgcqFQ55ADthVBgMc= github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240220203239-09be0ea34540 h1:xFSv8561jsLtF6gYZr/zW2z5qUUAkcFkApin2mnbYTo= diff --git a/integration-tests/load/go.mod b/integration-tests/load/go.mod index 4c7c2a1367a..cd321ab240a 100644 --- a/integration-tests/load/go.mod +++ b/integration-tests/load/go.mod @@ -16,7 +16,7 @@ require ( github.com/rs/zerolog v1.30.0 github.com/slack-go/slack v0.12.2 github.com/smartcontractkit/chainlink-automation v1.0.2 - github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 + github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 github.com/smartcontractkit/chainlink-testing-framework v1.28.1 github.com/smartcontractkit/chainlink/integration-tests v0.0.0-20240214231432-4ad5eb95178c github.com/smartcontractkit/chainlink/v2 v2.9.0-beta0.0.20240216210048-da02459ddad8 diff --git a/integration-tests/load/go.sum b/integration-tests/load/go.sum index c58a91e9197..71fae4d3857 100644 --- a/integration-tests/load/go.sum +++ b/integration-tests/load/go.sum @@ -1504,8 +1504,8 @@ github.com/smartcontractkit/chain-selectors v1.0.10 h1:t9kJeE6B6G+hKD0GYR4kGJSCq github.com/smartcontractkit/chain-selectors v1.0.10/go.mod h1:d4Hi+E1zqjy9HqMkjBE5q1vcG9VGgxf5VxiRHfzi2kE= github.com/smartcontractkit/chainlink-automation v1.0.2 h1:xsfyuswL15q2YBGQT3qn2SBz6fnSKiSW7XZ8IZQLpnI= github.com/smartcontractkit/chainlink-automation v1.0.2/go.mod h1:RjboV0Qd7YP+To+OrzHGXaxUxoSONveCoAK2TQ1INLU= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25 h1:fY2wMtlr/VQxPyVVQdi1jFvQHi0VbDnGGVXzLKOZTOY= -github.com/smartcontractkit/chainlink-common v0.1.7-0.20240404141006-77085a02ce25/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3 h1:LCVHf/ooB4HDkgfLUq+jK4CuCr6SsdNCQZt3/etJ8ms= +github.com/smartcontractkit/chainlink-common v0.1.7-0.20240405173118-f5bf144ec6b3/go.mod h1:kstYjAGqBswdZpl7YkSPeXBDVwaY1VaR6tUMPWl8ykA= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8 h1:I326nw5GwHQHsLKHwtu5Sb9EBLylC8CfUd7BFAS0jtg= github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240213120401-01a23955f9f8/go.mod h1:a65NtrK4xZb01mf0dDNghPkN2wXgcqFQ55ADthVBgMc= github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240220203239-09be0ea34540 h1:xFSv8561jsLtF6gYZr/zW2z5qUUAkcFkApin2mnbYTo= From a21c5f447990953fd492c91313a4759a35716180 Mon Sep 17 00:00:00 2001 From: Tate Date: Tue, 9 Apr 2024 12:04:33 -0600 Subject: [PATCH 4/5] Update Nix apple sdk usage (#12758) - the previous usage is a pattern that is being deprecated in nix - no longer need to add CGO_ENABLED=0 with this method - bumps flake.lock to get golang 1.21.9 in nix shells --- flake.lock | 12 ++++++------ flake.nix | 5 +---- shell.nix | 8 ++++++-- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/flake.lock b/flake.lock index bce30e58f58..94ed8931096 100644 --- a/flake.lock +++ b/flake.lock @@ -5,11 +5,11 @@ "systems": "systems" }, "locked": { - "lastModified": 1705309234, - "narHash": "sha256-uNRRNRKmJyCRC/8y1RqBkqWBLM034y4qN7EprSdmgyA=", + "lastModified": 1710146030, + "narHash": "sha256-SZ5L6eA7HJ/nmkzGG7/ISclqe6oZdOZTNoesiInkXPQ=", "owner": "numtide", "repo": "flake-utils", - "rev": "1ef2e671c3b0c19053962c07dbda38332dcebf26", + "rev": "b1d9ab70662946ef0850d488da1c9019f3a9752a", "type": "github" }, "original": { @@ -20,11 +20,11 @@ }, "nixpkgs": { "locked": { - "lastModified": 1707092692, - "narHash": "sha256-ZbHsm+mGk/izkWtT4xwwqz38fdlwu7nUUKXTOmm4SyE=", + "lastModified": 1712439257, + "narHash": "sha256-aSpiNepFOMk9932HOax0XwNxbA38GOUVOiXfUVPOrck=", "owner": "nixos", "repo": "nixpkgs", - "rev": "faf912b086576fd1a15fca610166c98d47bc667e", + "rev": "ff0dbd94265ac470dda06a657d5fe49de93b4599", "type": "github" }, "original": { diff --git a/flake.nix b/flake.nix index 928b165ca35..b4fd137da79 100644 --- a/flake.nix +++ b/flake.nix @@ -10,12 +10,9 @@ flake-utils.lib.eachDefaultSystem (system: let pkgs = import nixpkgs { inherit system; overlays = [ ]; }; - # The current default sdk for macOS fails to compile go projects, so we use a newer one for now. - # This has no effect on other platforms. - callPackage = pkgs.darwin.apple_sdk_11_0.callPackage or pkgs.callPackage; in rec { - devShell = callPackage ./shell.nix { + devShell = pkgs.callPackage ./shell.nix { inherit pkgs; }; formatter = pkgs.nixpkgs-fmt; diff --git a/shell.nix b/shell.nix index 7b64b7f58a1..ca785283fd5 100644 --- a/shell.nix +++ b/shell.nix @@ -5,8 +5,13 @@ let postgresql = postgresql_14; nodejs = nodejs-18_x; nodePackages = pkgs.nodePackages.override { inherit nodejs; }; + + mkShell' = mkShell.override { + # The current nix default sdk for macOS fails to compile go projects, so we use a newer one for now. + stdenv = if stdenv.isDarwin then overrideSDK stdenv "11.0" else stdenv; + }; in -mkShell { +mkShell' { nativeBuildInputs = [ go goreleaser @@ -46,7 +51,6 @@ mkShell { ]; LD_LIBRARY_PATH = "${stdenv.cc.cc.lib}/lib64:$LD_LIBRARY_PATH"; GOROOT = "${go}/share/go"; - CGO_ENABLED = 0; PGDATA = "db"; CL_DATABASE_URL = "postgresql://chainlink:chainlink@localhost:5432/chainlink_test?sslmode=disable"; From 6d37e64a17bcc952b333a4aa583415c4e726c747 Mon Sep 17 00:00:00 2001 From: Sneha Agnihotri <180277+snehaagni@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:58:29 -0700 Subject: [PATCH 5/5] Bump version and update CHANGELOG for core 2.11.0 (#12764) Signed-off-by: Sneha Agnihotri --- .changeset/afraid-baboons-nail.md | 12 -- .changeset/brave-games-drop.md | 5 - .changeset/breezy-taxis-breathe.md | 5 - .changeset/chilled-buses-reflect.md | 5 - .changeset/chilly-garlics-kneel.md | 5 - .changeset/cool-apricots-compare.md | 9 -- .changeset/dirty-weeks-shave.md | 5 - .changeset/dull-pugs-wonder.md | 5 - .changeset/famous-pets-confess.md | 5 - .changeset/few-swans-wonder.md | 5 - .changeset/fresh-oranges-brake.md | 5 - .changeset/fresh-spies-melt.md | 5 - .changeset/friendly-adults-pull.md | 5 - .changeset/friendly-coats-switch.md | 5 - .changeset/funny-poets-sneeze.md | 5 - .changeset/gentle-cups-carry.md | 5 - .changeset/giant-hotels-sparkle.md | 5 - .changeset/gold-rats-hide.md | 5 - .changeset/good-rabbits-beg.md | 5 - .changeset/gorgeous-crabs-repeat.md | 5 - .changeset/healthy-toes-destroy.md | 5 - .changeset/hot-pets-sneeze.md | 12 -- .changeset/hungry-cats-scream.md | 5 - .changeset/hungry-impalas-jog.md | 5 - .changeset/hungry-seas-attend.md | 5 - .changeset/kind-crabs-begin.md | 5 - .changeset/large-flowers-agree.md | 5 - .changeset/large-games-applaud.md | 5 - .changeset/large-oranges-warn.md | 5 - .changeset/lazy-cooks-agree.md | 5 - .changeset/lemon-balloons-pretend.md | 5 - .changeset/lemon-ladybugs-doubt.md | 5 - .changeset/little-hats-worry.md | 5 - .changeset/little-plums-grow.md | 5 - .changeset/mighty-timers-travel.md | 5 - .changeset/modern-candles-begin.md | 5 - .changeset/moody-ligers-walk.md | 5 - .changeset/nasty-humans-promise.md | 5 - .changeset/nasty-penguins-smash.md | 5 - .changeset/new-cheetahs-sell.md | 5 - .changeset/odd-mugs-divide.md | 5 - .changeset/odd-mugs-end.md | 5 - .changeset/pink-ducks-agree.md | 5 - .changeset/polite-jeans-knock.md | 5 - .changeset/poor-melons-vanish.md | 5 - .changeset/popular-buckets-hang.md | 18 --- .changeset/pretty-experts-unite.md | 7 - .changeset/pretty-fishes-jam.md | 5 - .changeset/quick-berries-sin.md | 5 - .changeset/rude-beds-change.md | 5 - .changeset/rude-falcons-beg.md | 5 - .changeset/rude-paws-cross.md | 5 - .changeset/selfish-timers-matter.md | 5 - .changeset/shaggy-pots-pretend.md | 5 - .changeset/shiny-forks-clap.md | 5 - .changeset/shy-jobs-speak.md | 5 - .changeset/silent-pets-sip.md | 5 - .changeset/silly-weeks-serve.md | 5 - .changeset/silver-months-glow.md | 5 - .changeset/sixty-turtles-rest.md | 5 - .changeset/small-beers-perform.md | 5 - .changeset/smart-kids-sip.md | 5 - .changeset/smooth-monkeys-help.md | 5 - .changeset/smooth-suits-provide.md | 5 - .changeset/spicy-horses-poke.md | 5 - .changeset/stale-sloths-drive.md | 5 - .changeset/strange-swans-compare.md | 5 - .changeset/strange-tables-occur.md | 21 --- .changeset/strong-ears-heal.md | 5 - .changeset/stupid-ducks-call.md | 5 - .changeset/swift-bobcats-punch.md | 5 - .changeset/tasty-bobcats-hammer.md | 5 - .changeset/tasty-buckets-relate.md | 5 - .changeset/ten-waves-wonder.md | 5 - .changeset/thick-apes-reply.md | 5 - .changeset/thin-coats-joke.md | 5 - .changeset/thirty-cheetahs-unite.md | 5 - .changeset/tiny-rabbits-crave.md | 5 - .changeset/tiny-suns-end.md | 5 - .changeset/twenty-zebras-joke.md | 5 - .changeset/warm-chefs-fry.md | 5 - .changeset/warm-owls-act.md | 5 - .changeset/wet-turtles-provide.md | 5 - .changeset/wicked-gorillas-sniff.md | 5 - .changeset/wicked-suits-watch.md | 5 - .changeset/wild-walls-suffer.md | 5 - .changeset/witty-jeans-wave.md | 5 - .changeset/young-deers-itch.md | 5 - CHANGELOG.md | 223 ++++++++++++++++++++++----- package.json | 2 +- 90 files changed, 182 insertions(+), 532 deletions(-) delete mode 100644 .changeset/afraid-baboons-nail.md delete mode 100644 .changeset/brave-games-drop.md delete mode 100644 .changeset/breezy-taxis-breathe.md delete mode 100644 .changeset/chilled-buses-reflect.md delete mode 100644 .changeset/chilly-garlics-kneel.md delete mode 100644 .changeset/cool-apricots-compare.md delete mode 100644 .changeset/dirty-weeks-shave.md delete mode 100644 .changeset/dull-pugs-wonder.md delete mode 100644 .changeset/famous-pets-confess.md delete mode 100644 .changeset/few-swans-wonder.md delete mode 100644 .changeset/fresh-oranges-brake.md delete mode 100644 .changeset/fresh-spies-melt.md delete mode 100644 .changeset/friendly-adults-pull.md delete mode 100644 .changeset/friendly-coats-switch.md delete mode 100644 .changeset/funny-poets-sneeze.md delete mode 100644 .changeset/gentle-cups-carry.md delete mode 100644 .changeset/giant-hotels-sparkle.md delete mode 100644 .changeset/gold-rats-hide.md delete mode 100644 .changeset/good-rabbits-beg.md delete mode 100644 .changeset/gorgeous-crabs-repeat.md delete mode 100644 .changeset/healthy-toes-destroy.md delete mode 100644 .changeset/hot-pets-sneeze.md delete mode 100644 .changeset/hungry-cats-scream.md delete mode 100644 .changeset/hungry-impalas-jog.md delete mode 100644 .changeset/hungry-seas-attend.md delete mode 100644 .changeset/kind-crabs-begin.md delete mode 100644 .changeset/large-flowers-agree.md delete mode 100644 .changeset/large-games-applaud.md delete mode 100644 .changeset/large-oranges-warn.md delete mode 100644 .changeset/lazy-cooks-agree.md delete mode 100644 .changeset/lemon-balloons-pretend.md delete mode 100644 .changeset/lemon-ladybugs-doubt.md delete mode 100644 .changeset/little-hats-worry.md delete mode 100644 .changeset/little-plums-grow.md delete mode 100644 .changeset/mighty-timers-travel.md delete mode 100644 .changeset/modern-candles-begin.md delete mode 100644 .changeset/moody-ligers-walk.md delete mode 100644 .changeset/nasty-humans-promise.md delete mode 100644 .changeset/nasty-penguins-smash.md delete mode 100644 .changeset/new-cheetahs-sell.md delete mode 100644 .changeset/odd-mugs-divide.md delete mode 100644 .changeset/odd-mugs-end.md delete mode 100644 .changeset/pink-ducks-agree.md delete mode 100644 .changeset/polite-jeans-knock.md delete mode 100644 .changeset/poor-melons-vanish.md delete mode 100644 .changeset/popular-buckets-hang.md delete mode 100644 .changeset/pretty-experts-unite.md delete mode 100644 .changeset/pretty-fishes-jam.md delete mode 100644 .changeset/quick-berries-sin.md delete mode 100644 .changeset/rude-beds-change.md delete mode 100644 .changeset/rude-falcons-beg.md delete mode 100644 .changeset/rude-paws-cross.md delete mode 100644 .changeset/selfish-timers-matter.md delete mode 100644 .changeset/shaggy-pots-pretend.md delete mode 100644 .changeset/shiny-forks-clap.md delete mode 100644 .changeset/shy-jobs-speak.md delete mode 100644 .changeset/silent-pets-sip.md delete mode 100644 .changeset/silly-weeks-serve.md delete mode 100644 .changeset/silver-months-glow.md delete mode 100644 .changeset/sixty-turtles-rest.md delete mode 100644 .changeset/small-beers-perform.md delete mode 100644 .changeset/smart-kids-sip.md delete mode 100644 .changeset/smooth-monkeys-help.md delete mode 100644 .changeset/smooth-suits-provide.md delete mode 100644 .changeset/spicy-horses-poke.md delete mode 100644 .changeset/stale-sloths-drive.md delete mode 100644 .changeset/strange-swans-compare.md delete mode 100644 .changeset/strange-tables-occur.md delete mode 100644 .changeset/strong-ears-heal.md delete mode 100644 .changeset/stupid-ducks-call.md delete mode 100644 .changeset/swift-bobcats-punch.md delete mode 100644 .changeset/tasty-bobcats-hammer.md delete mode 100644 .changeset/tasty-buckets-relate.md delete mode 100644 .changeset/ten-waves-wonder.md delete mode 100644 .changeset/thick-apes-reply.md delete mode 100644 .changeset/thin-coats-joke.md delete mode 100644 .changeset/thirty-cheetahs-unite.md delete mode 100644 .changeset/tiny-rabbits-crave.md delete mode 100644 .changeset/tiny-suns-end.md delete mode 100644 .changeset/twenty-zebras-joke.md delete mode 100644 .changeset/warm-chefs-fry.md delete mode 100644 .changeset/warm-owls-act.md delete mode 100644 .changeset/wet-turtles-provide.md delete mode 100644 .changeset/wicked-gorillas-sniff.md delete mode 100644 .changeset/wicked-suits-watch.md delete mode 100644 .changeset/wild-walls-suffer.md delete mode 100644 .changeset/witty-jeans-wave.md delete mode 100644 .changeset/young-deers-itch.md diff --git a/.changeset/afraid-baboons-nail.md b/.changeset/afraid-baboons-nail.md deleted file mode 100644 index d13bacea5e2..00000000000 --- a/.changeset/afraid-baboons-nail.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -"chainlink": patch ---- - -VerboseLogging is now turned on by default. - -You may disable if this results in excessive log volume. Disable like so: - -``` -[Pipeline] -VerboseLogging = false -``` diff --git a/.changeset/brave-games-drop.md b/.changeset/brave-games-drop.md deleted file mode 100644 index 5dee59fd9f2..00000000000 --- a/.changeset/brave-games-drop.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Fix kv_store migration fk cascade deletion diff --git a/.changeset/breezy-taxis-breathe.md b/.changeset/breezy-taxis-breathe.md deleted file mode 100644 index 79ce1ae96bd..00000000000 --- a/.changeset/breezy-taxis-breathe.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add json schema support to workflows diff --git a/.changeset/chilled-buses-reflect.md b/.changeset/chilled-buses-reflect.md deleted file mode 100644 index eccac3b7f5b..00000000000 --- a/.changeset/chilled-buses-reflect.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Dispatcher service for external peering diff --git a/.changeset/chilly-garlics-kneel.md b/.changeset/chilly-garlics-kneel.md deleted file mode 100644 index fc8b9425250..00000000000 --- a/.changeset/chilly-garlics-kneel.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Fix error log formatting for in memory data source cache for juels fee per coin diff --git a/.changeset/cool-apricots-compare.md b/.changeset/cool-apricots-compare.md deleted file mode 100644 index 945a3ffa390..00000000000 --- a/.changeset/cool-apricots-compare.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -"chainlink": patch ---- - -Increase default config for postgres max open conns from 20 to 100. - -Also, add autoscaling for mercury jobs. The max open conns limit will be -automatically increased to the number of mercury jobs if this exceeds the -configured value. diff --git a/.changeset/dirty-weeks-shave.md b/.changeset/dirty-weeks-shave.md deleted file mode 100644 index 9fcb0c39ab1..00000000000 --- a/.changeset/dirty-weeks-shave.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -update AutomationBase interface to check for ready only address on polygon zkEVM diff --git a/.changeset/dull-pugs-wonder.md b/.changeset/dull-pugs-wonder.md deleted file mode 100644 index f750db9f62c..00000000000 --- a/.changeset/dull-pugs-wonder.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Update config for zkevm polygon chains diff --git a/.changeset/famous-pets-confess.md b/.changeset/famous-pets-confess.md deleted file mode 100644 index 583b17a1603..00000000000 --- a/.changeset/famous-pets-confess.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Provides support for dynamic pipeline runs with persistence. diff --git a/.changeset/few-swans-wonder.md b/.changeset/few-swans-wonder.md deleted file mode 100644 index d6c3be39653..00000000000 --- a/.changeset/few-swans-wonder.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -small gas fix diff --git a/.changeset/fresh-oranges-brake.md b/.changeset/fresh-oranges-brake.md deleted file mode 100644 index 52562ee7413..00000000000 --- a/.changeset/fresh-oranges-brake.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -fix jfpc cache cleanup diff --git a/.changeset/fresh-spies-melt.md b/.changeset/fresh-spies-melt.md deleted file mode 100644 index ad341d1db91..00000000000 --- a/.changeset/fresh-spies-melt.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Update automation smoke test to use UpkeepCounter with time based counter diff --git a/.changeset/friendly-adults-pull.md b/.changeset/friendly-adults-pull.md deleted file mode 100644 index 5b74f367115..00000000000 --- a/.changeset/friendly-adults-pull.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -change auto 2.3 flat fees from link to USD diff --git a/.changeset/friendly-coats-switch.md b/.changeset/friendly-coats-switch.md deleted file mode 100644 index 3ae97c51519..00000000000 --- a/.changeset/friendly-coats-switch.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -safeTransfer and cleanups diff --git a/.changeset/funny-poets-sneeze.md b/.changeset/funny-poets-sneeze.md deleted file mode 100644 index 214ba4504a6..00000000000 --- a/.changeset/funny-poets-sneeze.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Remove LogPoller filters for outdated Functions coordinator contracts diff --git a/.changeset/gentle-cups-carry.md b/.changeset/gentle-cups-carry.md deleted file mode 100644 index 1b204dfee31..00000000000 --- a/.changeset/gentle-cups-carry.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -added logic C contract to automation 2.3 diff --git a/.changeset/giant-hotels-sparkle.md b/.changeset/giant-hotels-sparkle.md deleted file mode 100644 index 817078ae3cc..00000000000 --- a/.changeset/giant-hotels-sparkle.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -remove registerUpkeep from auto v21 common diff --git a/.changeset/gold-rats-hide.md b/.changeset/gold-rats-hide.md deleted file mode 100644 index b290847556a..00000000000 --- a/.changeset/gold-rats-hide.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -External peering core service diff --git a/.changeset/good-rabbits-beg.md b/.changeset/good-rabbits-beg.md deleted file mode 100644 index 91b5d54150f..00000000000 --- a/.changeset/good-rabbits-beg.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -implement offchain settlement for NOPs payment diff --git a/.changeset/gorgeous-crabs-repeat.md b/.changeset/gorgeous-crabs-repeat.md deleted file mode 100644 index a74f36ec3a9..00000000000 --- a/.changeset/gorgeous-crabs-repeat.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Soft delete consumer nonce in VRF coordinator v2.5 diff --git a/.changeset/healthy-toes-destroy.md b/.changeset/healthy-toes-destroy.md deleted file mode 100644 index 1c027fdcd01..00000000000 --- a/.changeset/healthy-toes-destroy.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -HeadTracker now respects the `FinalityTagEnabled` config option. If the flag is enabled, HeadTracker backfills blocks up to the latest finalized block provided by the corresponding RPC call. To address potential misconfigurations, `HistoryDepth` is now calculated from the latest finalized block instead of the head. NOTE: Consumers (e.g. TXM and LogPoller) do not fully utilize Finality Tag yet. diff --git a/.changeset/hot-pets-sneeze.md b/.changeset/hot-pets-sneeze.md deleted file mode 100644 index b60e7d7cde8..00000000000 --- a/.changeset/hot-pets-sneeze.md +++ /dev/null @@ -1,12 +0,0 @@ ---- -"chainlink": minor ---- - -- Misc VRF V2+ contract changes - - Reuse struct RequestCommitmentV2Plus from VRFTypes - - Fix interface name IVRFCoordinatorV2PlusFulfill in BatchVRFCoordinatorV2Plus to avoid confusion with IVRFCoordinatorV2Plus.sol - - Remove unused errors - - Rename variables for readability - - Fix comments - - Minor gas optimisation (++i) -- Fix integration tests diff --git a/.changeset/hungry-cats-scream.md b/.changeset/hungry-cats-scream.md deleted file mode 100644 index 2c9f66115f3..00000000000 --- a/.changeset/hungry-cats-scream.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -The `xdai` `ChainType` has been renamed to `gnosis` to match the chain's new name. The old value is still supported but has been deprecated and will be removed in v2.13.0. diff --git a/.changeset/hungry-impalas-jog.md b/.changeset/hungry-impalas-jog.md deleted file mode 100644 index efa23edabb2..00000000000 --- a/.changeset/hungry-impalas-jog.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Added a tx simulation feature to the chain client to enable testing for zk out-of-counter (OOC) errors diff --git a/.changeset/hungry-seas-attend.md b/.changeset/hungry-seas-attend.md deleted file mode 100644 index 1b6af484f8f..00000000000 --- a/.changeset/hungry-seas-attend.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -refactor foundry tests for auto 2.3 diff --git a/.changeset/kind-crabs-begin.md b/.changeset/kind-crabs-begin.md deleted file mode 100644 index 4718b21f126..00000000000 --- a/.changeset/kind-crabs-begin.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Helper VRF CLI command diff --git a/.changeset/large-flowers-agree.md b/.changeset/large-flowers-agree.md deleted file mode 100644 index 9f12ab42a65..00000000000 --- a/.changeset/large-flowers-agree.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Update keyvalue store to be compatible with the interface required in chainlink common diff --git a/.changeset/large-games-applaud.md b/.changeset/large-games-applaud.md deleted file mode 100644 index c6c0b3bf6f9..00000000000 --- a/.changeset/large-games-applaud.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -VRFV2PlusWrapper contract: subID param added to the constructor, removed migrate() method diff --git a/.changeset/large-oranges-warn.md b/.changeset/large-oranges-warn.md deleted file mode 100644 index db29d9b5d77..00000000000 --- a/.changeset/large-oranges-warn.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Adds prometheus metrics for automation streams error handling diff --git a/.changeset/lazy-cooks-agree.md b/.changeset/lazy-cooks-agree.md deleted file mode 100644 index 923d2404428..00000000000 --- a/.changeset/lazy-cooks-agree.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Chainlink Functions contracts v1.3 audit findings diff --git a/.changeset/lemon-balloons-pretend.md b/.changeset/lemon-balloons-pretend.md deleted file mode 100644 index 0cb7b41d3a2..00000000000 --- a/.changeset/lemon-balloons-pretend.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Added a RageP2P wrapper diff --git a/.changeset/lemon-ladybugs-doubt.md b/.changeset/lemon-ladybugs-doubt.md deleted file mode 100644 index d7d1c7a8492..00000000000 --- a/.changeset/lemon-ladybugs-doubt.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add kv store tied to jobs and use it for juels fee per coin cache to store persisted values for backup diff --git a/.changeset/little-hats-worry.md b/.changeset/little-hats-worry.md deleted file mode 100644 index eb3e86e153a..00000000000 --- a/.changeset/little-hats-worry.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Change LimitTransfer gasLimit type from uint32 to uint64 diff --git a/.changeset/little-plums-grow.md b/.changeset/little-plums-grow.md deleted file mode 100644 index fa362d2dc59..00000000000 --- a/.changeset/little-plums-grow.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -improve foundry tests and fix nits diff --git a/.changeset/mighty-timers-travel.md b/.changeset/mighty-timers-travel.md deleted file mode 100644 index 95dbb735b15..00000000000 --- a/.changeset/mighty-timers-travel.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -make reserveAmounts to be a map diff --git a/.changeset/modern-candles-begin.md b/.changeset/modern-candles-begin.md deleted file mode 100644 index 933c1749d03..00000000000 --- a/.changeset/modern-candles-begin.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -billing overrides diff --git a/.changeset/moody-ligers-walk.md b/.changeset/moody-ligers-walk.md deleted file mode 100644 index c93bf8517ee..00000000000 --- a/.changeset/moody-ligers-walk.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add new pipeline for testing EVM node compatibility on go-ethereum dependency bump diff --git a/.changeset/nasty-humans-promise.md b/.changeset/nasty-humans-promise.md deleted file mode 100644 index 8a366df1bae..00000000000 --- a/.changeset/nasty-humans-promise.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -add pending request counter for vrf v2.5 coordinator diff --git a/.changeset/nasty-penguins-smash.md b/.changeset/nasty-penguins-smash.md deleted file mode 100644 index 620e8068e08..00000000000 --- a/.changeset/nasty-penguins-smash.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add GetFilters function to the log_poller diff --git a/.changeset/new-cheetahs-sell.md b/.changeset/new-cheetahs-sell.md deleted file mode 100644 index 28ae6d81da1..00000000000 --- a/.changeset/new-cheetahs-sell.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Add table support to "type" property for step definitions diff --git a/.changeset/odd-mugs-divide.md b/.changeset/odd-mugs-divide.md deleted file mode 100644 index 8498593c6eb..00000000000 --- a/.changeset/odd-mugs-divide.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Set LINK native feed in VRFV2PlusWrapper to immutable diff --git a/.changeset/odd-mugs-end.md b/.changeset/odd-mugs-end.md deleted file mode 100644 index 7dba6199ce7..00000000000 --- a/.changeset/odd-mugs-end.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -address TODOs and docs for 2.3 diff --git a/.changeset/pink-ducks-agree.md b/.changeset/pink-ducks-agree.md deleted file mode 100644 index 0b1035c4d84..00000000000 --- a/.changeset/pink-ducks-agree.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add OCR3 capability contract wrapper diff --git a/.changeset/polite-jeans-knock.md b/.changeset/polite-jeans-knock.md deleted file mode 100644 index 69ec1715211..00000000000 --- a/.changeset/polite-jeans-knock.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Remote Trigger diff --git a/.changeset/poor-melons-vanish.md b/.changeset/poor-melons-vanish.md deleted file mode 100644 index 3b6d901b157..00000000000 --- a/.changeset/poor-melons-vanish.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Add the `pool_rpc_node_highest_finalized_block` metric that tracks the highest finalized block seen per RPC. If `FinalityTagEnabled = true`, a positive `NodePool.FinalizedBlockPollInterval` is needed to collect the metric. If the finality tag is not enabled, the metric is populated with a calculated latest finalized block based on the latest head and finality depth. diff --git a/.changeset/popular-buckets-hang.md b/.changeset/popular-buckets-hang.md deleted file mode 100644 index a80b4c90052..00000000000 --- a/.changeset/popular-buckets-hang.md +++ /dev/null @@ -1,18 +0,0 @@ ---- -"chainlink": patch ---- - -Add new config option Pipeline.VerboseLogging - -VerboseLogging enables detailed logging of pipeline execution steps. This is -disabled by default because it increases log volume for pipeline runs, but can -be useful for debugging failed runs without relying on the UI or database. -Consider enabling this if you disabled run saving by setting MaxSuccessfulRuns -to zero. - -Set it like the following example: - -``` -[Pipeline] -VerboseLogging = true -``` diff --git a/.changeset/pretty-experts-unite.md b/.changeset/pretty-experts-unite.md deleted file mode 100644 index 4a1f903d439..00000000000 --- a/.changeset/pretty-experts-unite.md +++ /dev/null @@ -1,7 +0,0 @@ ---- -"chainlink": patch ---- - -Added log buffer v1 with improved performance, stability and control over scaling parameters. - -Added a feature flag for using log buffer v1. diff --git a/.changeset/pretty-fishes-jam.md b/.changeset/pretty-fishes-jam.md deleted file mode 100644 index 6026bb27971..00000000000 --- a/.changeset/pretty-fishes-jam.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -VRF V2+ Coordinator msg.data len validation diff --git a/.changeset/quick-berries-sin.md b/.changeset/quick-berries-sin.md deleted file mode 100644 index e8c348a3561..00000000000 --- a/.changeset/quick-berries-sin.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -fix bug in auto2.3 withdrawERC20Fees diff --git a/.changeset/rude-beds-change.md b/.changeset/rude-beds-change.md deleted file mode 100644 index baf3e04216a..00000000000 --- a/.changeset/rude-beds-change.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -more auto 2.3 tests diff --git a/.changeset/rude-falcons-beg.md b/.changeset/rude-falcons-beg.md deleted file mode 100644 index 4957d31c407..00000000000 --- a/.changeset/rude-falcons-beg.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Calculate blockRate and logLimit defaults in the log provider based on chain ID diff --git a/.changeset/rude-paws-cross.md b/.changeset/rude-paws-cross.md deleted file mode 100644 index 395a6d76244..00000000000 --- a/.changeset/rude-paws-cross.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -L1Oracle handles OP Stack Ecotone encoded l1 gas price diff --git a/.changeset/selfish-timers-matter.md b/.changeset/selfish-timers-matter.md deleted file mode 100644 index cb598191ccc..00000000000 --- a/.changeset/selfish-timers-matter.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Refactor Log and TxStore ORMs diff --git a/.changeset/shaggy-pots-pretend.md b/.changeset/shaggy-pots-pretend.md deleted file mode 100644 index 644986ddb56..00000000000 --- a/.changeset/shaggy-pots-pretend.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Add error log if juels fee per coin cache is over 24h old and lower other logs severity in cache to warn diff --git a/.changeset/shiny-forks-clap.md b/.changeset/shiny-forks-clap.md deleted file mode 100644 index 4718b21f126..00000000000 --- a/.changeset/shiny-forks-clap.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Helper VRF CLI command diff --git a/.changeset/shy-jobs-speak.md b/.changeset/shy-jobs-speak.md deleted file mode 100644 index 1b1c3b4c91b..00000000000 --- a/.changeset/shy-jobs-speak.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -use common interface for v2.3 diff --git a/.changeset/silent-pets-sip.md b/.changeset/silent-pets-sip.md deleted file mode 100644 index ba2417f0922..00000000000 --- a/.changeset/silent-pets-sip.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Exposing information about LogPoller finality violation via Healthy method. It's raised whenever LogPoller sees reorg deeper than the finality diff --git a/.changeset/silly-weeks-serve.md b/.changeset/silly-weeks-serve.md deleted file mode 100644 index 0f7386e69fe..00000000000 --- a/.changeset/silly-weeks-serve.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -update starknet relayer to fix nonce issue. introduces optional api-key for starknet toml config. diff --git a/.changeset/silver-months-glow.md b/.changeset/silver-months-glow.md deleted file mode 100644 index 195525353fc..00000000000 --- a/.changeset/silver-months-glow.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Fixed a race condition bug around EVM nonce management, which could cause the Node to skip a nonce and get stuck. diff --git a/.changeset/sixty-turtles-rest.md b/.changeset/sixty-turtles-rest.md deleted file mode 100644 index 6fa4e551809..00000000000 --- a/.changeset/sixty-turtles-rest.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Add rebalancer support for feeds manager ocr2 plugins diff --git a/.changeset/small-beers-perform.md b/.changeset/small-beers-perform.md deleted file mode 100644 index a420116a44e..00000000000 --- a/.changeset/small-beers-perform.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Making LogPoller's replay more robust by backfilling up to finalized block and processing rest in the main loop diff --git a/.changeset/smart-kids-sip.md b/.changeset/smart-kids-sip.md deleted file mode 100644 index f5e290c5530..00000000000 --- a/.changeset/smart-kids-sip.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -native support diff --git a/.changeset/smooth-monkeys-help.md b/.changeset/smooth-monkeys-help.md deleted file mode 100644 index 23e44dd3032..00000000000 --- a/.changeset/smooth-monkeys-help.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -upgraded transmission to 0.8.19 diff --git a/.changeset/smooth-suits-provide.md b/.changeset/smooth-suits-provide.md deleted file mode 100644 index aefafb54ad3..00000000000 --- a/.changeset/smooth-suits-provide.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -docs: remove repeated words in documentation and comments diff --git a/.changeset/spicy-horses-poke.md b/.changeset/spicy-horses-poke.md deleted file mode 100644 index 982d425782d..00000000000 --- a/.changeset/spicy-horses-poke.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Updating prometheus metrics for Automation log triggers diff --git a/.changeset/stale-sloths-drive.md b/.changeset/stale-sloths-drive.md deleted file mode 100644 index e0394de0404..00000000000 --- a/.changeset/stale-sloths-drive.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -update solc version for vrf v2.5 coordinators diff --git a/.changeset/strange-swans-compare.md b/.changeset/strange-swans-compare.md deleted file mode 100644 index a5690cc5d93..00000000000 --- a/.changeset/strange-swans-compare.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -remove trailing slash diff --git a/.changeset/strange-tables-occur.md b/.changeset/strange-tables-occur.md deleted file mode 100644 index 68a39e43b54..00000000000 --- a/.changeset/strange-tables-occur.md +++ /dev/null @@ -1,21 +0,0 @@ ---- -"chainlink": patch ---- - -Mercury jobs can now broadcast to multiple mercury servers. - -Previously, a single mercury server would be specified in a job spec as so: - -```toml -[pluginConfig] -serverURL = "example.com/foo" -serverPubKey = "724ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93" -``` - -You may now specify multiple mercury servers, as so: - -```toml -[pluginConfig] -servers = { "example.com/foo" = "724ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93", "mercury2.example:1234/bar" = "524ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93" } -``` - diff --git a/.changeset/strong-ears-heal.md b/.changeset/strong-ears-heal.md deleted file mode 100644 index b6332407ea5..00000000000 --- a/.changeset/strong-ears-heal.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Refactor EVM ORMs to remove pg dependency diff --git a/.changeset/stupid-ducks-call.md b/.changeset/stupid-ducks-call.md deleted file mode 100644 index 9aae500e3fd..00000000000 --- a/.changeset/stupid-ducks-call.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -increase num optimizations to 500 for vrf v2.5 coordinator diff --git a/.changeset/swift-bobcats-punch.md b/.changeset/swift-bobcats-punch.md deleted file mode 100644 index 80de89c87cc..00000000000 --- a/.changeset/swift-bobcats-punch.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -add liquidity pool for automation 2.3 diff --git a/.changeset/tasty-bobcats-hammer.md b/.changeset/tasty-bobcats-hammer.md deleted file mode 100644 index 69ffb6c1bcb..00000000000 --- a/.changeset/tasty-bobcats-hammer.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Remove noisy log poller warning in VRFv2 & VRFv2+ listener loops diff --git a/.changeset/tasty-buckets-relate.md b/.changeset/tasty-buckets-relate.md deleted file mode 100644 index a627e392e82..00000000000 --- a/.changeset/tasty-buckets-relate.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Validation for premium limits added to VRFCoordinatorV2_5 contract diff --git a/.changeset/ten-waves-wonder.md b/.changeset/ten-waves-wonder.md deleted file mode 100644 index 301a48109a8..00000000000 --- a/.changeset/ten-waves-wonder.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Remove pg from evm tests diff --git a/.changeset/thick-apes-reply.md b/.changeset/thick-apes-reply.md deleted file mode 100644 index 83a0232d7bb..00000000000 --- a/.changeset/thick-apes-reply.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -bug fixes in s_reserveAmount accounting diff --git a/.changeset/thin-coats-joke.md b/.changeset/thin-coats-joke.md deleted file mode 100644 index 0cb6a0851e2..00000000000 --- a/.changeset/thin-coats-joke.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -fix withdraw LINK bug in auto 2.3 diff --git a/.changeset/thirty-cheetahs-unite.md b/.changeset/thirty-cheetahs-unite.md deleted file mode 100644 index 616f553c49d..00000000000 --- a/.changeset/thirty-cheetahs-unite.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -bump grafana to 1.1.1 diff --git a/.changeset/tiny-rabbits-crave.md b/.changeset/tiny-rabbits-crave.md deleted file mode 100644 index 55b6f71c523..00000000000 --- a/.changeset/tiny-rabbits-crave.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -Handle zkSync specific known transaction error diff --git a/.changeset/tiny-suns-end.md b/.changeset/tiny-suns-end.md deleted file mode 100644 index 3bdd12ea362..00000000000 --- a/.changeset/tiny-suns-end.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -add test for billing override diff --git a/.changeset/twenty-zebras-joke.md b/.changeset/twenty-zebras-joke.md deleted file mode 100644 index 354d112e468..00000000000 --- a/.changeset/twenty-zebras-joke.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -enable gas tests for auto 2.3 diff --git a/.changeset/warm-chefs-fry.md b/.changeset/warm-chefs-fry.md deleted file mode 100644 index 054dc56655c..00000000000 --- a/.changeset/warm-chefs-fry.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": patch ---- - -add version support for automation registry 2.\* diff --git a/.changeset/warm-owls-act.md b/.changeset/warm-owls-act.md deleted file mode 100644 index 22b674e7418..00000000000 --- a/.changeset/warm-owls-act.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Add support for eth_getLogs & finality tags in simulated_backend_client.go diff --git a/.changeset/wet-turtles-provide.md b/.changeset/wet-turtles-provide.md deleted file mode 100644 index 6a26eb52d12..00000000000 --- a/.changeset/wet-turtles-provide.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Copy common transmitter methods into FunctionsContractTransmitter to enable product specific modification diff --git a/.changeset/wicked-gorillas-sniff.md b/.changeset/wicked-gorillas-sniff.md deleted file mode 100644 index 7efb85aa18e..00000000000 --- a/.changeset/wicked-gorillas-sniff.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -VRFV2PlusWrapper config refactor diff --git a/.changeset/wicked-suits-watch.md b/.changeset/wicked-suits-watch.md deleted file mode 100644 index b4caf929a7a..00000000000 --- a/.changeset/wicked-suits-watch.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Extracted Gas Limit Multiplier from gas estimators to WrappedEvmEstimator. \ No newline at end of file diff --git a/.changeset/wild-walls-suffer.md b/.changeset/wild-walls-suffer.md deleted file mode 100644 index 7573d354806..00000000000 --- a/.changeset/wild-walls-suffer.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Validate if flat fee configs are configured correctly diff --git a/.changeset/witty-jeans-wave.md b/.changeset/witty-jeans-wave.md deleted file mode 100644 index e2a386384ab..00000000000 --- a/.changeset/witty-jeans-wave.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -Remove 0.6 and 0.7 Solidity source code diff --git a/.changeset/young-deers-itch.md b/.changeset/young-deers-itch.md deleted file mode 100644 index 8486595c4d0..00000000000 --- a/.changeset/young-deers-itch.md +++ /dev/null @@ -1,5 +0,0 @@ ---- -"chainlink": minor ---- - -pay deactivated transmitters in offchain settlement diff --git a/CHANGELOG.md b/CHANGELOG.md index b5566c64e58..bb21a1a904a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,16 +1,147 @@ # Changelog Chainlink Core +## 2.11.0 - UNRELEASED - +### Minor Changes -## [dev] +- [#12348](https://github.com/smartcontractkit/chainlink/pull/12348) [`efead72965`](https://github.com/smartcontractkit/chainlink/commit/efead72965fec7e822a16f4d50cc0e5a27dd4640) Thanks [@reductionista](https://github.com/reductionista)! - Update config for zkevm polygon chains + +- [#12082](https://github.com/smartcontractkit/chainlink/pull/12082) [`608ea0a467`](https://github.com/smartcontractkit/chainlink/commit/608ea0a467ee36e15fdc654a88494ae579d778a6) Thanks [@dhaidashenko](https://github.com/dhaidashenko)! - HeadTracker now respects the `FinalityTagEnabled` config option. If the flag is enabled, HeadTracker backfills blocks up to the latest finalized block provided by the corresponding RPC call. To address potential misconfigurations, `HistoryDepth` is now calculated from the latest finalized block instead of the head. NOTE: Consumers (e.g. TXM and LogPoller) do not fully utilize Finality Tag yet. + +- [#12489](https://github.com/smartcontractkit/chainlink/pull/12489) [`3a49094db2`](https://github.com/smartcontractkit/chainlink/commit/3a49094db25036e1948818e4030fca11be748914) Thanks [@kidambisrinivas](https://github.com/kidambisrinivas)! - - Misc VRF V2+ contract changes + + - Reuse struct RequestCommitmentV2Plus from VRFTypes + - Fix interface name IVRFCoordinatorV2PlusFulfill in BatchVRFCoordinatorV2Plus to avoid confusion with IVRFCoordinatorV2Plus.sol + - Remove unused errors + - Rename variables for readability + - Fix comments + - Minor gas optimisation (++i) + - Fix integration tests + +- [#12093](https://github.com/smartcontractkit/chainlink/pull/12093) [`3f6d901fe6`](https://github.com/smartcontractkit/chainlink/commit/3f6d901fe676698769cb6713250152e322747145) Thanks [@friedemannf](https://github.com/friedemannf)! - The `xdai` `ChainType` has been renamed to `gnosis` to match the chain's new name. The old value is still supported but has been deprecated and will be removed in v2.13.0. + +- [#12503](https://github.com/smartcontractkit/chainlink/pull/12503) [`dc224a2924`](https://github.com/smartcontractkit/chainlink/commit/dc224a29249c83c74a38d9ca9d16fb00e192a4e2) Thanks [@amit-momin](https://github.com/amit-momin)! - Added a tx simulation feature to the chain client to enable testing for zk out-of-counter (OOC) errors + +- [#12634](https://github.com/smartcontractkit/chainlink/pull/12634) [`e9e903bf4b`](https://github.com/smartcontractkit/chainlink/commit/e9e903bf4b34099f8b274eb1e0f013b4ab326bb4) Thanks [@ettec](https://github.com/ettec)! - Update keyvalue store to be compatible with the interface required in chainlink common + +- [#12496](https://github.com/smartcontractkit/chainlink/pull/12496) [`31350477ae`](https://github.com/smartcontractkit/chainlink/commit/31350477ae51f00e035b1b8c50775e5955258ac1) Thanks [@silaslenihan](https://github.com/silaslenihan)! - Change LimitTransfer gasLimit type from uint32 to uint64 + +- [#12622](https://github.com/smartcontractkit/chainlink/pull/12622) [`a2bdcf51ef`](https://github.com/smartcontractkit/chainlink/commit/a2bdcf51efc1efbbac21745b260bd130dbb1ce3d) Thanks [@HenryNguyen5](https://github.com/HenryNguyen5)! - Add table support to "type" property for step definitions + +- [#12339](https://github.com/smartcontractkit/chainlink/pull/12339) [`96d2fe13b8`](https://github.com/smartcontractkit/chainlink/commit/96d2fe13b8510631bbc92ffd20a4d923b93002e6) Thanks [@dhaidashenko](https://github.com/dhaidashenko)! - Add the `pool_rpc_node_highest_finalized_block` metric that tracks the highest finalized block seen per RPC. If `FinalityTagEnabled = true`, a positive `NodePool.FinalizedBlockPollInterval` is needed to collect the metric. If the finality tag is not enabled, the metric is populated with a calculated latest finalized block based on the latest head and finality depth. + +- [#12473](https://github.com/smartcontractkit/chainlink/pull/12473) [`f1d1f249eb`](https://github.com/smartcontractkit/chainlink/commit/f1d1f249ebecb37da7eacbc4cc12e1eb0205f29a) Thanks [@justinkaseman](https://github.com/justinkaseman)! - Copy common transmitter methods into FunctionsContractTransmitter to enable product specific modification + +- [#12534](https://github.com/smartcontractkit/chainlink/pull/12534) [`bd532b5e2a`](https://github.com/smartcontractkit/chainlink/commit/bd532b5e2a1bebe8c9fe689d059464c43365ced1) Thanks [@silaslenihan](https://github.com/silaslenihan)! - Extracted Gas Limit Multiplier from gas estimators to WrappedEvmEstimator. + +- [#12578](https://github.com/smartcontractkit/chainlink/pull/12578) [`ffd492295f`](https://github.com/smartcontractkit/chainlink/commit/ffd492295f03de8c3b946a003dacbded731d7899) Thanks [@RensR](https://github.com/RensR)! - Remove 0.6 and 0.7 Solidity source code + + +### Patch Changes + +- [#12570](https://github.com/smartcontractkit/chainlink/pull/12570) [`2d33524a35`](https://github.com/smartcontractkit/chainlink/commit/2d33524a3539e32ac32a84c4600e6cdfb8e01cf3) Thanks [@samsondav](https://github.com/samsondav)! - VerboseLogging is now turned on by default. + + You may disable if this results in excessive log volume. Disable like so: + + ``` + [Pipeline] + VerboseLogging = false + ``` + +- [#12458](https://github.com/smartcontractkit/chainlink/pull/12458) [`51b134700a`](https://github.com/smartcontractkit/chainlink/commit/51b134700afe6daa1a10692e6365fdbbaf3b1396) Thanks [@HenryNguyen5](https://github.com/HenryNguyen5)! - Add json schema support to workflows + +- [#12697](https://github.com/smartcontractkit/chainlink/pull/12697) [`33398b7945`](https://github.com/smartcontractkit/chainlink/commit/33398b7945bd522ef67329d89a4140d88277db0b) Thanks [@samsondav](https://github.com/samsondav)! - Increase default config for postgres max open conns from 20 to 100. + + Also, add autoscaling for mercury jobs. The max open conns limit will be + automatically increased to the number of mercury jobs if this exceeds the + configured value. + +- [#12540](https://github.com/smartcontractkit/chainlink/pull/12540) [`17c037678d`](https://github.com/smartcontractkit/chainlink/commit/17c037678d05c88f28a28a3ac760c742f549d5ec) Thanks [@RyanRHall](https://github.com/RyanRHall)! - change auto 2.3 flat fees from link to USD + +- [#12696](https://github.com/smartcontractkit/chainlink/pull/12696) [`ee52be7cf9`](https://github.com/smartcontractkit/chainlink/commit/ee52be7cf90076677b1e96e3b294f284e798194f) Thanks [@KuphJr](https://github.com/KuphJr)! - Remove LogPoller filters for outdated Functions coordinator contracts + +- [#12405](https://github.com/smartcontractkit/chainlink/pull/12405) [`2bd210bfa8`](https://github.com/smartcontractkit/chainlink/commit/2bd210bfa8c4705b0981a315cba939b0281d7bf3) Thanks [@jinhoonbang](https://github.com/jinhoonbang)! - Soft delete consumer nonce in VRF coordinator v2.5 + +- [#12387](https://github.com/smartcontractkit/chainlink/pull/12387) [`42e72d2d26`](https://github.com/smartcontractkit/chainlink/commit/42e72d2d2610d2481c5a9469fc9b49c167d37f79) Thanks [@ogtownsend](https://github.com/ogtownsend)! - Adds prometheus metrics for automation streams error handling + +- [#12388](https://github.com/smartcontractkit/chainlink/pull/12388) [`30b73a804d`](https://github.com/smartcontractkit/chainlink/commit/30b73a804dfba394180abe354569dade80a71be5) Thanks [@justinkaseman](https://github.com/justinkaseman)! - Chainlink Functions contracts v1.3 audit findings + +- [#12332](https://github.com/smartcontractkit/chainlink/pull/12332) [`89abd726b6`](https://github.com/smartcontractkit/chainlink/commit/89abd726b6c3f29a84e0fc5d230a1324f622755b) Thanks [@Tofel](https://github.com/Tofel)! - Add new pipeline for testing EVM node compatibility on go-ethereum dependency bump + +- [#12621](https://github.com/smartcontractkit/chainlink/pull/12621) [`9c2764adbf`](https://github.com/smartcontractkit/chainlink/commit/9c2764adbf3969654795ed2c35c5fb56eaf70785) Thanks [@KuphJr](https://github.com/KuphJr)! - Add GetFilters function to the log_poller + +- [#12592](https://github.com/smartcontractkit/chainlink/pull/12592) [`b512ef5a7d`](https://github.com/smartcontractkit/chainlink/commit/b512ef5a7d1bc87d0cbd5357c5c47cc0dcb75e0b) Thanks [@ibrajer](https://github.com/ibrajer)! - Set LINK native feed in VRFV2PlusWrapper to immutable + +- [#12404](https://github.com/smartcontractkit/chainlink/pull/12404) [`b74079b672`](https://github.com/smartcontractkit/chainlink/commit/b74079b672f36fb0c241f90ea1e875ea3a9524da) Thanks [@HenryNguyen5](https://github.com/HenryNguyen5)! - Add OCR3 capability contract wrapper + +- [#12498](https://github.com/smartcontractkit/chainlink/pull/12498) [`1c576d0e34`](https://github.com/smartcontractkit/chainlink/commit/1c576d0e34d93a6298ddcb662ee89fd04eeda53e) Thanks [@samsondav](https://github.com/samsondav)! - Add new config option Pipeline.VerboseLogging + + VerboseLogging enables detailed logging of pipeline execution steps. This is + disabled by default because it increases log volume for pipeline runs, but can + be useful for debugging failed runs without relying on the UI or database. + Consider enabling this if you disabled run saving by setting MaxSuccessfulRuns + to zero. + + Set it like the following example: + + ``` + [Pipeline] + VerboseLogging = true + ``` + +- [#12357](https://github.com/smartcontractkit/chainlink/pull/12357) [`a532accd6a`](https://github.com/smartcontractkit/chainlink/commit/a532accd6ad56195e77a0314b613a1e2b5d4d07a) Thanks [@amirylm](https://github.com/amirylm)! - Added log buffer v1 with improved performance, stability and control over scaling parameters. + + Added a feature flag for using log buffer v1. + +- [#12152](https://github.com/smartcontractkit/chainlink/pull/12152) [`a6a2acfe20`](https://github.com/smartcontractkit/chainlink/commit/a6a2acfe2017dc766d401d55627f0c5016c824b9) Thanks [@ferglor](https://github.com/ferglor)! - Calculate blockRate and logLimit defaults in the log provider based on chain ID + +- [#12584](https://github.com/smartcontractkit/chainlink/pull/12584) [`c7cacd0710`](https://github.com/smartcontractkit/chainlink/commit/c7cacd0710f5040a46532e6dae7eac1b9eafe645) Thanks [@matYang](https://github.com/matYang)! - L1Oracle handles OP Stack Ecotone encoded l1 gas price + +- [#12564](https://github.com/smartcontractkit/chainlink/pull/12564) [`246762ceeb`](https://github.com/smartcontractkit/chainlink/commit/246762ceebba7923641ec00e66ae1aaf59bbcdc2) Thanks [@mateusz-sekara](https://github.com/mateusz-sekara)! - Exposing information about LogPoller finality violation via Healthy method. It's raised whenever LogPoller sees reorg deeper than the finality + +- [#12575](https://github.com/smartcontractkit/chainlink/pull/12575) [`23254c4bf5`](https://github.com/smartcontractkit/chainlink/commit/23254c4bf577e84b71bda1d9a8b2c11e7b548267) Thanks [@augustbleeds](https://github.com/augustbleeds)! - update starknet relayer to fix nonce issue. introduces optional api-key for starknet toml config. + +- [#12353](https://github.com/smartcontractkit/chainlink/pull/12353) [`07c9f6cadd`](https://github.com/smartcontractkit/chainlink/commit/07c9f6cadd449989b21977af461305ded8e5b2f0) Thanks [@amit-momin](https://github.com/amit-momin)! - Fixed a race condition bug around EVM nonce management, which could cause the Node to skip a nonce and get stuck. + +- [#12344](https://github.com/smartcontractkit/chainlink/pull/12344) [`6fa1f5dddc`](https://github.com/smartcontractkit/chainlink/commit/6fa1f5dddc6e257c2223503f1592297ca69521bd) Thanks [@eutopian](https://github.com/eutopian)! - Add rebalancer support for feeds manager ocr2 plugins + +- [#12484](https://github.com/smartcontractkit/chainlink/pull/12484) [`590cad6126`](https://github.com/smartcontractkit/chainlink/commit/590cad61269c75a6b22be1f6a73c74adfd1baa40) Thanks [@mateusz-sekara](https://github.com/mateusz-sekara)! - Making LogPoller's replay more robust by backfilling up to finalized block and processing rest in the main loop + +- [#12612](https://github.com/smartcontractkit/chainlink/pull/12612) [`d44abe3769`](https://github.com/smartcontractkit/chainlink/commit/d44abe37693d6995377fa1329e433e7fba26885d) Thanks [@RensR](https://github.com/RensR)! - upgraded transmission to 0.8.19 + +- [#12444](https://github.com/smartcontractkit/chainlink/pull/12444) [`dde7fdff33`](https://github.com/smartcontractkit/chainlink/commit/dde7fdff33cfc0690844cf0a88295bef57e2a269) Thanks [@ogtownsend](https://github.com/ogtownsend)! - Updating prometheus metrics for Automation log triggers + +- [#12479](https://github.com/smartcontractkit/chainlink/pull/12479) [`93762ccbd8`](https://github.com/smartcontractkit/chainlink/commit/93762ccbd868b9e227abf3220afb9ad22ba41b92) Thanks [@jinhoonbang](https://github.com/jinhoonbang)! - update solc version for vrf v2.5 coordinators + +- [#12337](https://github.com/smartcontractkit/chainlink/pull/12337) [`195b504a93`](https://github.com/smartcontractkit/chainlink/commit/195b504a93b1a241c1981ec21726e4b722d40b2b) Thanks [@samsondav](https://github.com/samsondav)! - Mercury jobs can now broadcast to multiple mercury servers. + + Previously, a single mercury server would be specified in a job spec as so: + + ```toml + [pluginConfig] + serverURL = "example.com/foo" + serverPubKey = "724ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93" + ``` + + You may now specify multiple mercury servers, as so: + + ```toml + [pluginConfig] + servers = { "example.com/foo" = "724ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93", "mercury2.example:1234/bar" = "524ff6eae9e900270edfff233e16322a70ec06e1a6e62a81ef13921f398f6c93" } + ``` + +- [#11899](https://github.com/smartcontractkit/chainlink/pull/11899) [`67560b9f1d`](https://github.com/smartcontractkit/chainlink/commit/67560b9f1dc052712a76eeb245fba12f2daf8e8d) Thanks [@DylanTinianov](https://github.com/DylanTinianov)! - Refactor EVM ORMs to remove pg dependency + +- [#12531](https://github.com/smartcontractkit/chainlink/pull/12531) [`88e010d604`](https://github.com/smartcontractkit/chainlink/commit/88e010d604682c54c4f99e0a0916f94c0d13ece6) Thanks [@jinhoonbang](https://github.com/jinhoonbang)! - increase num optimizations to 500 for vrf v2.5 coordinator + +- [#12375](https://github.com/smartcontractkit/chainlink/pull/12375) [`831aea819d`](https://github.com/smartcontractkit/chainlink/commit/831aea819dd6b3415770cc927c4857a1da4557b5) Thanks [@shileiwill](https://github.com/shileiwill)! - add liquidity pool for automation 2.3 + +- [#12412](https://github.com/smartcontractkit/chainlink/pull/12412) [`83c8688a14`](https://github.com/smartcontractkit/chainlink/commit/83c8688a14ac04111f999d132673ebaf6a364b4a) Thanks [@poopoothegorilla](https://github.com/poopoothegorilla)! - bump grafana to 1.1.1 + +- [#12248](https://github.com/smartcontractkit/chainlink/pull/12248) [`e1950769ee`](https://github.com/smartcontractkit/chainlink/commit/e1950769ee3ff2a40ca5772b9634c45f8be241cc) Thanks [@FelixFan1992](https://github.com/FelixFan1992)! - add version support for automation registry 2.\* -### Changed -- HeadTracker now respects the `FinalityTagEnabled` config option. If the flag is enabled, HeadTracker backfills blocks up to the latest finalized block provided by the corresponding RPC call. To address potential misconfigurations, `HistoryDepth` is now calculated from the latest finalized block instead of the head. NOTE: Consumers (e.g. TXM and LogPoller) do not fully utilize Finality Tag yet. -... - ## 2.10.0 - 2024-04-05 @@ -49,7 +180,7 @@ - Environment variables `CL_MEDIAN_ENV`, `CL_SOLANA_ENV`, and `CL_STARKNET_ENV` for setting environment variables in LOOP Plugins with an `.env` file. ``` echo "Foo=Bar" >> median.env - echo "Baz=Val" >> median.env + echo "Baz=Val" >> median.env CL_MEDIAN_ENV="median.env" ``` @@ -61,6 +192,7 @@ - `P2P.V1` is no longer supported and must not be set in TOML configuration in order to boot. Use `P2P.V2` instead. If you are using both, `V1` can simply be removed. - Removed `TelemetryIngress.URL` and `TelemetryIngress.ServerPubKey` from TOML configuration, these fields are replaced by `[[TelemetryIngress.Endpoints]]`: + ```toml [[TelemetryIngress.Endpoints]] Network = '...' # e.g. EVM. Solana, Starknet, Cosmos @@ -76,45 +208,45 @@ - Added distributed tracing in the OpenTelemetry trace format to the node, currently focused at the LOOPP Plugin development effort. This includes a new set of `Tracing` TOML configurations. The default for collecting traces is off - you must explicitly enable traces and setup a valid OpenTelemetry collector. Refer to `.github/tracing/README.md` for more details. - Added a new, optional WebServer authentication option that supports LDAP as a user identity provider. This enables user login access and user roles to be managed and provisioned via a centralized remote server that supports the LDAP protocol, which can be helpful when running multiple nodes. See the documentation for more information and config setup instructions. There is a new `[WebServer].AuthenticationMethod` config option, when set to `ldap` requires the new `[WebServer.LDAP]` config section to be defined, see the reference `docs/core.toml`. - New prom metrics for mercury transmit queue: - `mercury_transmit_queue_delete_error_count` - `mercury_transmit_queue_insert_error_count` - `mercury_transmit_queue_push_error_count` - Nops should consider alerting on these. + `mercury_transmit_queue_delete_error_count` + `mercury_transmit_queue_insert_error_count` + `mercury_transmit_queue_push_error_count` + Nops should consider alerting on these. - Mercury now implements a local cache for fetching prices for fees, which ought to reduce latency and load on the mercury server, as well as increasing performance. It is enabled by default and can be configured with the following new config variables: - ``` - [Mercury] - - # Mercury.Cache controls settings for the price retrieval cache querying a mercury server - [Mercury.Cache] - # LatestReportTTL controls how "stale" we will allow a price to be e.g. if - # set to 1s, a new price will always be fetched if the last result was - # from 1 second ago or older. - # - # Another way of looking at it is such: the cache will _never_ return a - # price that was queried from now-LatestReportTTL or before. - # - # Setting to zero disables caching entirely. - LatestReportTTL = "1s" # Default - # MaxStaleAge is that maximum amount of time that a value can be stale - # before it is deleted from the cache (a form of garbage collection). - # - # This should generally be set to something much larger than - # LatestReportTTL. Setting to zero disables garbage collection. - MaxStaleAge = "1h" # Default - # LatestReportDeadline controls how long to wait for a response from the - # mercury server before retrying. Setting this to zero will wait indefinitely. - LatestReportDeadline = "5s" # Default - ``` + ``` + [Mercury] + + # Mercury.Cache controls settings for the price retrieval cache querying a mercury server + [Mercury.Cache] + # LatestReportTTL controls how "stale" we will allow a price to be e.g. if + # set to 1s, a new price will always be fetched if the last result was + # from 1 second ago or older. + # + # Another way of looking at it is such: the cache will _never_ return a + # price that was queried from now-LatestReportTTL or before. + # + # Setting to zero disables caching entirely. + LatestReportTTL = "1s" # Default + # MaxStaleAge is that maximum amount of time that a value can be stale + # before it is deleted from the cache (a form of garbage collection). + # + # This should generally be set to something much larger than + # LatestReportTTL. Setting to zero disables garbage collection. + MaxStaleAge = "1h" # Default + # LatestReportDeadline controls how long to wait for a response from the + # mercury server before retrying. Setting this to zero will wait indefinitely. + LatestReportDeadline = "5s" # Default + ``` - New prom metrics for the mercury cache: - `mercury_cache_fetch_failure_count` - `mercury_cache_hit_count` - `mercury_cache_wait_count` - `mercury_cache_miss_count` + `mercury_cache_fetch_failure_count` + `mercury_cache_hit_count` + `mercury_cache_wait_count` + `mercury_cache_miss_count` - Added new `EVM.OCR` TOML config fields `DeltaCOverride` and `DeltaCJitterOverride` for overriding the config DeltaC. - Mercury v0.2 has improved consensus around current block that uses the most recent 5 blocks instead of only the latest one - Two new prom metrics for mercury, nops should consider adding alerting on these: - - `mercury_insufficient_blocks_count` - - `mercury_zero_blocks_count` + - `mercury_insufficient_blocks_count` + - `mercury_zero_blocks_count` - Added new `Mercury.TLS` TOML config field `CertFile` for configuring transport credentials when the node acts as a client and initiates a TLS handshake. ### Changed @@ -146,7 +278,9 @@ - Fixed a bug that caused the Telemetry Manager to report incorrect health ### Upcoming Required Configuration Changes + Starting in `v2.9.0`: + - `TelemetryIngress.URL` and `TelemetryIngress.ServerPubKey` will no longer be allowed. Any TOML configuration that sets this fields will prevent the node from booting. These fields will be replaced by `[[TelemetryIngress.Endpoints]]` - `P2P.V1` will no longer be supported and must not be set in TOML configuration in order to boot. Use `P2P.V2` instead. If you are using both, `V1` can simply be removed. @@ -154,7 +288,7 @@ Starting in `v2.9.0`: ### Fixed -- Fixed a bug that caused nodes without OCR or OCR2 enabled to fail config validation if `P2P.V2` was not explicitly disabled. With this fix, NOPs will not have to make changes to their config. +- Fixed a bug that caused nodes without OCR or OCR2 enabled to fail config validation if `P2P.V2` was not explicitly disabled. With this fix, NOPs will not have to make changes to their config. ## 2.7.1 - 2023-11-21 @@ -168,6 +302,7 @@ Starting in `v2.9.0`: - Added new configuration field named `LeaseDuration` for `EVM.NodePool` that will periodically check if internal subscriptions are connected to the "best" (as defined by the `SelectionMode`) node and switch to it if necessary. Setting this value to `0s` will disable this feature. - Added multichain telemetry support. Each network/chainID pair must be configured using the new fields: + ```toml [[TelemetryIngress.Endpoints]] Network = '...' # e.g. EVM. Solana, Starknet, Cosmos @@ -175,7 +310,9 @@ ChainID = '...' # e.g. 1, 5, devnet, mainnet-beta URL = '...' ServerPubKey = '...' ``` + These will eventually replace `TelemetryIngress.URL` and `TelemetryIngress.ServerPubKey`. Setting `TelemetryIngress.URL` and `TelemetryIngress.ServerPubKey` alongside `[[TelemetryIngress.Endpoints]]` will prevent the node from booting. Only one way of configuring telemetry endpoints is supported. + - Added bridge_name label to `pipeline_tasks_total_finished` prometheus metric. This should make it easier to see directly what bridge was failing out from the CL NODE perspective. - LogPoller will now use finality tags to dynamically determine finality on evm chains if `EVM.FinalityTagEnabled=true`, rather than the fixed `EVM.FinalityDepth` specified in toml config @@ -186,7 +323,9 @@ These will eventually replace `TelemetryIngress.URL` and `TelemetryIngress.Serve - `P2P.V2` is now enabled (`Enabled = true`) by default. ### Upcoming Required Configuration Changes + Starting in `v2.9.0`: + - `TelemetryIngress.URL` and `TelemetryIngress.ServerPubKey` will no longer be allowed. Any TOML configuration that sets this fields will prevent the node from booting. These fields will be replaced by `[[TelemetryIngress.Endpoints]]` - `P2P.V1` will no longer be supported and must not be set in TOML configuration in order to boot. Use `P2P.V2` instead. If you are using both, `V1` can simply be removed. diff --git a/package.json b/package.json index 7ee2aaba64b..8ec8161f573 100644 --- a/package.json +++ b/package.json @@ -1,6 +1,6 @@ { "name": "chainlink", - "version": "2.10.0", + "version": "2.11.0", "description": "node of the decentralized oracle network, bridging on and off-chain computation", "main": "index.js", "private": true,